WIP [VOL-2811] - Incorporate preliminary onu-adapter-go code into opencord repo

- functionality: reason "initial-mib-downloaded" reached
- omciAgent removed, object relation restructured
- adapter code restructuring + internal lib interface update

Change-Id: Ie53450a5e2c093aaaf68fe652f95e7b4c4a014c7
Signed-off-by: Holger Hildebrandt <holger.hildebrandt@adtran.com>
diff --git a/VERSION b/VERSION
index db68d9a..7a8795e 100644
--- a/VERSION
+++ b/VERSION
@@ -1 +1 @@
-0.0.26-dev
+0.1.3-dev
diff --git a/adaptercoreont/device_handler.go b/adaptercoreont/device_handler.go
deleted file mode 100644
index c6e81ad..0000000
--- a/adaptercoreont/device_handler.go
+++ /dev/null
@@ -1,661 +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 adaptercoreont provides the utility for onu devices, flows and statistics
-package adaptercoreont
-
-import (
-	"context"
-	"encoding/hex"
-	"errors"
-	"fmt"
-	"sync"
-	"time"
-
-	"github.com/gogo/protobuf/proto"
-	"github.com/golang/protobuf/ptypes"
-	"github.com/opencord/voltha-lib-go/v3/pkg/adapters/adapterif"
-	"github.com/opencord/voltha-lib-go/v3/pkg/log"
-	ic "github.com/opencord/voltha-protos/v3/go/inter_container"
-	oop "github.com/opencord/voltha-protos/v3/go/openolt"
-	"github.com/opencord/voltha-protos/v3/go/voltha"
-)
-
-/*
-// Constants for number of retries and for timeout
-const (
-	MaxRetry       = 10
-	MaxTimeOutInMs = 500
-)
-*/
-
-//DeviceHandler will interact with the ONU ? device.
-type DeviceHandler struct {
-	deviceID         string
-	DeviceType       string
-	adminState       string
-	device           *voltha.Device
-	logicalDeviceID  string
-	ProxyAddressID   string
-	ProxyAddressType string
-
-	coreProxy     adapterif.CoreProxy
-	AdapterProxy  adapterif.AdapterProxy
-	EventProxy    adapterif.EventProxy
-	openOnuAc     *OpenONUAC
-	transitionMap *TransitionMap
-	omciAgent     *OpenOMCIAgent
-	ponPort       *voltha.Port
-	onuOmciDevice *OnuDeviceEntry
-	exitChannel   chan int
-	lockDevice    sync.RWMutex
-
-	//Client        oop.OpenoltClient
-	//clientCon     *grpc.ClientConn
-	//flowMgr       *OpenOltFlowMgr
-	//eventMgr      *OpenOltEventMgr
-	//resourceMgr   *rsrcMgr.OpenOltResourceMgr
-
-	//discOnus sync.Map
-	//onus     sync.Map
-	//portStats          *OpenOltStatisticsMgr
-	//metrics            *pmmetrics.PmMetrics
-	stopCollector      chan bool
-	stopHeartbeatCheck chan bool
-	activePorts        sync.Map
-}
-
-/*
-//OnuDevice represents ONU related info
-type OnuDevice struct {
-	deviceID      string
-	deviceType    string
-	serialNumber  string
-	onuID         uint32
-	intfID        uint32
-	proxyDeviceID string
-	uniPorts      map[uint32]struct{}
-}
-
-//NewOnuDevice creates a new Onu Device
-func NewOnuDevice(devID, deviceTp, serialNum string, onuID, intfID uint32, proxyDevID string) *OnuDevice {
-	var device OnuDevice
-	device.deviceID = devID
-	device.deviceType = deviceTp
-	device.serialNumber = serialNum
-	device.onuID = onuID
-	device.intfID = intfID
-	device.proxyDeviceID = proxyDevID
-	device.uniPorts = make(map[uint32]struct{})
-	return &device
-}
-*/
-
-//NewDeviceHandler creates a new device handler
-func NewDeviceHandler(cp adapterif.CoreProxy, ap adapterif.AdapterProxy, ep adapterif.EventProxy, device *voltha.Device, adapter *OpenONUAC) *DeviceHandler {
-	var dh DeviceHandler
-	dh.coreProxy = cp
-	dh.AdapterProxy = ap
-	dh.EventProxy = ep
-	cloned := (proto.Clone(device)).(*voltha.Device)
-	dh.deviceID = cloned.Id
-	dh.DeviceType = cloned.Type
-	dh.adminState = "up"
-	dh.device = cloned
-	dh.openOnuAc = adapter
-	dh.transitionMap = nil
-	dh.exitChannel = make(chan int, 1)
-	dh.lockDevice = sync.RWMutex{}
-	dh.stopCollector = make(chan bool, 2)
-	dh.stopHeartbeatCheck = make(chan bool, 2)
-	//dh.metrics = pmmetrics.NewPmMetrics(cloned.Id, pmmetrics.Frequency(150), pmmetrics.FrequencyOverride(false), pmmetrics.Grouped(false), pmmetrics.Metrics(pmNames))
-	dh.activePorts = sync.Map{}
-	//TODO initialize the support classes.
-	return &dh
-}
-
-// start save the device to the data model
-func (dh *DeviceHandler) Start(ctx context.Context) {
-	dh.lockDevice.Lock()
-	defer dh.lockDevice.Unlock()
-	log.Debugw("starting-device-handler", log.Fields{"device": dh.device, "deviceId": dh.deviceID})
-	// Add the initial device to the local model
-	log.Debug("device-handler-started")
-}
-
-// stop stops the device dh.  Not much to do for now
-func (dh *DeviceHandler) stop(ctx context.Context) {
-	dh.lockDevice.Lock()
-	defer dh.lockDevice.Unlock()
-	log.Debug("stopping-device-agent")
-	dh.exitChannel <- 1
-	log.Debug("device-agent-stopped")
-}
-
-// ##########################################################################################
-// DeviceHandler methods that implement the adapters interface requests ##### begin #########
-
-//AdoptDevice adopts the OLT device
-func (dh *DeviceHandler) AdoptDevice(ctx context.Context, device *voltha.Device) {
-	log.Debugw("Adopt_device", log.Fields{"deviceID": device.Id, "Address": device.GetHostAndPort()})
-
-	if dh.transitionMap == nil {
-		dh.transitionMap = NewTransitionMap(dh)
-		dh.transitionMap.Handle(ctx, DeviceInit)
-	} else {
-		log.Debug("AdoptDevice: Agent/device init already done")
-	}
-
-	/*
-		// Now, set the initial PM configuration for that device
-		if err := dh.coreProxy.DevicePMConfigUpdate(nil, dh.metrics.ToPmConfigs()); err != nil {
-			log.Errorw("error-updating-PMs", log.Fields{"deviceId": device.Id, "error": err})
-		}
-
-		go startCollector(dh)
-		go startHeartbeatCheck(dh)
-	*/
-}
-
-//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(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
-	log.Debugw("InterAdapter message header", log.Fields{"msgID": msgID, "msgType": msgType,
-		"fromTopic": fromTopic, "toTopic": toTopic, "toDeviceID": toDeviceID, "proxyDeviceID": proxyDeviceID})
-
-	switch msgType {
-	case ic.InterAdapterMessageType_OMCI_REQUEST:
-		{
-			/* TOBECHECKED: I assume, ONU Adapter receives the message hier already 'unmarshalled'? else: (howTo?)*/
-			msgBody := msg.GetBody()
-
-			omciMsg := &ic.InterAdapterOmciMessage{}
-			if err := ptypes.UnmarshalAny(msgBody, omciMsg); err != nil {
-				log.Warnw("cannot-unmarshal-omci-msg-body", log.Fields{"error": err})
-				return err
-			}
-
-			//assuming omci message content is hex coded!
-			// with restricted output of 16(?) bytes would be ...omciMsg.Message[:16]
-			log.Debugw("inter-adapter-recv-omci",
-				log.Fields{"RxOmciMessage": hex.EncodeToString(omciMsg.Message)})
-			//receive_message(omci_msg.message)
-			return dh.onuOmciDevice.PDevOmciCC.ReceiveMessage(context.TODO(), omciMsg.Message)
-		}
-	case ic.InterAdapterMessageType_ONU_IND_REQUEST:
-		{
-			/* TOBECHECKED: I assume, ONU Adapter receives the message hier already 'unmarshalled'? else: see above omci block */
-			msgBody := msg.GetBody()
-
-			onu_indication := &oop.OnuIndication{}
-			if err := ptypes.UnmarshalAny(msgBody, onu_indication); err != nil {
-				log.Warnw("cannot-unmarshal-onu-indication-msg-body", log.Fields{"error": err})
-				return err
-			}
-
-			onu_operstate := onu_indication.GetOperState()
-			log.Debugw("inter-adapter-recv-onu-ind", log.Fields{"OnuId": onu_indication.GetOnuId(),
-				"AdminState": onu_indication.GetAdminState(), "OperState": onu_operstate,
-				"SNR": onu_indication.GetSerialNumber()})
-
-			//interface related functioons might be error checked ....
-			if onu_operstate == "up" {
-				dh.create_interface(onu_indication)
-			} else if (onu_operstate == "down") || (onu_operstate == "unreachable") {
-				dh.update_interface(onu_indication)
-			} else {
-				log.Errorw("unknown-onu-indication operState", log.Fields{"OnuId": onu_indication.GetOnuId()})
-				return errors.New("InvalidOperState")
-			}
-		}
-	default:
-		{
-			log.Errorw("inter-adapter-unhandled-type", log.Fields{"msgType": msg.Header.Type})
-			return errors.New("unimplemented")
-		}
-	}
-
-	/* form py code:
-	   elif request.header.type == InterAdapterMessageType.TECH_PROFILE_DOWNLOAD_REQUEST:
-	       tech_msg = InterAdapterTechProfileDownloadMessage()
-	       request.body.Unpack(tech_msg)
-	       self.log.debug('inter-adapter-recv-tech-profile', tech_msg=tech_msg)
-
-	       self.load_and_configure_tech_profile(tech_msg.uni_id, tech_msg.path)
-
-	   elif request.header.type == InterAdapterMessageType.DELETE_GEM_PORT_REQUEST:
-	       del_gem_msg = InterAdapterDeleteGemPortMessage()
-	       request.body.Unpack(del_gem_msg)
-	       self.log.debug('inter-adapter-recv-del-gem', gem_del_msg=del_gem_msg)
-
-	       self.delete_tech_profile(uni_id=del_gem_msg.uni_id,
-	                                gem_port_id=del_gem_msg.gem_port_id,
-	                                tp_path=del_gem_msg.tp_path)
-
-	   elif request.header.type == InterAdapterMessageType.DELETE_TCONT_REQUEST:
-	       del_tcont_msg = InterAdapterDeleteTcontMessage()
-	       request.body.Unpack(del_tcont_msg)
-	       self.log.debug('inter-adapter-recv-del-tcont', del_tcont_msg=del_tcont_msg)
-
-	       self.delete_tech_profile(uni_id=del_tcont_msg.uni_id,
-	                                alloc_id=del_tcont_msg.alloc_id,
-	                                tp_path=del_tcont_msg.tp_path)
-	   else:
-	       self.log.error("inter-adapter-unhandled-type", request=request)
-	*/
-	return nil
-}
-
-//  DeviceHandler methods that implement the adapters interface requests## end #########
-// #####################################################################################
-
-// ################  to be updated acc. needs of ONU Device ########################
-// DeviceHandler StateMachine related state transition methods ##### begin #########
-
-// doStateInit provides the device update to the core
-func (dh *DeviceHandler) doStateInit(ctx context.Context) error {
-	log.Debug("doStateInit-started")
-	/*
-		var err error
-		dh.clientCon, err = grpc.Dial(dh.device.GetHostAndPort(), grpc.WithInsecure(), grpc.WithBlock())
-		if err != nil {
-			log.Errorw("Failed to dial device", log.Fields{"DeviceId": dh.deviceID, "HostAndPort": dh.device.GetHostAndPort(), "err": err})
-			return err
-		}
-		return nil
-	*/
-	// populate what we know.  rest comes later after mib sync
-	dh.device.Root = false
-	dh.device.Vendor = "OpenONU"
-	dh.device.Model = "go"
-	dh.device.Reason = "activating-onu"
-	dh.logicalDeviceID = dh.deviceID
-
-	dh.coreProxy.DeviceUpdate(ctx, dh.device)
-
-	// store proxy parameters for later communication - assumption: invariant, else they have to be requested dynamically!!
-	dh.ProxyAddressID = dh.device.ProxyAddress.GetDeviceId()
-	dh.ProxyAddressType = dh.device.ProxyAddress.GetDeviceType()
-	log.Debugw("device-updated", log.Fields{"deviceID": dh.deviceID, "proxyAddressID": dh.ProxyAddressID,
-		"proxyAddressType": dh.ProxyAddressType, "SNR": dh.device.SerialNumber,
-		"ParentId": dh.device.ParentId, "ParentPortNo": dh.device.ParentPortNo})
-
-	/*
-		self._pon = PonPort.create(self, self._pon_port_number)
-		self._pon.add_peer(self.parent_id, self._pon_port_number)
-		self.log.debug('adding-pon-port-to-agent',
-				   type=self._pon.get_port().type,
-				   admin_state=self._pon.get_port().admin_state,
-				   oper_status=self._pon.get_port().oper_status,
-				   )
-	*/
-	log.Debug("adding-pon-port")
-	ponPortNo := uint32(1)
-	if dh.device.ParentPortNo != 0 {
-		ponPortNo = dh.device.ParentPortNo
-	}
-
-	ponPort := &voltha.Port{
-		PortNo:     ponPortNo,
-		Label:      fmt.Sprintf("pon-%d", ponPortNo),
-		Type:       voltha.Port_PON_ONU,
-		OperStatus: voltha.OperStatus_ACTIVE,
-		Peers: []*voltha.Port_PeerPort{{DeviceId: dh.device.ParentId, // Peer device  is OLT
-			PortNo: dh.device.ParentPortNo}}, // Peer port is parent's port number
-	}
-	var err error
-	if err = dh.coreProxy.PortCreated(context.TODO(), dh.deviceID, ponPort); err != nil {
-		log.Fatalf("PortCreated-failed-%s", err)
-	}
-
-	log.Debug("doStateInit-done")
-	return nil
-}
-
-// postInit setups the DeviceEntry for the conerned device
-func (dh *DeviceHandler) postInit(ctx context.Context) error {
-	/*
-		dh.Client = oop.NewOpenoltClient(dh.clientCon)
-		dh.transitionMap.Handle(ctx, GrpcConnected)
-		return nil
-	*/
-	//start the Agent object with no specific FSM setting
-	dh.omciAgent = NewOpenOMCIAgent(ctx, dh.coreProxy, dh.AdapterProxy)
-	dh.omciAgent.Start(ctx)
-	// might be updated with some error handling !!!
-	dh.onuOmciDevice, _ = dh.omciAgent.Add_device(ctx, dh.deviceID, dh)
-	//dh.transitionMap.Handle(ctx, GrpcConnected)
-
-	/*
-			############################################################################
-			# Setup Alarm handler
-			self.events = AdapterEvents(self.core_proxy, device.id, self.logical_device_id,
-										device.serial_number)
-			############################################################################
-			# Setup PM configuration for this device
-			# Pass in ONU specific options
-			kwargs = {
-				OnuPmMetrics.DEFAULT_FREQUENCY_KEY: OnuPmMetrics.DEFAULT_ONU_COLLECTION_FREQUENCY,
-				'heartbeat': self.heartbeat,
-				OnuOmciPmMetrics.OMCI_DEV_KEY: self._onu_omci_device
-			}
-			self.log.debug('create-pm-metrics', device_id=device.id, serial_number=device.serial_number)
-			self._pm_metrics = OnuPmMetrics(self.events, self.core_proxy, self.device_id,
-										   self.logical_device_id, device.serial_number,
-										   grouped=True, freq_override=False, **kwargs)
-			pm_config = self._pm_metrics.make_proto()
-			self._onu_omci_device.set_pm_config(self._pm_metrics.omci_pm.openomci_interval_pm)
-			self.log.info("initial-pm-config", device_id=device.id, serial_number=device.serial_number)
-			yield self.core_proxy.device_pm_config_update(pm_config, init=True)
-
-			# Note, ONU ID and UNI intf set in add_uni_port method
-			self._onu_omci_device.alarm_synchronizer.set_alarm_params(mgr=self.events,
-																	  ani_ports=[self._pon])
-
-			# Code to Run OMCI Test Action
-			kwargs_omci_test_action = {
-				OmciTestRequest.DEFAULT_FREQUENCY_KEY:
-					OmciTestRequest.DEFAULT_COLLECTION_FREQUENCY
-			}
-			serial_number = device.serial_number
-			self._test_request = OmciTestRequest(self.core_proxy,
-										   self.omci_agent, self.device_id,
-										   AniG, serial_number,
-										   self.logical_device_id,
-										   exclusive=False,
-										   **kwargs_omci_test_action)
-
-			self.enabled = True
-		else:
-			self.log.info('onu-already-activated')
-	*/
-
-	return nil
-}
-
-// doStateUp handle the onu up indication and update to voltha core
-func (dh *DeviceHandler) doStateUp(ctx context.Context) error {
-	/*
-		// Synchronous call to update device state - this method is run in its own go routine
-		if err := dh.coreProxy.DeviceStateUpdate(ctx, dh.device.Id, voltha.ConnectStatus_REACHABLE,
-			voltha.OperStatus_ACTIVE); err != nil {
-			log.Errorw("Failed to update device with OLT UP indication", log.Fields{"deviceID": dh.device.Id, "error": err})
-			return err
-		}
-		return nil
-	*/
-	return errors.New("unimplemented")
-}
-
-// doStateDown handle the onu down indication
-func (dh *DeviceHandler) doStateDown(ctx context.Context) error {
-	dh.lockDevice.Lock()
-	defer dh.lockDevice.Unlock()
-	log.Debug("do-state-down-start")
-
-	device, err := dh.coreProxy.GetDevice(ctx, dh.device.Id, dh.device.Id)
-	if err != nil || device == nil {
-		/*TODO: needs to handle error scenarios */
-		log.Errorw("Failed to fetch device device", log.Fields{"err": err})
-		return errors.New("failed to fetch device device")
-	}
-
-	cloned := proto.Clone(device).(*voltha.Device)
-	log.Debugw("do-state-down", log.Fields{"ClonedDeviceID": cloned.Id})
-	/*
-		// Update the all ports state on that device to disable
-		if er := dh.coreProxy.PortsStateUpdate(ctx, cloned.Id, voltha.OperStatus_UNKNOWN); er != nil {
-			log.Errorw("updating-ports-failed", log.Fields{"deviceID": device.Id, "error": er})
-			return er
-		}
-
-		//Update the device oper state and connection status
-		cloned.OperStatus = voltha.OperStatus_UNKNOWN
-		cloned.ConnectStatus = common.ConnectStatus_UNREACHABLE
-		dh.device = cloned
-
-		if er := dh.coreProxy.DeviceStateUpdate(ctx, cloned.Id, cloned.ConnectStatus, cloned.OperStatus); er != nil {
-			log.Errorw("error-updating-device-state", log.Fields{"deviceID": device.Id, "error": er})
-			return er
-		}
-
-		//get the child device for the parent device
-		onuDevices, err := dh.coreProxy.GetChildDevices(ctx, dh.device.Id)
-		if err != nil {
-			log.Errorw("failed to get child devices information", log.Fields{"deviceID": dh.device.Id, "error": err})
-			return err
-		}
-		for _, onuDevice := range onuDevices.Items {
-
-			// Update onu state as down in onu adapter
-			onuInd := oop.OnuIndication{}
-			onuInd.OperState = "down"
-			er := dh.AdapterProxy.SendInterAdapterMessage(ctx, &onuInd, ic.InterAdapterMessageType_ONU_IND_REQUEST,
-				"openolt", onuDevice.Type, onuDevice.Id, onuDevice.ProxyAddress.DeviceId, "")
-			if er != nil {
-				log.Errorw("Failed to send inter-adapter-message", log.Fields{"OnuInd": onuInd,
-					"From Adapter": "openolt", "DevieType": onuDevice.Type, "DeviceID": onuDevice.Id})
-				//Do not return here and continue to process other ONUs
-			}
-		}
-		// * Discovered ONUs entries need to be cleared , since after OLT
-		//   is up, it starts sending discovery indications again* /
-		dh.discOnus = sync.Map{}
-		log.Debugw("do-state-down-end", log.Fields{"deviceID": device.Id})
-		return nil
-	*/
-	return errors.New("unimplemented")
-}
-
-// doStateConnected get the device info and update to voltha core
-// for comparison of the original method (not that easy to uncomment): compare here:
-//  voltha-openolt-adapter/adaptercore/device_handler.go
-//  -> this one obviously initiates all communication interfaces of the device ...?
-func (dh *DeviceHandler) doStateConnected(ctx context.Context) error {
-	log.Debug("OLT device has been connected")
-	return errors.New("unimplemented")
-}
-
-// DeviceHandler StateMachine related state transition methods ##### end #########
-// #################################################################################
-
-// ###################################################
-// DeviceHandler utility methods ##### begin #########
-
-// doStateInit provides the device update to the core
-func (dh *DeviceHandler) create_interface(onuind *oop.OnuIndication) error {
-	log.Debug("create_interface-started - not yet fully implemented (only device state update)")
-
-	if err := dh.coreProxy.DeviceStateUpdate(context.TODO(), dh.deviceID, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVATING); err != nil {
-		log.Errorw("error-updating-device-state", log.Fields{"deviceID": dh.deviceID, "error": err})
-	}
-
-	device, err := dh.coreProxy.GetDevice(context.TODO(), dh.device.Id, dh.device.Id)
-	if err != nil || device == nil {
-		/*TODO: needs to handle error scenarios */
-		log.Errorw("Failed to fetch device device at creating If", log.Fields{"err": err})
-		return errors.New("Voltha Device not found")
-	}
-
-	dh.onuOmciDevice.Start(context.TODO())
-	if err := dh.coreProxy.DeviceReasonUpdate(context.TODO(), dh.deviceID, "starting-openomci"); err != nil {
-		log.Errorw("error-DeviceReasonUpdate to starting-openomci", log.Fields{"deviceID": dh.deviceID, "error": err})
-	}
-
-	/* this might be a good time for Omci Verify message?  */
-	verifyExec := make(chan bool)
-	omci_verify := NewOmciTestRequest(context.TODO(),
-		dh.device.Id, dh.onuOmciDevice.PDevOmciCC,
-		true, true) //eclusive and allowFailure (anyway not yet checked)
-	omci_verify.PerformOmciTest(context.TODO(), verifyExec)
-
-	/* 	give the handler some time here to wait for the OMCi verification result
-	after Timeout start and try MibUpload FSM anyway
-	(to prevent stopping on just not supported OMCI verification from ONU) */
-	select {
-	case <-time.After(2 * time.Second):
-		log.Warn("omci start-verification timed out (continue normal)")
-	case testresult := <-verifyExec:
-		log.Infow("Omci start verification done", log.Fields{"result": testresult})
-	}
-
-	/* In py code it looks earlier (on activate ..)
-			# Code to Run OMCI Test Action
-			kwargs_omci_test_action = {
-				OmciTestRequest.DEFAULT_FREQUENCY_KEY:
-					OmciTestRequest.DEFAULT_COLLECTION_FREQUENCY
-			}
-			serial_number = device.serial_number
-			self._test_request = OmciTestRequest(self.core_proxy,
-											self.omci_agent, self.device_id,
-											AniG, serial_number,
-											self.logical_device_id,
-											exclusive=False,
-											**kwargs_omci_test_action)
-	...
-	                    # Start test requests after a brief pause
-	                    if not self._test_request_started:
-	                        self._test_request_started = True
-	                        tststart = _STARTUP_RETRY_WAIT * (random.randint(1, 5))
-	                        reactor.callLater(tststart, self._test_request.start_collector)
-
-	*/
-	/* which is then: in omci_test_request.py : */
-	/*
-	   def start_collector(self, callback=None):
-	       """
-	               Start the collection loop for an adapter if the frequency > 0
-
-	               :param callback: (callable) Function to call to collect PM data
-	       """
-	       self.log.info("starting-pm-collection", device_name=self.name, default_freq=self.default_freq)
-	       if callback is None:
-	           callback = self.perform_test_omci
-
-	       if self.lc is None:
-	           self.lc = LoopingCall(callback)
-
-	       if self.default_freq > 0:
-	           self.lc.start(interval=self.default_freq / 10)
-
-	   def perform_test_omci(self):
-	       """
-	       Perform the initial test request
-	       """
-	       ani_g_entities = self._device.configuration.ani_g_entities
-	       ani_g_entities_ids = list(ani_g_entities.keys()) if ani_g_entities \
-	                                                     is not None else None
-	       self._entity_id = ani_g_entities_ids[0]
-	       self.log.info('perform-test', entity_class=self._entity_class,
-	                     entity_id=self._entity_id)
-	       try:
-	           frame = MEFrame(self._entity_class, self._entity_id, []).test()
-	           result = yield self._device.omci_cc.send(frame)
-	           if not result.fields['omci_message'].fields['success_code']:
-	               self.log.info('Self-Test Submitted Successfully',
-	                             code=result.fields[
-	                                 'omci_message'].fields['success_code'])
-	           else:
-	               raise TestFailure('Test Failure: {}'.format(
-	                   result.fields['omci_message'].fields['success_code']))
-	       except TimeoutError as e:
-	           self.deferred.errback(failure.Failure(e))
-
-	       except Exception as e:
-	           self.log.exception('perform-test-Error', e=e,
-	                              class_id=self._entity_class,
-	                              entity_id=self._entity_id)
-	           self.deferred.errback(failure.Failure(e))
-
-	*/
-
-	// PM related heartbeat??? !!!TODO....
-	//self._heartbeat.enabled = True
-
-	//example how to call FSM - transition up to state "uploading"
-	if dh.onuOmciDevice.MibSyncFsm.Is("disabled") {
-
-		if err := dh.onuOmciDevice.MibSyncFsm.Event("start"); err != nil {
-			log.Errorw("MibSyncFsm: Can't go to state starting", log.Fields{"err": err})
-			return errors.New("Can't go to state starting")
-		} else {
-			log.Debug("MibSyncFsm", log.Fields{"state": string(dh.onuOmciDevice.MibSyncFsm.Current())})
-			//Determine ONU status and start/re-start MIB Synchronization tasks
-			//Determine if this ONU has ever synchronized
-			if true { //TODO: insert valid check
-				if err := dh.onuOmciDevice.MibSyncFsm.Event("load_mib_template"); err != nil {
-					log.Errorw("MibSyncFsm: Can't go to state loading_mib_template", log.Fields{"err": err})
-					return errors.New("Can't go to state loading_mib_template")
-				} else {
-					log.Debug("MibSyncFsm", log.Fields{"state": string(dh.onuOmciDevice.MibSyncFsm.Current())})
-					//Find and load a mib template. If not found proceed with mib_upload
-					// callbacks to be handled:
-					// Event("success")
-					// Event("timeout")
-					//no mib template found
-					if true { //TODO: insert valid check
-						if err := dh.onuOmciDevice.MibSyncFsm.Event("upload_mib"); err != nil {
-							log.Errorw("MibSyncFsm: Can't go to state uploading", log.Fields{"err": err})
-							return errors.New("Can't go to state uploading")
-						} else {
-							log.Debug("state of MibSyncFsm", log.Fields{"state": string(dh.onuOmciDevice.MibSyncFsm.Current())})
-							//Begin full MIB data upload, starting with a MIB RESET
-							// callbacks to be handled:
-							// success: e.Event("success")
-							// failure: e.Event("timeout")
-						}
-					}
-				}
-			} else {
-				dh.onuOmciDevice.MibSyncFsm.Event("examine_mds")
-				log.Debug("state of MibSyncFsm", log.Fields{"state": string(dh.onuOmciDevice.MibSyncFsm.Current())})
-				//Examine the MIB Data Sync
-				// callbacks to be handled:
-				// Event("success")
-				// Event("timeout")
-				// Event("mismatch")
-			}
-		}
-	} else {
-		log.Errorw("wrong state of MibSyncFsm - want: disabled", log.Fields{"have": string(dh.onuOmciDevice.MibSyncFsm.Current())})
-		return errors.New("wrong state of MibSyncFsm")
-	}
-	return nil
-}
-
-func (dh *DeviceHandler) update_interface(onuind *oop.OnuIndication) error {
-	log.Debug("update_interface-started - not yet implemented")
-	return nil
-}
-
-func (dh *DeviceHandler) DeviceStateUpdate(dev_Event OnuDeviceEvent) {
-	if dev_Event == MibDatabaseSync {
-		log.Debug("MibInSync Event: update dev state to 'MibSync complete'")
-		//initiate DevStateUpdate
-		if err := dh.coreProxy.DeviceReasonUpdate(context.TODO(), dh.deviceID, "discovery-mibsync-complete"); err != nil {
-			log.Errorw("error-DeviceReasonUpdate to mibsync-complete", log.Fields{"deviceID": dh.deviceID, "error": err})
-		}
-	} else {
-		log.Warnw("unhandled-device-event", log.Fields{"event": dev_Event})
-	}
-}
diff --git a/adaptercoreont/mib_sync.go b/adaptercoreont/mib_sync.go
deleted file mode 100644
index 16dc49f..0000000
--- a/adaptercoreont/mib_sync.go
+++ /dev/null
@@ -1,216 +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 adaptercoreont provides the utility for onu devices, flows and statistics
-package adaptercoreont
-
-import (
-	"context"
-	"errors"
-
-	"github.com/looplab/fsm"
-
-	//"sync"
-	//"time"
-
-	//"github.com/opencord/voltha-lib-go/v3/pkg/kafka"
-	"github.com/opencord/omci-lib-go"
-	me "github.com/opencord/omci-lib-go/generated"
-	"github.com/opencord/voltha-lib-go/v3/pkg/log"
-	//ic "github.com/opencord/voltha-protos/v3/go/inter_container"
-	//"github.com/opencord/voltha-protos/v3/go/openflow_13"
-	//"github.com/opencord/voltha-protos/v3/go/voltha"
-)
-
-func (onuDeviceEntry *OnuDeviceEntry) logStateChange(e *fsm.Event) {
-	log.Debugw("MibSync FSM", log.Fields{"event name": string(e.Event), "src state": string(e.Src), "dst state": string(e.Dst), "device-id": onuDeviceEntry.deviceID})
-}
-
-func (onuDeviceEntry *OnuDeviceEntry) enterStartingState(e *fsm.Event) {
-	log.Debugw("MibSync FSM", log.Fields{"Start processing MibSync-msgs in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
-
-	// create channel and start go routine for processing of MibSync messages
-	onuDeviceEntry.MibSyncChan = make(chan Message, 2048)
-	go onuDeviceEntry.ProcessMibSyncMessages()
-}
-
-func (onuDeviceEntry *OnuDeviceEntry) enterLoadingMibTemplateState(e *fsm.Event) {
-	log.Debugw("MibSync FSM", log.Fields{"Start MibTemplate processing in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
-	log.Debug("function not implemented yet")
-}
-
-func (onuDeviceEntry *OnuDeviceEntry) enterUploadingState(e *fsm.Event) {
-	log.Debugw("MibSync FSM", log.Fields{"send mibReset in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
-	onuDeviceEntry.PDevOmciCC.sendMibReset(context.TODO(), ConstDefaultOmciTimeout, true)
-}
-
-func (onuDeviceEntry *OnuDeviceEntry) enterInSyncState(e *fsm.Event) {
-	log.Debugw("MibSync FSM", log.Fields{"send notification to core in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
-	onuDeviceEntry.transferSystemEvent(MibDatabaseSync)
-}
-
-func (onuDeviceEntry *OnuDeviceEntry) enterExaminingMdsState(e *fsm.Event) {
-	log.Debugw("MibSync FSM", log.Fields{"Start GetMds processing in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
-	log.Debug("function not implemented yet")
-}
-
-func (onuDeviceEntry *OnuDeviceEntry) enterResynchronizingState(e *fsm.Event) {
-	log.Debugw("MibSync FSM", log.Fields{"Start MibResync processing in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
-	log.Debug("function not implemented yet")
-}
-
-func (onuDeviceEntry *OnuDeviceEntry) enterAuditingState(e *fsm.Event) {
-	log.Debugw("MibSync FSM", log.Fields{"Start MibResync processing in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
-	log.Debug("function not implemented yet")
-}
-
-func (onuDeviceEntry *OnuDeviceEntry) enterOutOfSyncState(e *fsm.Event) {
-	log.Debugw("MibSync FSM", log.Fields{"Start  MibReconcile processing in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
-	log.Debug("function not implemented yet")
-}
-
-func (onuDeviceEntry *OnuDeviceEntry) ProcessMibSyncMessages( /*ctx context.Context*/ ) {
-	log.Debugw("MibSync Msg", log.Fields{"Start routine to process OMCI-messages for device-id": onuDeviceEntry.deviceID})
-loop:
-	for {
-		select {
-		// case <-ctx.Done():
-		// 	log.Info("MibSync Msg", log.Fields{"Message handling canceled via context for device-id": onuDeviceEntry.deviceID})
-		// 	break loop
-		case message, ok := <-onuDeviceEntry.MibSyncChan:
-			if !ok {
-				log.Info("MibSync Msg", log.Fields{"Message couldn't be read from channel for device-id": onuDeviceEntry.deviceID})
-				break loop
-			}
-			log.Debugw("MibSync Msg", log.Fields{"Received message on ONU MibSyncChan for device-id": onuDeviceEntry.deviceID})
-
-			switch message.Type {
-			case TestMsg:
-				msg, _ := message.Data.(TestMessage)
-				onuDeviceEntry.handleTestMsg(msg)
-			case OMCI:
-				msg, _ := message.Data.(OmciMessage)
-				onuDeviceEntry.handleOmciMessage(msg)
-			default:
-				log.Warn("MibSync Msg", log.Fields{"Unknown message type received for device-id": onuDeviceEntry.deviceID, "message.Type": message.Type})
-			}
-		}
-	}
-	log.Info("MibSync Msg", log.Fields{"Stopped handling of MibSyncChan for device-id": onuDeviceEntry.deviceID})
-	// TODO: only this action?
-	onuDeviceEntry.MibSyncFsm.Event("stop")
-}
-
-func (onuDeviceEntry *OnuDeviceEntry) handleTestMsg(msg TestMessage) {
-
-	log.Debugw("MibSync Msg", log.Fields{"TestMessage received for device-id": onuDeviceEntry.deviceID, "msg.TestMessageVal": msg.TestMessageVal})
-
-	switch msg.TestMessageVal {
-	case AnyTriggerForMibSyncUploadMib:
-		onuDeviceEntry.MibSyncFsm.Event("upload_mib")
-		log.Debugw("MibSync Msg", log.Fields{"state": string(onuDeviceEntry.MibSyncFsm.Current())})
-	default:
-		log.Warn("MibSync Msg", log.Fields{"Unknown message type received for device-id": onuDeviceEntry.deviceID, "msg.TestMessageVal": msg.TestMessageVal})
-	}
-}
-
-func (onuDeviceEntry *OnuDeviceEntry) handleOmciMessage(msg OmciMessage) {
-
-	log.Debugw("MibSync Msg", log.Fields{"OmciMessage received for device-id": onuDeviceEntry.deviceID,
-		"msgType": msg.OmciMsg.MessageType})
-
-	//further analysis could be done here based on msg.OmciMsg.Payload, e.g. verification of error code ...
-	switch msg.OmciMsg.MessageType {
-	case omci.MibResetResponseType:
-		msgLayer := (*msg.OmciPacket).Layer(omci.LayerTypeMibResetResponse)
-		if msgLayer == nil {
-			log.Error("Omci Msg layer could not be detected")
-			return
-		}
-		msgObj, msgOk := msgLayer.(*omci.MibResetResponse)
-		if !msgOk {
-			log.Error("Omci Msg layer could not be assigned")
-			return
-		}
-		log.Debugw("MibResetResponse Data", log.Fields{"data-fields": msgObj})
-		if msgObj.Result != me.Success {
-			log.Errorw("Omci MibResetResponse Error - strange - what to do?", log.Fields{"Error": msgObj.Result})
-			return
-		}
-		onuDeviceEntry.PDevOmciCC.sendMibUpload(context.TODO(), ConstDefaultOmciTimeout, true)
-	case omci.MibUploadResponseType:
-		msgLayer := (*msg.OmciPacket).Layer(omci.LayerTypeMibUploadResponse)
-		if msgLayer == nil {
-			log.Error("Omci Msg layer could not be detected")
-			return
-		}
-		msgObj, msgOk := msgLayer.(*omci.MibUploadResponse)
-		if !msgOk {
-			log.Error("Omci Msg layer could not be assigned")
-			return
-		}
-		log.Debugw("MibUploadResponse Data for:", log.Fields{"deviceId": onuDeviceEntry.PDevOmciCC.deviceID, "data-fields": msgObj})
-		/* to be verified / reworked !!! */
-		onuDeviceEntry.PDevOmciCC.uploadNoOfCmds = msgObj.NumberOfCommands
-		if onuDeviceEntry.PDevOmciCC.uploadSequNo < onuDeviceEntry.PDevOmciCC.uploadNoOfCmds {
-			onuDeviceEntry.PDevOmciCC.sendMibUploadNext(context.TODO(), ConstDefaultOmciTimeout, true)
-		} else {
-			log.Error("Invalid number of commands received for:", log.Fields{"deviceId": onuDeviceEntry.PDevOmciCC.deviceID, "uploadNoOfCmds": onuDeviceEntry.PDevOmciCC.uploadNoOfCmds})
-			//TODO right action?
-			onuDeviceEntry.MibSyncFsm.Event("timeout")
-		}
-	case omci.MibUploadNextResponseType:
-		msgLayer := (*msg.OmciPacket).Layer(omci.LayerTypeMibUploadNextResponse)
-		if msgLayer == nil {
-			log.Error("Omci Msg layer could not be detected")
-			return
-		}
-		msgObj, msgOk := msgLayer.(*omci.MibUploadNextResponse)
-		if !msgOk {
-			log.Error("Omci Msg layer could not be assigned")
-			return
-		}
-		log.Debugw("MibUploadNextResponse Data for:", log.Fields{"deviceId": onuDeviceEntry.PDevOmciCC.deviceID, "data-fields": msgObj})
-		// TODO !!! content evaluation ?????
-		if onuDeviceEntry.PDevOmciCC.uploadSequNo < onuDeviceEntry.PDevOmciCC.uploadNoOfCmds {
-			onuDeviceEntry.PDevOmciCC.sendMibUploadNext(context.TODO(), ConstDefaultOmciTimeout, true)
-		} else {
-			//TODO
-			onuDeviceEntry.MibSyncFsm.Event("success")
-		}
-	}
-}
-
-func (onuDeviceEntry *OnuDeviceEntry) MibDbVolatileDict() error {
-	log.Debug("MibVolatileDict- running")
-	return errors.New("not_implemented")
-}
-
-// func (onuDeviceEntry *OnuDeviceEntry) MibTemplateTask() error {
-// 	return errors.New("not_implemented")
-// }
-// func (onuDeviceEntry *OnuDeviceEntry) MibUploadTask() error {
-// 	return errors.New("not_implemented")
-// }
-// func (onuDeviceEntry *OnuDeviceEntry) GetMdsTask() error {
-// 	return errors.New("not_implemented")
-// }
-// func (onuDeviceEntry *OnuDeviceEntry) MibResyncTask() error {
-// 	return errors.New("not_implemented")
-// }
-// func (onuDeviceEntry *OnuDeviceEntry) MibReconcileTask() error {
-// 	return errors.New("not_implemented")
-// }
diff --git a/adaptercoreont/omci_agent.go b/adaptercoreont/omci_agent.go
deleted file mode 100644
index ff462f0..0000000
--- a/adaptercoreont/omci_agent.go
+++ /dev/null
@@ -1,184 +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 adaptercoreont provides the utility for onu devices, flows and statistics
-package adaptercoreont
-
-import (
-	"context"
-	//"errors"
-	//"sync"
-	//"time"
-
-	"github.com/opencord/voltha-lib-go/v3/pkg/adapters/adapterif"
-
-	//"github.com/opencord/voltha-lib-go/v3/pkg/kafka"
-	"github.com/opencord/voltha-lib-go/v3/pkg/log"
-	//ic "github.com/opencord/voltha-protos/v3/go/inter_container"
-	//"github.com/opencord/voltha-protos/v3/go/openflow_13"
-	//"github.com/opencord/voltha-protos/v3/go/voltha"
-)
-
-/*
-OpenOmciAgentDefaults = {
-    'mib-synchronizer': {
-        'state-machine': MibSynchronizer,  # Implements the MIB synchronization state machine
-        'database': MibDbVolatileDict,     # Implements volatile ME MIB database
-        # 'database': MibDbExternal,         # Implements persistent ME MIB database
-        'advertise-events': True,          # Advertise events on OpenOMCI event bus
-        'audit-delay': 60,                 # Time to wait between MIB audits.  0 to disable audits.
-        'tasks': {
-            'mib-upload': MibUploadTask,
-            'mib-template': MibTemplateTask,
-            'get-mds': GetMdsTask,
-            'mib-audit': GetMdsTask,
-            'mib-resync': MibResyncTask,
-            'mib-reconcile': MibReconcileTask
-        }
-    },
-    'omci-capabilities': {
-        'state-machine': OnuOmciCapabilities,   # Implements OMCI capabilities state machine
-        'advertise-events': False,              # Advertise events on OpenOMCI event bus
-        'tasks': {
-            'get-capabilities': OnuCapabilitiesTask # Get supported ME and Commands
-        }
-    },
-    'performance-intervals': {
-        'state-machine': PerformanceIntervals,  # Implements PM Intervals State machine
-        'advertise-events': False,              # Advertise events on OpenOMCI event bus
-        'tasks': {
-            'sync-time': SyncTimeTask,
-            'collect-data': IntervalDataTask,
-            'create-pm': OmciCreatePMRequest,
-            'delete-pm': OmciDeletePMRequest,
-        },
-    },
-    'alarm-synchronizer': {
-        'state-machine': AlarmSynchronizer,    # Implements the Alarm sync state machine
-        'database': AlarmDbExternal,           # For any State storage needs
-        'advertise-events': True,              # Advertise events on OpenOMCI event bus
-        'tasks': {
-            'alarm-resync': AlarmResyncTask
-        }
-     },
-    'image_downloader': {
-        'state-machine': ImageDownloadeSTM,
-        'advertise-event': True,
-        'tasks': {
-            'download-file': FileDownloadTask
-        }
-    },
-    'image_upgrader': {
-        'state-machine': OmciSoftwareImageDownloadSTM,
-        'advertise-event': True,
-        'tasks': {
-            'omci_upgrade_task': OmciSwImageUpgradeTask
-        }
-    }
-    # 'image_activator': {
-    #     'state-machine': OmciSoftwareImageActivateSTM,
-    #     'advertise-event': True,
-    # }
-}
-*/
-
-//OpenOMCIAgent structure holds the ONU core information
-type OpenOMCIAgent struct {
-	coreProxy     adapterif.CoreProxy
-	adapterProxy  adapterif.AdapterProxy
-	started       bool
-	deviceEntries map[string]*OnuDeviceEntry
-	mibDbClass    func() error
-}
-
-//NewOpenOMCIAgent returns a new instance of OpenOMCIAgent
-func NewOpenOMCIAgent(ctx context.Context,
-	coreProxy adapterif.CoreProxy, adapterProxy adapterif.AdapterProxy) *OpenOMCIAgent {
-	log.Info("init-openOmciAgent")
-	var openomciagent OpenOMCIAgent
-	openomciagent.started = false
-	openomciagent.coreProxy = coreProxy
-	openomciagent.adapterProxy = adapterProxy
-	openomciagent.deviceEntries = make(map[string]*OnuDeviceEntry)
-	return &openomciagent
-}
-
-//Start starts (logs) the omci agent
-func (oo *OpenOMCIAgent) Start(ctx context.Context) error {
-	log.Info("starting-openOmciAgent")
-	//TODO .....
-	//mib_db.start()
-	oo.started = true
-	log.Info("openOmciAgent-started")
-	return nil
-}
-
-//Stop terminates the session
-func (oo *OpenOMCIAgent) Stop(ctx context.Context) error {
-	log.Info("stopping-openOmciAgent")
-	oo.started = false
-	//oo.exitChannel <- 1
-	log.Info("openOmciAgent-stopped")
-	return nil
-}
-
-//
-//Add a new ONU to be managed.
-
-//To provide vendor-specific or custom Managed Entities, create your own Entity
-//  ID to class mapping dictionary.
-
-//Since ONU devices can be added at any time (even during Device Handler
-//  startup), the ONU device handler is responsible for calling start()/stop()
-//  for this object.
-
-//:param device_id: (str) Device ID of ONU to add
-//:param core_proxy: (CoreProxy) Remote API to VOLTHA core
-//:param adapter_proxy: (AdapterProxy) Remote API to other adapters via VOLTHA core
-//:param custom_me_map: (dict) Additional/updated ME to add to class map
-//:param support_classes: (dict) State machines and tasks for this ONU
-
-//:return: (OnuDeviceEntry) The ONU device
-//
-func (oo *OpenOMCIAgent) Add_device(ctx context.Context, device_id string,
-	dh *DeviceHandler) (*OnuDeviceEntry, error) {
-	log.Info("openOmciAgent-adding-deviceEntry")
-
-	deviceEntry := oo.GetDevice(device_id)
-	if deviceEntry == nil {
-		/* costum_me_map in python code seems always to be None,
-		   we omit that here first (declaration unclear) -> todo at Adapter specialization ...*/
-		/* also no 'clock' argument - usage open ...*/
-		/* and no alarm_db yet (oo.alarm_db)  */
-		deviceEntry = NewOnuDeviceEntry(ctx, device_id, dh, oo.coreProxy, oo.adapterProxy,
-			oo.mibDbClass, nil)
-		oo.deviceEntries[device_id] = deviceEntry
-		log.Infow("openOmciAgent-OnuDeviceEntry-added", log.Fields{"for deviceId": device_id})
-	} else {
-		log.Infow("openOmciAgent-OnuDeviceEntry-add: Device already exists", log.Fields{"for deviceId": device_id})
-	}
-	// might be updated with some error handling !!!
-	return deviceEntry, nil
-}
-
-// Get ONU device entry for a specific Id
-func (oo *OpenOMCIAgent) GetDevice(device_id string) *OnuDeviceEntry {
-	if _, exist := oo.deviceEntries[device_id]; !exist {
-		return nil
-	} else {
-		return oo.deviceEntries[device_id]
-	}
-}
diff --git a/adaptercoreont/ont_device_state_transition.go b/adaptercoreont/ont_device_state_transition.go
deleted file mode 100644
index e11f918..0000000
--- a/adaptercoreont/ont_device_state_transition.go
+++ /dev/null
@@ -1,189 +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 adaptercoreont provides the utility for onu devices, flows and statistics
-package adaptercoreont
-
-import (
-	"context"
-	"reflect"
-	"runtime"
-
-	"github.com/opencord/voltha-lib-go/v3/pkg/log"
-)
-
-// DeviceState ONU Device state
-type DeviceState int
-
-//ONU device state machine may have to be adapted to specific ONU Adapter device needd ... !!!!!
-
-const (
-	// deviceStateNull OLT is not instantiated
-	deviceStateNull DeviceState = iota
-	// deviceStateInit OLT is instantiated
-	deviceStateInit
-	// deviceStateConnected Grpc session established with OLT
-	deviceStateConnected
-	// deviceStateUp Admin state of OLT is UP
-	deviceStateUp
-	// deviceStateDown Admin state of OLT is down
-	deviceStateDown
-)
-
-// Trigger for changing the state
-type Trigger int
-
-const (
-	// DeviceInit Go to Device init state
-	DeviceInit Trigger = iota
-	// GrpcConnected Go to connected state
-	GrpcConnected
-	// DeviceUpInd Go to Device up state
-	DeviceUpInd
-	// DeviceDownInd Go to Device down state
-	DeviceDownInd
-	// GrpcDisconnected Go to Device init state
-	GrpcDisconnected
-)
-
-// TransitionHandler function type for handling transition
-type TransitionHandler func(ctx context.Context) error
-
-// Transition to store state machine
-type Transition struct {
-	previousState []DeviceState
-	currentState  DeviceState
-	before        []TransitionHandler
-	after         []TransitionHandler
-}
-
-// TransitionMap to store all the states and current device state
-type TransitionMap struct {
-	transitions        map[Trigger]Transition
-	currentDeviceState DeviceState
-}
-
-//    OpenOnuDevice state machine:
-//
-//        null ----> init ------> connected -----> up -----> down
-//                   ^ ^             |             ^         | |
-//                   | |             |             |         | |
-//                   | +-------------+             +---------+ |
-//                   |                                         |
-//                   +-----------------------------------------+
-
-// NewTransitionMap create a new state machine with all the transitions
-func NewTransitionMap(dh *DeviceHandler) *TransitionMap {
-	var transitionMap TransitionMap
-	transitionMap.currentDeviceState = deviceStateNull
-	transitionMap.transitions = make(map[Trigger]Transition)
-	// In doInit create the Pon port, then set the device communication
-	transitionMap.transitions[DeviceInit] =
-		Transition{
-			previousState: []DeviceState{deviceStateNull, deviceStateDown},
-			currentState:  deviceStateInit,
-			before:        []TransitionHandler{dh.doStateInit},
-			after:         []TransitionHandler{dh.postInit}}
-	// not yet relevant?
-	transitionMap.transitions[GrpcDisconnected] =
-		Transition{
-			previousState: []DeviceState{deviceStateConnected, deviceStateDown},
-			currentState:  deviceStateInit,
-			before:        []TransitionHandler{dh.doStateInit},
-			after:         []TransitionHandler{dh.postInit}}
-	// in doConnected, create logical device and read the indications
-	transitionMap.transitions[GrpcConnected] =
-		Transition{
-			previousState: []DeviceState{deviceStateInit},
-			currentState:  deviceStateConnected,
-			before:        []TransitionHandler{dh.doStateConnected}}
-
-	// Once the olt UP is indication received, then do state up
-	transitionMap.transitions[DeviceUpInd] =
-		Transition{
-			previousState: []DeviceState{deviceStateConnected, deviceStateDown},
-			currentState:  deviceStateUp,
-			before:        []TransitionHandler{dh.doStateUp}}
-	// If olt DOWN indication comes then do sate down
-	transitionMap.transitions[DeviceDownInd] =
-		Transition{
-			previousState: []DeviceState{deviceStateUp},
-			currentState:  deviceStateDown,
-			before:        []TransitionHandler{dh.doStateDown}}
-
-	return &transitionMap
-}
-
-// funcName gets the handler function name
-func funcName(f interface{}) string {
-	p := reflect.ValueOf(f).Pointer()
-	rf := runtime.FuncForPC(p)
-	return rf.Name()
-}
-
-// isValidTransition checks for the new state transition is valid from current state
-func (tMap *TransitionMap) isValidTransition(trigger Trigger) bool {
-	// Validate the state transition
-	for _, state := range tMap.transitions[trigger].previousState {
-		if tMap.currentDeviceState == state {
-			return true
-		}
-	}
-	return false
-}
-
-// Handle moves the state machine to next state based on the trigger and invokes the before and
-// after handlers if the transition is a valid transition
-func (tMap *TransitionMap) Handle(ctx context.Context, trigger Trigger) {
-
-	// Check whether the transtion is valid from current state
-	if !tMap.isValidTransition(trigger) {
-		log.Errorw("Invalid transition triggered ", log.Fields{"CurrentState": tMap.currentDeviceState, "Trigger": trigger})
-		return
-	}
-
-	// Invoke the before handlers
-	beforeHandlers := tMap.transitions[trigger].before
-	if beforeHandlers == nil {
-		log.Debugw("No handlers for before", log.Fields{"trigger": trigger})
-	}
-	for _, handler := range beforeHandlers {
-		log.Debugw("running-before-handler", log.Fields{"handler": funcName(handler)})
-		if err := handler(ctx); err != nil {
-			// TODO handle error
-			log.Error(err)
-			return
-		}
-	}
-
-	// Update the state
-	tMap.currentDeviceState = tMap.transitions[trigger].currentState
-	log.Debugw("Updated device state ", log.Fields{"CurrentDeviceState": tMap.currentDeviceState})
-
-	// Invoke the after handlers
-	afterHandlers := tMap.transitions[trigger].after
-	if afterHandlers == nil {
-		log.Debugw("No handlers for after", log.Fields{"trigger": trigger})
-	}
-	for _, handler := range afterHandlers {
-		log.Debugw("running-after-handler", log.Fields{"handler": funcName(handler)})
-		if err := handler(ctx); err != nil {
-			// TODO handle error
-			log.Error(err)
-			return
-		}
-	}
-}
diff --git a/cmd/openonu-adapter/common.go b/cmd/openonu-adapter/common.go
new file mode 100644
index 0000000..7507288
--- /dev/null
+++ b/cmd/openonu-adapter/common.go
@@ -0,0 +1,34 @@
+/*
+ * 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 openonu-adapter main Common Logger initialization
+package main
+
+import (
+	"github.com/opencord/voltha-lib-go/v3/pkg/log"
+)
+
+var logger log.Logger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.AddPackage(log.JSON, log.ErrorLevel, log.Fields{"pkg": "main"})
+	// previously log.DebugLevel was set !?
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/main.go b/cmd/openonu-adapter/main.go
similarity index 72%
rename from main.go
rename to cmd/openonu-adapter/main.go
index 7a3a0d5..a022cef 100644
--- a/main.go
+++ b/cmd/openonu-adapter/main.go
@@ -14,14 +14,14 @@
  * limitations under the License.
  */
 
-//Package main -> this is the entry point of the OpenAdapter
+//Package main -> this is the entry point of the OpenOnuAdapter
 package main
 
 import (
 	"context"
 	"errors"
 	"fmt"
-	"github.com/opencord/voltha-lib-go/v3/pkg/db"
+	"io/ioutil"
 	"os"
 	"os/signal"
 	"strconv"
@@ -31,16 +31,17 @@
 	"github.com/opencord/voltha-lib-go/v3/pkg/adapters"
 	"github.com/opencord/voltha-lib-go/v3/pkg/adapters/adapterif"
 	com "github.com/opencord/voltha-lib-go/v3/pkg/adapters/common"
+	conf "github.com/opencord/voltha-lib-go/v3/pkg/config"
 	"github.com/opencord/voltha-lib-go/v3/pkg/db/kvstore"
 	"github.com/opencord/voltha-lib-go/v3/pkg/kafka"
 	"github.com/opencord/voltha-lib-go/v3/pkg/log"
 	"github.com/opencord/voltha-lib-go/v3/pkg/probe"
+	"github.com/opencord/voltha-lib-go/v3/pkg/version"
 	ic "github.com/opencord/voltha-protos/v3/go/inter_container"
 	"github.com/opencord/voltha-protos/v3/go/voltha"
 
-	ac "test.internal/openadapter/adaptercoreont"
-	"test.internal/openadapter/config"
-	"test.internal/openadapter/config/version"
+	"test.internal/openadapter/internal/pkg/config"
+	ac "test.internal/openadapter/internal/pkg/onuadaptercore"
 )
 
 type adapter struct {
@@ -59,11 +60,6 @@
 	receiverChannels []<-chan *ic.InterContainerMessage //from inter-container
 }
 
-//package single start function (run at first package instantiation)
-func init() {
-	_, _ = log.AddPackage(log.JSON, log.DebugLevel, nil)
-}
-
 func newAdapter(cf *config.AdapterFlags) *adapter {
 	var a adapter
 	a.instanceID = cf.InstanceID
@@ -75,7 +71,7 @@
 }
 
 func (a *adapter) start(ctx context.Context) error {
-	log.Info("Starting Core Adapter components")
+	logger.Info("Starting Core Adapter components")
 	var err error
 
 	var p *probe.Probe
@@ -93,18 +89,22 @@
 	}
 
 	// Setup KV Client
-	log.Debugw("create-kv-client", log.Fields{"kvstore": a.config.KVStoreType})
+	logger.Debugw("create-kv-client", log.Fields{"kvstore": a.config.KVStoreType})
 	if err = a.setKVClient(); err != nil {
-		log.Fatal("error-setting-kv-client")
+		logger.Fatalw("error-setting-kv-client", log.Fields{"error": err})
 	}
 
 	if p != nil {
 		p.UpdateStatus("kv-store", probe.ServiceStatusRunning)
 	}
 
+	// Setup Log Config
+	cm := conf.NewConfigManager(a.kvClient, a.config.KVStoreType, a.config.KVStoreHost, a.config.KVStorePort, a.config.KVStoreTimeout)
+	go conf.StartLogLevelConfigProcessing(cm, ctx)
+
 	// Setup Kafka Client
 	if a.kafkaClient, err = newKafkaClient("sarama", a.config.KafkaAdapterHost, a.config.KafkaAdapterPort); err != nil {
-		log.Fatal("Unsupported-common-client")
+		logger.Fatalw("Unsupported-common-client", log.Fields{"error": err})
 	}
 
 	if p != nil {
@@ -113,7 +113,7 @@
 
 	// 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 {
-		log.Fatal("error-starting-inter-container-proxy")
+		logger.Fatalw("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
 	}
@@ -122,16 +122,7 @@
 	a.coreProxy = com.NewCoreProxy(a.kip, a.config.Topic, a.config.CoreTopic)
 
 	// Create the adaptor proxy to handle request between olt and onu
-	//a.adapterProxy = com.NewAdapterProxy(a.kip, "brcm_openomci_onu", a.config.CoreTopic)
-	backend := &db.Backend{
-		Client:     a.kvClient,
-		StoreType:  a.config.KVStoreType,
-		Host:       a.config.KVStoreHost,
-		Port:       a.config.KVStorePort,
-		Timeout:    a.config.KVStoreTimeout,
-		PathPrefix: "service/voltha",
-	}
-	a.adapterProxy = com.NewAdapterProxy(a.kip, "openolt", a.config.CoreTopic, backend)
+	a.adapterProxy = com.NewAdapterProxy(a.kip, "openolt", a.config.CoreTopic, cm.Backend)
 
 	// Create the event proxy to post events to KAFKA
 	a.eventProxy = com.NewEventProxy(com.MsgClient(a.kafkaClient), com.MsgTopic(kafka.Topic{Name: a.config.EventTopic}))
@@ -139,17 +130,17 @@
 	// Create the open ONU interface adapter
 	if a.iAdapter, err = a.startVolthaInterfaceAdapter(ctx, a.kip, a.coreProxy, a.adapterProxy, a.eventProxy,
 		a.config); err != nil {
-		log.Fatal("error-starting-VolthaInterfaceAdapter for OpenOnt")
+		logger.Fatalw("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 {
-		log.Fatal("error-setting-core-request-handler")
+		logger.Fatalw("error-setting-core-request-handler", log.Fields{"error": err})
 	}
 
 	// Register this adapter to the Core - retries indefinitely
 	if err = a.registerWithCore(ctx, -1); err != nil {
-		log.Fatal("error-registering-with-core")
+		logger.Fatalw("error-registering-with-core", log.Fields{"error": err})
 	}
 
 	// check the readiness and liveliness and update the probe status
@@ -168,25 +159,23 @@
 	if a.kvClient != nil {
 		// Release all reservations
 		if err := a.kvClient.ReleaseAllReservations(ctx); err != nil {
-			log.Infow("fail-to-release-all-reservations", log.Fields{"error": err})
+			logger.Infow("fail-to-release-all-reservations", log.Fields{"error": err})
 		}
 		// Close the DB connection
 		a.kvClient.Close()
 	}
 
-	// TODO:  More cleanup
+	if a.kip != nil {
+		a.kip.Stop()
+	}
 }
 
 // #############################################
 // Adapter Utility methods ##### begin #########
 
-func (a *adapter) getAdapterInstance() string {
-	return a.instanceID
-}
-
 func newKVClient(storeType, address string, timeout int) (kvstore.Client, error) {
 
-	log.Infow("kv-store-type", log.Fields{"store": storeType})
+	logger.Infow("kv-store-type", log.Fields{"store": storeType})
 	switch storeType {
 	case "consul":
 		return kvstore.NewConsulClient(address, timeout)
@@ -198,7 +187,7 @@
 
 func newKafkaClient(clientType, host string, port int) (kafka.Client, error) {
 
-	log.Infow("common-client-type", log.Fields{"client": clientType})
+	logger.Infow("common-client-type", log.Fields{"client": clientType})
 	switch clientType {
 	case "sarama":
 		return kafka.NewSaramaClient(
@@ -219,7 +208,7 @@
 	client, err := newKVClient(a.config.KVStoreType, addr, a.config.KVStoreTimeout)
 	if err != nil {
 		a.kvClient = nil
-		log.Error(err)
+		logger.Errorw("error-starting-KVClient", log.Fields{"error": err})
 		return err
 	}
 	a.kvClient = client
@@ -227,7 +216,7 @@
 }
 
 func (a *adapter) startInterContainerProxy(ctx context.Context, retries int) (kafka.InterContainerProxy, error) {
-	log.Infow("starting-intercontainer-messaging-proxy", log.Fields{"host": a.config.KafkaAdapterHost,
+	logger.Infow("starting-intercontainer-messaging-proxy", log.Fields{"host": a.config.KafkaAdapterHost,
 		"port": a.config.KafkaAdapterPort, "topic": a.config.Topic})
 	var err error
 	kip := kafka.NewInterContainerProxy(
@@ -235,11 +224,11 @@
 		kafka.InterContainerPort(a.config.KafkaAdapterPort),
 		kafka.MsgClient(a.kafkaClient),
 		kafka.DefaultTopic(&kafka.Topic{Name: a.config.Topic}))
-	var count uint8 = 0
+	count := 0
 	for {
 		if err = kip.Start(); err != nil {
-			log.Warnw("error-starting-messaging-proxy", log.Fields{"error": err, "retry": retries, "count": count})
-			if retries == int(count) {
+			logger.Warnw("error-starting-messaging-proxy", log.Fields{"error": err, "retry": retries, "count": count})
+			if retries == count {
 				return nil, err
 			}
 			count++
@@ -250,31 +239,10 @@
 		}
 	}
 	probe.UpdateStatusFromContext(ctx, "container-proxy", probe.ServiceStatusRunning)
-	log.Info("common-messaging-proxy-created")
+	logger.Info("common-messaging-proxy-created")
 	return kip, nil
 }
 
-/*
-func (a *adapter) startOpenOLT(ctx context.Context, kip kafka.InterContainerProxy,
-	cp adapterif.CoreProxy, ap adapterif.AdapterProxy, ep adapterif.EventProxy,
-	cfg *config.AdapterFlags) (*ac.OpenOLT, error) {
-	log.Info("starting-open-olt")
-	var err error
-	sOLT := ac.NewOpenOLT(ctx, a.kip, cp, ap, ep, cfg)
-
-	if err = sOLT.Start(ctx); err != nil {
-		log.Fatalw("error-starting-messaging-proxy", log.Fields{"error": err})
-		return nil, err
-	}
-
-	log.Info("open-olt-started")
-	return sOLT, nil
-}
-func (a *adapter) startVolthaInterfaceAdapter(ctx context.Context, kip kafka.InterContainerProxy,
-	cp adapterif.CoreProxy, ap adapterif.AdapterProxy, ep adapterif.EventProxy,
-	cfg *config.AdapterFlags) (adapters.IAdapter, error) {
-*/
-
 func (a *adapter) startVolthaInterfaceAdapter(ctx context.Context, kip kafka.InterContainerProxy,
 	cp adapterif.CoreProxy, ap adapterif.AdapterProxy, ep adapterif.EventProxy,
 	cfg *config.AdapterFlags) (*ac.OpenONUAC, error) {
@@ -282,46 +250,44 @@
 	sAcONU := ac.NewOpenONUAC(ctx, a.kip, cp, ap, ep, cfg)
 
 	if err = sAcONU.Start(ctx); err != nil {
-		log.Fatalw("error-starting-messaging-proxy", log.Fields{"error": err})
+		logger.Fatalw("error-starting-OpenOnuAdapterCore", log.Fields{"error": err})
 		return nil, err
 	}
 
-	log.Info("open-ont-adaptercore-started")
+	logger.Info("open-ont-OpenOnuAdapterCore-started")
 	return sAcONU, nil
 }
 
 func (a *adapter) setupRequestHandler(ctx context.Context, coreInstanceID string, iadapter adapters.IAdapter) error {
-	log.Info("setting-request-handler")
+	logger.Info("setting-request-handler")
 	requestProxy := com.NewRequestHandlerProxy(coreInstanceID, iadapter, a.coreProxy)
 	if err := a.kip.SubscribeWithRequestHandlerInterface(kafka.Topic{Name: a.config.Topic}, requestProxy); err != nil {
-		log.Errorw("request-handler-setup-failed", log.Fields{"error": err})
+		logger.Errorw("request-handler-setup-failed", log.Fields{"error": err})
 		return err
 
 	}
 	probe.UpdateStatusFromContext(ctx, "core-request-handler", probe.ServiceStatusRunning)
-	log.Info("request-handler-setup-done")
+	logger.Info("request-handler-setup-done")
 	return nil
 }
 
 func (a *adapter) registerWithCore(ctx context.Context, retries int) error {
-	log.Info("registering-with-core")
-	/*
-		adapterDescription := &voltha.Adapter{Id: "openolt", // Unique name for the device type
-			Vendor:  "VOLTHA OpenOLT",
-			Version: version.VersionInfo.Version}
-		types := []*voltha.DeviceType{{Id: "openolt",
-			Adapter:                     "openolt", // Name of the adapter that handles device type
-			AcceptsBulkFlowUpdate:       false,     // Currently openolt adapter does not support bulk flow handling
-			AcceptsAddRemoveFlowUpdates: true}}
-	*/
+	adapterID := fmt.Sprintf("brcm_openomci_onu_%d", a.config.CurrentReplica)
+	logger.Infow("registering-with-core", log.Fields{
+		"adapterID":      adapterID,
+		"currentReplica": a.config.CurrentReplica,
+		"totalReplicas":  a.config.TotalReplicas,
+	})
 	adapterDescription := &voltha.Adapter{
-		Id: "brcm_openomci_onu", // Unique name for the device type ->exact type required for OLT comm????
+		Id:      adapterID, // Unique name for the device type ->exact type required for OLT comm????
 		Vendor:  "VOLTHA OpenONUGo",
 		Version: version.VersionInfo.Version,
+		// TODO once we'll be ready to support multiple versions of the adapter
+		// the Endpoint will have to change to `brcm_openomci_onu_<currentReplica`>
 		Endpoint:       "brcm_openomci_onu",
 		Type:           "brcm_openomci_onu",
-		CurrentReplica: 1,
-		TotalReplicas:  1,
+		CurrentReplica: int32(a.config.CurrentReplica),
+		TotalReplicas:  int32(a.config.TotalReplicas),
 	}
 	types := []*voltha.DeviceType{{Id: "brcm_openomci_onu",
 		VendorIds:                   []string{"OPEN", "ALCL", "BRCM", "TWSH", "ALPH", "ISKT", "SFAA", "BBSM", "SCOM"},
@@ -332,7 +298,7 @@
 	count := 0
 	for {
 		if err := a.coreProxy.RegisterAdapter(context.TODO(), adapterDescription, deviceTypes); err != nil {
-			log.Warnw("registering-with-core-failed", log.Fields{"error": err})
+			logger.Warnw("registering-with-core-failed", log.Fields{"error": err})
 			if retries == count {
 				return err
 			}
@@ -344,7 +310,7 @@
 		}
 	}
 	probe.UpdateStatusFromContext(ctx, "register-with-core", probe.ServiceStatusRunning)
-	log.Info("registered-with-core")
+	logger.Info("registered-with-core")
 	return nil
 }
 
@@ -390,7 +356,7 @@
 			}
 		case <-timeoutTimer.C:
 			// Check the status of the kv-store
-			log.Info("kv-store liveliness-recheck")
+			logger.Info("kv-store liveliness-recheck")
 			if a.kvClient.IsConnectionUp(ctx) {
 				kvStoreChannel <- true
 			} else {
@@ -414,8 +380,8 @@
 		select {
 		case healthiness := <-healthinessChannel:
 			if !healthiness {
-				// log.Fatal will call os.Exit(1) to terminate
-				log.Fatal("Kafka service has become unhealthy")
+				// logger.Fatal will call os.Exit(1) to terminate
+				logger.Fatal("Kafka service has become unhealthy")
 			}
 		case liveliness := <-livelinessChannel:
 			if !liveliness {
@@ -432,13 +398,13 @@
 				<-timeoutTimer.C
 			}
 		case <-timeoutTimer.C:
-			log.Info("kafka-proxy-liveness-recheck")
+			logger.Info("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()
 			if err != nil {
 				// Catch possible error case if sending liveness after Sarama has been stopped.
-				log.Warnw("error-kafka-send-liveness", log.Fields{"error": err})
+				logger.Warnw("error-kafka-send-liveness", log.Fields{"error": err})
 			}
 		}
 	}
@@ -447,6 +413,18 @@
 // Adapter Utility methods ##### end   #########
 // #############################################
 
+func getVerifiedCodeVersion() string {
+	if version.VersionInfo.Version == "unknown-version" {
+		content, err := ioutil.ReadFile("VERSION")
+		if err == nil {
+			return (string(content))
+		} else {
+			logger.Error("'VERSION'-file not readable")
+		}
+	}
+	return version.VersionInfo.Version
+}
+
 func printVersion(appName string) {
 	fmt.Println(appName)
 	fmt.Println(version.VersionInfo.String("  "))
@@ -477,7 +455,7 @@
 	go func() {
 		select {
 		case <-ctx.Done():
-			log.Infow("Adapter run aborted due to internal errors", log.Fields{"context": "done"})
+			logger.Infow("Adapter run aborted due to internal errors", log.Fields{"context": "done"})
 			exitChannel <- 2
 		case s := <-signalChannel:
 			switch s {
@@ -485,10 +463,10 @@
 				syscall.SIGINT,
 				syscall.SIGTERM,
 				syscall.SIGQUIT:
-				log.Infow("closing-signal-received", log.Fields{"signal": s})
+				logger.Infow("closing-signal-received", log.Fields{"signal": s})
 				exitChannel <- 0
 			default:
-				log.Infow("unexpected-signal-received", log.Fields{"signal": s})
+				logger.Infow("unexpected-signal-received", log.Fields{"signal": s})
 				exitChannel <- 1
 			}
 		}
@@ -502,27 +480,29 @@
 	start := time.Now()
 
 	cf := config.NewAdapterFlags()
-	defaultAppName := cf.InstanceID + "_" + version.GetCodeVersion()
+	defaultAppName := cf.InstanceID + "_" + getVerifiedCodeVersion()
 	cf.ParseCommandArguments()
 
 	// Setup logging
 
-	loglevel, err := log.StringToLogLevel(cf.LogLevel)
+	logLevel, err := log.StringToLogLevel(cf.LogLevel)
 	if err != nil {
-		log.Fatalf("Cannot setup logging, %s", err)
+		logger.Fatalf("Cannot setup logging, %s", err)
 	}
 
 	// Setup default logger - applies for packages that do not have specific logger set
-	if _, err := log.SetDefaultLogger(log.JSON, loglevel, log.Fields{"instanceId": cf.InstanceID}); err != nil {
+	if _, err := log.SetDefaultLogger(log.JSON, logLevel, log.Fields{"instanceId": cf.InstanceID}); err != nil {
 		log.With(log.Fields{"error": err}).Fatal("Cannot setup logging")
 	}
 
-	// Update all loggers (provisionned via init) with a common field
+	// Update all loggers (provisioned via init) with a common field
 	if err := log.UpdateAllLoggers(log.Fields{"instanceId": cf.InstanceID}); err != nil {
-		log.With(log.Fields{"error": err}).Fatal("Cannot setup logging")
+		logger.With(log.Fields{"error": err}).Fatal("Cannot setup logging")
 	}
 
-	log.SetAllLogLevel(loglevel)
+	log.SetAllLogLevel(logLevel)
+
+	realMain() //fatal on httpListen(0,6060) ...
 
 	defer log.CleanUp()
 
@@ -531,9 +511,9 @@
 		printVersion(defaultAppName)
 		return
 	} else {
-		log.Infow("config", log.Fields{"StartName": defaultAppName})
-		log.Infow("config", log.Fields{"BuildVersion": version.VersionInfo.String("  ")})
-		log.Infow("config", log.Fields{"Arguments": os.Args[1:]})
+		logger.Infow("config", log.Fields{"StartName": defaultAppName})
+		logger.Infow("config", log.Fields{"BuildVersion": version.VersionInfo.String("  ")})
+		logger.Infow("config", log.Fields{"Arguments": os.Args[1:]})
 	}
 
 	// Print banner if specified
@@ -541,18 +521,18 @@
 		printBanner()
 	}
 
-	log.Infow("config", log.Fields{"config": *cf})
+	logger.Infow("config", log.Fields{"config": *cf})
 
 	ctx, cancel := context.WithCancel(context.Background())
-	//defer cancel()
+	defer cancel()
 
 	ad := newAdapter(cf)
 
 	p := &probe.Probe{}
-	log.Infow("resources", log.Fields{"Context": ctx, "Adapter": ad.getAdapterInstance(), "ProbeCoreState": p.GetStatus("register-with-core")})
+	logger.Infow("resources", log.Fields{"Context": ctx, "Adapter": ad.instanceID, "ProbeCoreState": p.GetStatus("register-with-core")})
 
 	go p.ListenAndServe(fmt.Sprintf("%s:%d", ad.config.ProbeHost, ad.config.ProbePort))
-	log.Infow("probeState", log.Fields{"ProbeCoreState": p.GetStatus("register-with-core")})
+	logger.Infow("probeState", log.Fields{"ProbeCoreState": p.GetStatus("register-with-core")})
 
 	probeCtx := context.WithValue(ctx, probe.ProbeContextKey, p)
 
@@ -566,12 +546,12 @@
 	}()
 
 	code := waitForExit(ctx)
-	log.Infow("received-a-closing-signal", log.Fields{"code": code})
+	logger.Infow("received-a-closing-signal", log.Fields{"code": code})
 
 	// Cleanup before leaving
 	ad.stop(ctx)
 
 	elapsed := time.Since(start)
-	log.Infow("run-time", log.Fields{"Name": "openadapter", "time": elapsed / time.Microsecond})
-	//log.Infow("run-time", log.Fields{"instanceId": ad.config.InstanceID, "time": elapsed / time.Second})
+	logger.Infow("run-time", log.Fields{"Name": "openadapter", "time": elapsed / time.Microsecond})
+	//logger.Infow("run-time", log.Fields{"instanceId": ad.config.InstanceID, "time": elapsed / time.Second})
 }
diff --git a/main_test.go b/cmd/openonu-adapter/main_test.go
similarity index 100%
rename from main_test.go
rename to cmd/openonu-adapter/main_test.go
diff --git a/cmd/openonu-adapter/profile.go b/cmd/openonu-adapter/profile.go
new file mode 100644
index 0000000..f539afd
--- /dev/null
+++ b/cmd/openonu-adapter/profile.go
@@ -0,0 +1,31 @@
+// +build profile
+
+/*
+ * 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 main
+
+import (
+	"net/http"
+	_ "net/http/pprof"
+)
+
+func realMain() {
+	go func() {
+		logger.Fatal(http.ListenAndServe("0.0.0.0:6060", nil))
+	}()
+
+}
diff --git a/cmd/openonu-adapter/release.go b/cmd/openonu-adapter/release.go
new file mode 100644
index 0000000..a8fb220
--- /dev/null
+++ b/cmd/openonu-adapter/release.go
@@ -0,0 +1,24 @@
+// +build !profile
+
+/*
+ * 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 main invokes the application
+package main
+
+func realMain() {
+	logger.Infoln("NOT PROFILING")
+}
diff --git a/docker/Dockerfile.openonu b/docker/Dockerfile.openonu
index dba6602..7c370e6 100644
--- a/docker/Dockerfile.openonu
+++ b/docker/Dockerfile.openonu
@@ -36,14 +36,14 @@
 ENV CGO_ENABLED=0
 RUN go build -mod=vendor -o /go/bin/openonu \
 	-ldflags \
-	"-X test.internal/openadapter/config/version.version=$org_label_schema_version \
-	 -X test.internal/openadapter/config/version.vcsRef=$org_label_schema_vcs_ref  \
-	 -X test.internal/openadapter/config/version.vcsDirty=$org_opencord_vcs_dirty \
-	 -X test.internal/openadapter/config/version.goVersion=$(go version 2>&1 | sed -E  's/.*go([0-9]+\.[0-9]+\.[0-9]+).*/\1/g') \
-	 -X test.internal/openadapter/config/version.os=$(go env GOHOSTOS) \
-	 -X test.internal/openadapter/config/version.arch=$(go env GOHOSTARCH) \
-	 -X test.internal/openadapter/config/version.buildTime=$org_label_schema_build_date"  \
-	 ./main.go
+	"-X github.com/opencord/voltha-lib-go/v3/pkg/version.version=$org_label_schema_version \
+	 -X github.com/opencord/voltha-lib-go/v3/pkg/version.vcsRef=$org_label_schema_vcs_ref  \
+	 -X github.com/opencord/voltha-lib-go/v3/pkg/version.vcsDirty=$org_opencord_vcs_dirty \
+	 -X github.com/opencord/voltha-lib-go/v3/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/v3/pkg/version.os=$(go env GOHOSTOS) \
+	 -X github.com/opencord/voltha-lib-go/v3/pkg/version.arch=$(go env GOHOSTARCH) \
+	 -X github.com/opencord/voltha-lib-go/v3/pkg/version.buildTime=$org_label_schema_build_date" \
+	 cmd/openonu-adapter/main.go cmd/openonu-adapter/main_test.go cmd/openonu-adapter/common.go cmd/openonu-adapter/profile.go
 
 # -------------
 # Image creation stage
diff --git a/go.sum b/go.sum
index 69d1557..d0aba8a 100644
--- a/go.sum
+++ b/go.sum
@@ -31,6 +31,7 @@
 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/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/circonus-labs/circonus-gometrics v2.3.1+incompatible/go.mod h1:nmEj6Dob7S7YxXgwXpfOuvO54S+tGdZdw9fuRZt25Ag=
 github.com/circonus-labs/circonusllhist v0.1.3/go.mod h1:kMXHVDlOchFAehlya5ePtbp5jckzBHf4XRpQvBOLI+I=
diff --git a/config/config.go b/internal/pkg/config/config.go
similarity index 93%
rename from config/config.go
rename to internal/pkg/config/config.go
index 52f43fa..b3fc4d0 100644
--- a/config/config.go
+++ b/internal/pkg/config/config.go
@@ -22,8 +22,6 @@
 	"fmt"
 	"os"
 	"time"
-
-	"github.com/opencord/voltha-lib-go/v3/pkg/log"
 )
 
 // Open ONU default constants
@@ -38,7 +36,7 @@
 	defaultKvstoretimeout       = 5 //in seconds
 	defaultKvstorehost          = "localhost"
 	defaultKvstoreport          = 2379 // Consul = 8500; Etcd = 2379
-	defaultLoglevel             = "DEBUG"
+	defaultLoglevel             = "WARN"
 	defaultBanner               = false
 	defaultDisplayVersionOnly   = false
 	defaultTopic                = "openonu"
@@ -55,6 +53,8 @@
 	defaultHearbeatFailReportInterval = 180 * time.Second
 	//defaultKafkaReconnectRetries -1: reconnect endlessly.
 	defaultKafkaReconnectRetries = -1
+	defaultCurrentReplica        = 1
+	defaultTotalReplicas         = 1
 )
 
 // AdapterFlags represents the set of configurations used by the read-write adaptercore service
@@ -83,10 +83,8 @@
 	HeartbeatCheckInterval      time.Duration
 	HeartbeatFailReportInterval time.Duration
 	KafkaReconnectRetries       int
-}
-
-func init() {
-	_, _ = log.AddPackage(log.JSON, log.WarnLevel, nil)
+	CurrentReplica              int
+	TotalReplicas               int
 }
 
 // NewAdapterFlags returns a new RWCore config
@@ -115,6 +113,8 @@
 		HeartbeatCheckInterval:      defaultHearbeatCheckInterval,
 		HeartbeatFailReportInterval: defaultHearbeatFailReportInterval,
 		KafkaReconnectRetries:       defaultKafkaReconnectRetries,
+		CurrentReplica:              defaultCurrentReplica,
+		TotalReplicas:               defaultTotalReplicas,
 	}
 	return &adapterFlags
 }
@@ -188,6 +188,12 @@
 	help = fmt.Sprintf("Number of retries to connect to Kafka.")
 	flag.IntVar(&(so.KafkaReconnectRetries), "kafka_reconnect_retries", defaultKafkaReconnectRetries, help)
 
+	help = "Replica number of this particular instance (default: %s)"
+	flag.IntVar(&(so.CurrentReplica), "current_replica", defaultCurrentReplica, help)
+
+	help = "Total number of instances for this adapter"
+	flag.IntVar(&(so.TotalReplicas), "total_replica", defaultTotalReplicas, help)
+
 	flag.Parse()
 	containerName := getContainerInfo()
 	if len(containerName) > 0 {
diff --git a/internal/pkg/onuadaptercore/common.go b/internal/pkg/onuadaptercore/common.go
new file mode 100644
index 0000000..ed55bbd
--- /dev/null
+++ b/internal/pkg/onuadaptercore/common.go
@@ -0,0 +1,33 @@
+/*
+ * 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 core Common Logger initialization
+package adaptercoreonu
+
+import (
+	"github.com/opencord/voltha-lib-go/v3/pkg/log"
+)
+
+var logger log.Logger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.AddPackage(log.JSON, log.ErrorLevel, log.Fields{"pkg": "adaptercoreonu"})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/internal/pkg/onuadaptercore/device_handler.go b/internal/pkg/onuadaptercore/device_handler.go
new file mode 100644
index 0000000..4f0b030
--- /dev/null
+++ b/internal/pkg/onuadaptercore/device_handler.go
@@ -0,0 +1,823 @@
+/*
+ * 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 adaptercoreonu provides the utility for onu devices, flows and statistics
+package adaptercoreonu
+
+import (
+	"context"
+	"encoding/hex"
+	"errors"
+	"fmt"
+	"sync"
+	"time"
+
+	"github.com/gogo/protobuf/proto"
+	"github.com/golang/protobuf/ptypes"
+	"github.com/looplab/fsm"
+	"github.com/opencord/voltha-lib-go/v3/pkg/adapters/adapterif"
+	"github.com/opencord/voltha-lib-go/v3/pkg/log"
+	ic "github.com/opencord/voltha-protos/v3/go/inter_container"
+	oop "github.com/opencord/voltha-protos/v3/go/openolt"
+	"github.com/opencord/voltha-protos/v3/go/voltha"
+)
+
+/*
+// Constants for number of retries and for timeout
+const (
+	MaxRetry       = 10
+	MaxTimeOutInMs = 500
+)
+*/
+
+//DeviceHandler will interact with the ONU ? device.
+type DeviceHandler struct {
+	deviceID         string
+	DeviceType       string
+	adminState       string
+	device           *voltha.Device
+	logicalDeviceID  string
+	ProxyAddressID   string
+	ProxyAddressType string
+
+	coreProxy       adapterif.CoreProxy
+	AdapterProxy    adapterif.AdapterProxy
+	EventProxy      adapterif.EventProxy
+	pOpenOnuAc      *OpenONUAC
+	pDeviceStateFsm *fsm.FSM
+	pPonPort        *voltha.Port
+	pOnuOmciDevice  *OnuDeviceEntry
+	exitChannel     chan int
+	lockDevice      sync.RWMutex
+
+	//Client        oop.OpenoltClient
+	//clientCon     *grpc.ClientConn
+	//flowMgr       *OpenOltFlowMgr
+	//eventMgr      *OpenOltEventMgr
+	//resourceMgr   *rsrcMgr.OpenOltResourceMgr
+
+	//discOnus sync.Map
+	//onus     sync.Map
+	//portStats          *OpenOltStatisticsMgr
+	//metrics            *pmmetrics.PmMetrics
+	stopCollector      chan bool
+	stopHeartbeatCheck chan bool
+	activePorts        sync.Map
+	uniEntityMap       map[uint16]*OnuUniPort
+}
+
+/*
+//OnuDevice represents ONU related info
+type OnuDevice struct {
+	deviceID      string
+	deviceType    string
+	serialNumber  string
+	onuID         uint32
+	intfID        uint32
+	proxyDeviceID string
+	uniPorts      map[uint32]struct{}
+}
+
+//NewOnuDevice creates a new Onu Device
+func NewOnuDevice(devID, deviceTp, serialNum string, onuID, intfID uint32, proxyDevID string) *OnuDevice {
+	var device OnuDevice
+	device.deviceID = devID
+	device.deviceType = deviceTp
+	device.serialNumber = serialNum
+	device.onuID = onuID
+	device.intfID = intfID
+	device.proxyDeviceID = proxyDevID
+	device.uniPorts = make(map[uint32]struct{})
+	return &device
+}
+*/
+
+//NewDeviceHandler creates a new device handler
+func NewDeviceHandler(cp adapterif.CoreProxy, ap adapterif.AdapterProxy, ep adapterif.EventProxy, device *voltha.Device, adapter *OpenONUAC) *DeviceHandler {
+	var dh DeviceHandler
+	dh.coreProxy = cp
+	dh.AdapterProxy = ap
+	dh.EventProxy = ep
+	cloned := (proto.Clone(device)).(*voltha.Device)
+	dh.deviceID = cloned.Id
+	dh.DeviceType = cloned.Type
+	dh.adminState = "up"
+	dh.device = cloned
+	dh.pOpenOnuAc = adapter
+	dh.exitChannel = make(chan int, 1)
+	dh.lockDevice = sync.RWMutex{}
+	dh.stopCollector = make(chan bool, 2)
+	dh.stopHeartbeatCheck = make(chan bool, 2)
+	//dh.metrics = pmmetrics.NewPmMetrics(cloned.Id, pmmetrics.Frequency(150), pmmetrics.FrequencyOverride(false), pmmetrics.Grouped(false), pmmetrics.Metrics(pmNames))
+	dh.activePorts = sync.Map{}
+	//TODO initialize the support classes.
+	dh.uniEntityMap = make(map[uint16]*OnuUniPort)
+
+	// Device related state machine
+	dh.pDeviceStateFsm = fsm.NewFSM(
+		"null",
+		fsm.Events{
+			{Name: "DeviceInit", Src: []string{"null", "down"}, Dst: "init"},
+			{Name: "GrpcConnected", Src: []string{"init"}, Dst: "connected"},
+			{Name: "GrpcDisconnected", Src: []string{"connected", "down"}, Dst: "init"},
+			{Name: "DeviceUpInd", Src: []string{"connected", "down"}, Dst: "up"},
+			{Name: "DeviceDownInd", Src: []string{"up"}, Dst: "down"},
+		},
+		fsm.Callbacks{
+			"before_event":            func(e *fsm.Event) { dh.logStateChange(e) },
+			"before_DeviceInit":       func(e *fsm.Event) { dh.doStateInit(e) },
+			"after_DeviceInit":        func(e *fsm.Event) { dh.postInit(e) },
+			"before_GrpcConnected":    func(e *fsm.Event) { dh.doStateConnected(e) },
+			"before_GrpcDisconnected": func(e *fsm.Event) { dh.doStateInit(e) },
+			"after_GrpcDisconnected":  func(e *fsm.Event) { dh.postInit(e) },
+			"before_DeviceUpInd":      func(e *fsm.Event) { dh.doStateUp(e) },
+			"before_DeviceDownInd":    func(e *fsm.Event) { dh.doStateDown(e) },
+		},
+	)
+	return &dh
+}
+
+// start save the device to the data model
+func (dh *DeviceHandler) Start(ctx context.Context) {
+	logger.Debugw("starting-device-handler", log.Fields{"device": dh.device, "deviceId": dh.deviceID})
+	// Add the initial device to the local model
+	logger.Debug("device-handler-started")
+}
+
+// stop stops the device dh.  Not much to do for now
+func (dh *DeviceHandler) stop(ctx context.Context) {
+	logger.Debug("stopping-device-handler")
+	dh.exitChannel <- 1
+}
+
+// ##########################################################################################
+// DeviceHandler methods that implement the adapters interface requests ##### begin #########
+
+//AdoptDevice adopts the OLT device
+func (dh *DeviceHandler) AdoptDevice(ctx context.Context, device *voltha.Device) {
+	logger.Debugw("Adopt_device", log.Fields{"deviceID": device.Id, "Address": device.GetHostAndPort()})
+
+	logger.Debug("Device FSM: ", log.Fields{"state": string(dh.pDeviceStateFsm.Current())})
+	if dh.pDeviceStateFsm.Is("null") {
+		if err := dh.pDeviceStateFsm.Event("DeviceInit"); err != nil {
+			logger.Errorw("Device FSM: Can't go to state DeviceInit", log.Fields{"err": err})
+		}
+		logger.Debug("Device FSM: ", log.Fields{"state": string(dh.pDeviceStateFsm.Current())})
+	} else {
+		logger.Debug("AdoptDevice: Agent/device init already done")
+	}
+
+	/*
+		// Now, set the initial PM configuration for that device
+		if err := dh.coreProxy.DevicePMConfigUpdate(nil, dh.metrics.ToPmConfigs()); err != nil {
+			logger.Errorw("error-updating-PMs", log.Fields{"deviceId": device.Id, "error": err})
+		}
+
+		go startCollector(dh)
+		go startHeartbeatCheck(dh)
+	*/
+}
+
+//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(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("InterAdapter message header", log.Fields{"msgID": msgID, "msgType": msgType,
+		"fromTopic": fromTopic, "toTopic": toTopic, "toDeviceID": toDeviceID, "proxyDeviceID": proxyDeviceID})
+
+	switch msgType {
+	case ic.InterAdapterMessageType_OMCI_REQUEST:
+		{
+			/* TOBECHECKED: I assume, ONU Adapter receives the message hier already 'unmarshalled'? else: (howTo?)*/
+			msgBody := msg.GetBody()
+
+			omciMsg := &ic.InterAdapterOmciMessage{}
+			if err := ptypes.UnmarshalAny(msgBody, omciMsg); err != nil {
+				logger.Warnw("cannot-unmarshal-omci-msg-body", log.Fields{"error": err})
+				return err
+			}
+
+			//assuming omci message content is hex coded!
+			// with restricted output of 16(?) bytes would be ...omciMsg.Message[:16]
+			logger.Debugw("inter-adapter-recv-omci",
+				log.Fields{"RxOmciMessage": hex.EncodeToString(omciMsg.Message)})
+			//receive_message(omci_msg.message)
+			return dh.GetOnuDeviceEntry().PDevOmciCC.ReceiveMessage(context.TODO(), omciMsg.Message)
+		}
+	case ic.InterAdapterMessageType_ONU_IND_REQUEST:
+		{
+			/* TOBECHECKED: I assume, ONU Adapter receives the message hier already 'unmarshalled'? else: see above omci block */
+			msgBody := msg.GetBody()
+
+			onu_indication := &oop.OnuIndication{}
+			if err := ptypes.UnmarshalAny(msgBody, onu_indication); err != nil {
+				logger.Warnw("cannot-unmarshal-onu-indication-msg-body", log.Fields{"error": err})
+				return err
+			}
+
+			onu_operstate := onu_indication.GetOperState()
+			logger.Debugw("inter-adapter-recv-onu-ind", log.Fields{"OnuId": onu_indication.GetOnuId(),
+				"AdminState": onu_indication.GetAdminState(), "OperState": onu_operstate,
+				"SNR": onu_indication.GetSerialNumber()})
+
+			//interface related functioons might be error checked ....
+			if onu_operstate == "up" {
+				dh.create_interface(onu_indication)
+			} else if (onu_operstate == "down") || (onu_operstate == "unreachable") {
+				dh.update_interface(onu_indication)
+			} else {
+				logger.Errorw("unknown-onu-indication operState", log.Fields{"OnuId": onu_indication.GetOnuId()})
+				return errors.New("InvalidOperState")
+			}
+		}
+	default:
+		{
+			logger.Errorw("inter-adapter-unhandled-type", log.Fields{"msgType": msg.Header.Type})
+			return errors.New("unimplemented")
+		}
+	}
+
+	/* form py code:
+	   elif request.header.type == InterAdapterMessageType.TECH_PROFILE_DOWNLOAD_REQUEST:
+	       tech_msg = InterAdapterTechProfileDownloadMessage()
+	       request.body.Unpack(tech_msg)
+	       self.logger.debug('inter-adapter-recv-tech-profile', tech_msg=tech_msg)
+
+	       self.load_and_configure_tech_profile(tech_msg.uni_id, tech_msg.path)
+
+	   elif request.header.type == InterAdapterMessageType.DELETE_GEM_PORT_REQUEST:
+	       del_gem_msg = InterAdapterDeleteGemPortMessage()
+	       request.body.Unpack(del_gem_msg)
+	       self.logger.debug('inter-adapter-recv-del-gem', gem_del_msg=del_gem_msg)
+
+	       self.delete_tech_profile(uni_id=del_gem_msg.uni_id,
+	                                gem_port_id=del_gem_msg.gem_port_id,
+	                                tp_path=del_gem_msg.tp_path)
+
+	   elif request.header.type == InterAdapterMessageType.DELETE_TCONT_REQUEST:
+	       del_tcont_msg = InterAdapterDeleteTcontMessage()
+	       request.body.Unpack(del_tcont_msg)
+	       self.logger.debug('inter-adapter-recv-del-tcont', del_tcont_msg=del_tcont_msg)
+
+	       self.delete_tech_profile(uni_id=del_tcont_msg.uni_id,
+	                                alloc_id=del_tcont_msg.alloc_id,
+	                                tp_path=del_tcont_msg.tp_path)
+	   else:
+	       self.logger.error("inter-adapter-unhandled-type", request=request)
+	*/
+	return nil
+}
+
+//  DeviceHandler methods that implement the adapters interface requests## end #########
+// #####################################################################################
+
+// ################  to be updated acc. needs of ONU Device ########################
+// DeviceHandler StateMachine related state transition methods ##### begin #########
+
+func (dh *DeviceHandler) logStateChange(e *fsm.Event) {
+	logger.Debugw("Device FSM: ", log.Fields{"event name": string(e.Event), "src state": string(e.Src), "dst state": string(e.Dst), "device-id": dh.deviceID})
+}
+
+// doStateInit provides the device update to the core
+func (dh *DeviceHandler) doStateInit(e *fsm.Event) {
+
+	logger.Debug("doStateInit-started")
+	var err error
+
+	/*
+		var err error
+		dh.clientCon, err = grpc.Dial(dh.device.GetHostAndPort(), grpc.WithInsecure(), grpc.WithBlock())
+		if err != nil {
+			logger.Errorw("Failed to dial device", log.Fields{"DeviceId": dh.deviceID, "HostAndPort": dh.device.GetHostAndPort(), "err": err})
+			return err
+		}
+		return nil
+	*/
+
+	// populate what we know.  rest comes later after mib sync
+	dh.device.Root = false
+	dh.device.Vendor = "OpenONU"
+	dh.device.Model = "go"
+	dh.device.Reason = "activating-onu"
+	dh.logicalDeviceID = dh.deviceID
+
+	dh.coreProxy.DeviceUpdate(context.TODO(), dh.device)
+
+	// store proxy parameters for later communication - assumption: invariant, else they have to be requested dynamically!!
+	dh.ProxyAddressID = dh.device.ProxyAddress.GetDeviceId()
+	dh.ProxyAddressType = dh.device.ProxyAddress.GetDeviceType()
+	logger.Debugw("device-updated", log.Fields{"deviceID": dh.deviceID, "proxyAddressID": dh.ProxyAddressID,
+		"proxyAddressType": dh.ProxyAddressType, "SNR": dh.device.SerialNumber,
+		"ParentId": dh.device.ParentId, "ParentPortNo": dh.device.ParentPortNo})
+
+	/*
+		self._pon = PonPort.create(self, self._pon_port_number)
+		self._pon.add_peer(self.parent_id, self._pon_port_number)
+		self.logger.debug('adding-pon-port-to-agent',
+				   type=self._pon.get_port().type,
+				   admin_state=self._pon.get_port().admin_state,
+				   oper_status=self._pon.get_port().oper_status,
+				   )
+	*/
+	logger.Debug("adding-pon-port")
+	pPonPortNo := uint32(1)
+	if dh.device.ParentPortNo != 0 {
+		pPonPortNo = dh.device.ParentPortNo
+	}
+
+	pPonPort := &voltha.Port{
+		PortNo:     pPonPortNo,
+		Label:      fmt.Sprintf("pon-%d", pPonPortNo),
+		Type:       voltha.Port_PON_ONU,
+		OperStatus: voltha.OperStatus_ACTIVE,
+		Peers: []*voltha.Port_PeerPort{{DeviceId: dh.device.ParentId, // Peer device  is OLT
+			PortNo: dh.device.ParentPortNo}}, // Peer port is parent's port number
+	}
+	if err = dh.coreProxy.PortCreated(context.TODO(), dh.deviceID, pPonPort); err != nil {
+		logger.Fatalf("Device FSM: PortCreated-failed-%s", err)
+		e.Cancel(err)
+		return
+	}
+	logger.Debug("doStateInit-done")
+}
+
+// postInit setups the DeviceEntry for the conerned device
+func (dh *DeviceHandler) postInit(e *fsm.Event) {
+
+	logger.Debug("postInit-started")
+	var err error
+	/*
+		dh.Client = oop.NewOpenoltClient(dh.clientCon)
+		dh.pTransitionMap.Handle(ctx, GrpcConnected)
+		return nil
+	*/
+	if err = dh.Add_OnuDeviceEntry(context.TODO()); err != nil {
+		logger.Fatalf("Device FSM: Add_OnuDeviceEntry-failed-%s", err)
+		e.Cancel(err)
+		return
+	}
+
+	/*
+			############################################################################
+			# Setup Alarm handler
+			self.events = AdapterEvents(self.core_proxy, device.id, self.logical_device_id,
+										device.serial_number)
+			############################################################################
+			# Setup PM configuration for this device
+			# Pass in ONU specific options
+			kwargs = {
+				OnuPmMetrics.DEFAULT_FREQUENCY_KEY: OnuPmMetrics.DEFAULT_ONU_COLLECTION_FREQUENCY,
+				'heartbeat': self.heartbeat,
+				OnuOmciPmMetrics.OMCI_DEV_KEY: self._onu_omci_device
+			}
+			self.logger.debug('create-pm-metrics', device_id=device.id, serial_number=device.serial_number)
+			self._pm_metrics = OnuPmMetrics(self.events, self.core_proxy, self.device_id,
+										   self.logical_device_id, device.serial_number,
+										   grouped=True, freq_override=False, **kwargs)
+			pm_config = self._pm_metrics.make_proto()
+			self._onu_omci_device.set_pm_config(self._pm_metrics.omci_pm.openomci_interval_pm)
+			self.logger.info("initial-pm-config", device_id=device.id, serial_number=device.serial_number)
+			yield self.core_proxy.device_pm_config_update(pm_config, init=True)
+
+			# Note, ONU ID and UNI intf set in add_uni_port method
+			self._onu_omci_device.alarm_synchronizer.set_alarm_params(mgr=self.events,
+																	  ani_ports=[self._pon])
+
+			# Code to Run OMCI Test Action
+			kwargs_omci_test_action = {
+				OmciTestRequest.DEFAULT_FREQUENCY_KEY:
+					OmciTestRequest.DEFAULT_COLLECTION_FREQUENCY
+			}
+			serial_number = device.serial_number
+			self._test_request = OmciTestRequest(self.core_proxy,
+										   self.omci_agent, self.device_id,
+										   AniG, serial_number,
+										   self.logical_device_id,
+										   exclusive=False,
+										   **kwargs_omci_test_action)
+
+			self.enabled = True
+		else:
+			self.logger.info('onu-already-activated')
+	*/
+	logger.Debug("postInit-done")
+}
+
+// doStateConnected get the device info and update to voltha core
+// for comparison of the original method (not that easy to uncomment): compare here:
+//  voltha-openolt-adapter/adaptercore/device_handler.go
+//  -> this one obviously initiates all communication interfaces of the device ...?
+func (dh *DeviceHandler) doStateConnected(e *fsm.Event) {
+
+	logger.Debug("doStateConnected-started")
+	var err error
+	err = errors.New("Device FSM: function not implemented yet!")
+	e.Cancel(err)
+	return
+	logger.Debug("doStateConnected-done")
+}
+
+// doStateUp handle the onu up indication and update to voltha core
+func (dh *DeviceHandler) doStateUp(e *fsm.Event) {
+
+	logger.Debug("doStateUp-started")
+	var err error
+	err = errors.New("Device FSM: function not implemented yet!")
+	e.Cancel(err)
+	return
+	logger.Debug("doStateUp-done")
+
+	/*
+		// Synchronous call to update device state - this method is run in its own go routine
+		if err := dh.coreProxy.DeviceStateUpdate(ctx, dh.device.Id, voltha.ConnectStatus_REACHABLE,
+			voltha.OperStatus_ACTIVE); err != nil {
+			logger.Errorw("Failed to update device with OLT UP indication", log.Fields{"deviceID": dh.device.Id, "error": err})
+			return err
+		}
+		return nil
+	*/
+}
+
+// doStateDown handle the onu down indication
+func (dh *DeviceHandler) doStateDown(e *fsm.Event) {
+
+	logger.Debug("doStateDown-started")
+	var err error
+
+	device, err := dh.coreProxy.GetDevice(context.TODO(), dh.device.Id, dh.device.Id)
+	if err != nil || device == nil {
+		/*TODO: needs to handle error scenarios */
+		logger.Errorw("Failed to fetch device device", log.Fields{"err": err})
+		e.Cancel(err)
+		return
+	}
+
+	cloned := proto.Clone(device).(*voltha.Device)
+	logger.Debugw("do-state-down", log.Fields{"ClonedDeviceID": cloned.Id})
+	/*
+		// Update the all ports state on that device to disable
+		if er := dh.coreProxy.PortsStateUpdate(ctx, cloned.Id, voltha.OperStatus_UNKNOWN); er != nil {
+			logger.Errorw("updating-ports-failed", log.Fields{"deviceID": device.Id, "error": er})
+			return er
+		}
+
+		//Update the device oper state and connection status
+		cloned.OperStatus = voltha.OperStatus_UNKNOWN
+		cloned.ConnectStatus = common.ConnectStatus_UNREACHABLE
+		dh.device = cloned
+
+		if er := dh.coreProxy.DeviceStateUpdate(ctx, cloned.Id, cloned.ConnectStatus, cloned.OperStatus); er != nil {
+			logger.Errorw("error-updating-device-state", log.Fields{"deviceID": device.Id, "error": er})
+			return er
+		}
+
+		//get the child device for the parent device
+		onuDevices, err := dh.coreProxy.GetChildDevices(ctx, dh.device.Id)
+		if err != nil {
+			logger.Errorw("failed to get child devices information", log.Fields{"deviceID": dh.device.Id, "error": err})
+			return err
+		}
+		for _, onuDevice := range onuDevices.Items {
+
+			// Update onu state as down in onu adapter
+			onuInd := oop.OnuIndication{}
+			onuInd.OperState = "down"
+			er := dh.AdapterProxy.SendInterAdapterMessage(ctx, &onuInd, ic.InterAdapterMessageType_ONU_IND_REQUEST,
+				"openolt", onuDevice.Type, onuDevice.Id, onuDevice.ProxyAddress.DeviceId, "")
+			if er != nil {
+				logger.Errorw("Failed to send inter-adapter-message", log.Fields{"OnuInd": onuInd,
+					"From Adapter": "openolt", "DevieType": onuDevice.Type, "DeviceID": onuDevice.Id})
+				//Do not return here and continue to process other ONUs
+			}
+		}
+		// * Discovered ONUs entries need to be cleared , since after OLT
+		//   is up, it starts sending discovery indications again* /
+		dh.discOnus = sync.Map{}
+		logger.Debugw("do-state-down-end", log.Fields{"deviceID": device.Id})
+		return nil
+	*/
+	err = errors.New("Device FSM: function not implemented yet!")
+	e.Cancel(err)
+	return
+	logger.Debug("doStateDown-done")
+}
+
+// DeviceHandler StateMachine related state transition methods ##### end #########
+// #################################################################################
+
+// ###################################################
+// DeviceHandler utility methods ##### begin #########
+
+// Get ONU device entry for this deviceId specific handler
+func (dh *DeviceHandler) GetOnuDeviceEntry() *OnuDeviceEntry {
+	dh.lockDevice.Lock()
+	defer dh.lockDevice.Unlock()
+	if dh.pOnuOmciDevice != nil {
+		logger.Debugw("GetOnuDeviceEntry params:",
+			log.Fields{"onu_device_entry": dh.pOnuOmciDevice, "device_id": dh.pOnuOmciDevice.deviceID,
+				"device_handler": dh.pOnuOmciDevice.baseDeviceHandler, "core_proxy": dh.pOnuOmciDevice.coreProxy, "adapter_proxy": dh.pOnuOmciDevice.adapterProxy})
+	} else {
+		logger.Error("GetOnuDeviceEntry returns nil")
+	}
+	return dh.pOnuOmciDevice
+}
+
+// Set ONU device entry
+func (dh *DeviceHandler) SetOnuDeviceEntry(pDeviceEntry *OnuDeviceEntry) error {
+	dh.lockDevice.Lock()
+	defer dh.lockDevice.Unlock()
+	dh.pOnuOmciDevice = pDeviceEntry
+	return nil
+}
+
+//creates a new ONU device or returns the existing
+func (dh *DeviceHandler) Add_OnuDeviceEntry(ctx context.Context) error {
+	logger.Debugw("adding-deviceEntry", log.Fields{"for deviceId": dh.deviceID})
+
+	deviceEntry := dh.GetOnuDeviceEntry()
+	if deviceEntry == nil {
+		/* costum_me_map in python code seems always to be None,
+		   we omit that here first (declaration unclear) -> todo at Adapter specialization ...*/
+		/* also no 'clock' argument - usage open ...*/
+		/* and no alarm_db yet (oo.alarm_db)  */
+		deviceEntry = NewOnuDeviceEntry(ctx, dh.deviceID, dh, dh.coreProxy, dh.AdapterProxy,
+			dh.pOpenOnuAc.pSupportedFsms) //nil as FSM pointer would yield deviceEntry internal defaults ...
+		//error treatment possible //TODO!!!
+		dh.SetOnuDeviceEntry(deviceEntry)
+		logger.Infow("onuDeviceEntry-added", log.Fields{"for deviceId": dh.deviceID})
+	} else {
+		logger.Infow("onuDeviceEntry-add: Device already exists", log.Fields{"for deviceId": dh.deviceID})
+	}
+	// might be updated with some error handling !!!
+	return nil
+}
+
+// doStateInit provides the device update to the core
+func (dh *DeviceHandler) create_interface(onuind *oop.OnuIndication) error {
+	logger.Debug("create_interface-started - not yet fully implemented (only device state update)")
+
+	if err := dh.coreProxy.DeviceStateUpdate(context.TODO(), dh.deviceID, voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVATING); err != nil {
+		logger.Errorw("error-updating-device-state", log.Fields{"deviceID": dh.deviceID, "error": err})
+	}
+
+	device, err := dh.coreProxy.GetDevice(context.TODO(), dh.device.Id, dh.device.Id)
+	if err != nil || device == nil {
+		/*TODO: needs to handle error scenarios */
+		logger.Errorw("Failed to fetch device device at creating If", log.Fields{"err": err})
+		return errors.New("Voltha Device not found")
+	}
+
+	dh.GetOnuDeviceEntry().Start(context.TODO())
+	if err := dh.coreProxy.DeviceReasonUpdate(context.TODO(), dh.deviceID, "starting-openomci"); err != nil {
+		logger.Errorw("error-DeviceReasonUpdate to starting-openomci", log.Fields{"deviceID": dh.deviceID, "error": err})
+	}
+
+	/* this might be a good time for Omci Verify message?  */
+	verifyExec := make(chan bool)
+	omci_verify := NewOmciTestRequest(context.TODO(),
+		dh.device.Id, dh.GetOnuDeviceEntry().PDevOmciCC,
+		true, true) //eclusive and allowFailure (anyway not yet checked)
+	omci_verify.PerformOmciTest(context.TODO(), verifyExec)
+
+	/* 	give the handler some time here to wait for the OMCi verification result
+	after Timeout start and try MibUpload FSM anyway
+	(to prevent stopping on just not supported OMCI verification from ONU) */
+	select {
+	case <-time.After(2 * time.Second):
+		logger.Warn("omci start-verification timed out (continue normal)")
+	case testresult := <-verifyExec:
+		logger.Infow("Omci start verification done", log.Fields{"result": testresult})
+	}
+
+	/* In py code it looks earlier (on activate ..)
+			# Code to Run OMCI Test Action
+			kwargs_omci_test_action = {
+				OmciTestRequest.DEFAULT_FREQUENCY_KEY:
+					OmciTestRequest.DEFAULT_COLLECTION_FREQUENCY
+			}
+			serial_number = device.serial_number
+			self._test_request = OmciTestRequest(self.core_proxy,
+											self.omci_agent, self.device_id,
+											AniG, serial_number,
+											self.logical_device_id,
+											exclusive=False,
+											**kwargs_omci_test_action)
+	...
+	                    # Start test requests after a brief pause
+	                    if not self._test_request_started:
+	                        self._test_request_started = True
+	                        tststart = _STARTUP_RETRY_WAIT * (random.randint(1, 5))
+	                        reactor.callLater(tststart, self._test_request.start_collector)
+
+	*/
+	/* which is then: in omci_test_request.py : */
+	/*
+	   def start_collector(self, callback=None):
+	       """
+	               Start the collection loop for an adapter if the frequency > 0
+
+	               :param callback: (callable) Function to call to collect PM data
+	       """
+	       self.logger.info("starting-pm-collection", device_name=self.name, default_freq=self.default_freq)
+	       if callback is None:
+	           callback = self.perform_test_omci
+
+	       if self.lc is None:
+	           self.lc = LoopingCall(callback)
+
+	       if self.default_freq > 0:
+	           self.lc.start(interval=self.default_freq / 10)
+
+	   def perform_test_omci(self):
+	       """
+	       Perform the initial test request
+	       """
+	       ani_g_entities = self._device.configuration.ani_g_entities
+	       ani_g_entities_ids = list(ani_g_entities.keys()) if ani_g_entities \
+	                                                     is not None else None
+	       self._entity_id = ani_g_entities_ids[0]
+	       self.logger.info('perform-test', entity_class=self._entity_class,
+	                     entity_id=self._entity_id)
+	       try:
+	           frame = MEFrame(self._entity_class, self._entity_id, []).test()
+	           result = yield self._device.omci_cc.send(frame)
+	           if not result.fields['omci_message'].fields['success_code']:
+	               self.logger.info('Self-Test Submitted Successfully',
+	                             code=result.fields[
+	                                 'omci_message'].fields['success_code'])
+	           else:
+	               raise TestFailure('Test Failure: {}'.format(
+	                   result.fields['omci_message'].fields['success_code']))
+	       except TimeoutError as e:
+	           self.deferred.errback(failure.Failure(e))
+
+	       except Exception as e:
+	           self.logger.exception('perform-test-Error', e=e,
+	                              class_id=self._entity_class,
+	                              entity_id=self._entity_id)
+	           self.deferred.errback(failure.Failure(e))
+
+	*/
+
+	// PM related heartbeat??? !!!TODO....
+	//self._heartbeat.enabled = True
+
+	//example how to call FSM - transition up to state "uploading"
+	if dh.GetOnuDeviceEntry().MibSyncFsm.Is("disabled") {
+
+		if err := dh.GetOnuDeviceEntry().MibSyncFsm.Event("start"); err != nil {
+			logger.Errorw("MibSyncFsm: Can't go to state starting", log.Fields{"err": err})
+			return errors.New("Can't go to state starting")
+		} else {
+			logger.Debug("MibSyncFsm", log.Fields{"state": string(dh.GetOnuDeviceEntry().MibSyncFsm.Current())})
+			//Determine ONU status and start/re-start MIB Synchronization tasks
+			//Determine if this ONU has ever synchronized
+			if true { //TODO: insert valid check
+				if err := dh.GetOnuDeviceEntry().MibSyncFsm.Event("load_mib_template"); err != nil {
+					logger.Errorw("MibSyncFsm: Can't go to state loading_mib_template", log.Fields{"err": err})
+					return errors.New("Can't go to state loading_mib_template")
+				} else {
+					logger.Debug("MibSyncFsm", log.Fields{"state": string(dh.GetOnuDeviceEntry().MibSyncFsm.Current())})
+					//Find and load a mib template. If not found proceed with mib_upload
+					// callbacks to be handled:
+					// Event("success")
+					// Event("timeout")
+					//no mib template found
+					if true { //TODO: insert valid check
+						if err := dh.GetOnuDeviceEntry().MibSyncFsm.Event("upload_mib"); err != nil {
+							logger.Errorw("MibSyncFsm: Can't go to state uploading", log.Fields{"err": err})
+							return errors.New("Can't go to state uploading")
+						} else {
+							logger.Debug("state of MibSyncFsm", log.Fields{"state": string(dh.GetOnuDeviceEntry().MibSyncFsm.Current())})
+							//Begin full MIB data upload, starting with a MIB RESET
+							// callbacks to be handled:
+							// success: e.Event("success")
+							// failure: e.Event("timeout")
+						}
+					}
+				}
+			} else {
+				dh.GetOnuDeviceEntry().MibSyncFsm.Event("examine_mds")
+				logger.Debug("state of MibSyncFsm", log.Fields{"state": string(dh.GetOnuDeviceEntry().MibSyncFsm.Current())})
+				//Examine the MIB Data Sync
+				// callbacks to be handled:
+				// Event("success")
+				// Event("timeout")
+				// Event("mismatch")
+			}
+		}
+	} else {
+		logger.Errorw("wrong state of MibSyncFsm - want: disabled", log.Fields{"have": string(dh.GetOnuDeviceEntry().MibSyncFsm.Current())})
+		return errors.New("wrong state of MibSyncFsm")
+	}
+	return nil
+}
+
+func (dh *DeviceHandler) update_interface(onuind *oop.OnuIndication) error {
+	logger.Debug("update_interface-started - not yet implemented")
+	return nil
+}
+
+func (dh *DeviceHandler) DeviceStateUpdate(dev_Event OnuDeviceEvent) {
+	if dev_Event == MibDatabaseSync {
+		logger.Debug("MibInSync event: update dev state to 'MibSync complete'")
+		//initiate DevStateUpdate
+		if err := dh.coreProxy.DeviceReasonUpdate(context.TODO(), dh.deviceID, "discovery-mibsync-complete"); err != nil {
+			logger.Errorw("error-DeviceReasonUpdate to 'mibsync-complete'", log.Fields{"deviceID": dh.deviceID, "error": err})
+		}
+
+		// fixed assumption about PPTP/UNI-G ONU-config
+		// to be replaced by DB parsing of MibUpload data TODO!!!
+		// parameters are: InstanceNo, running UniNo, type
+		dh.addUniPort(257, 0, UniPPTP)
+		dh.addUniPort(258, 1, UniPPTP)
+		dh.addUniPort(259, 2, UniPPTP)
+		dh.addUniPort(260, 3, UniPPTP)
+
+		// start the MibDownload (assumed here to be done via some FSM again - open //TODO!!!)
+		/* the mib-download code may look something like that:
+		if err := dh.GetOnuDeviceEntry().MibDownloadFsm.Event("start"); err != nil {
+			logger.Errorw("MibDownloadFsm: Can't go to state starting", log.Fields{"err": err})
+			return errors.New("Can't go to state starting")
+		} else {
+			logger.Debug("MibDownloadFsm", log.Fields{"state": string(dh.GetOnuDeviceEntry().MibDownloadFsm.Current())})
+			//Determine ONU status and start/re-start MIB MibDownloadFsm
+			//Determine if this ONU has ever synchronized
+			if true { //TODO: insert valid check
+				if err := dh.GetOnuDeviceEntry().MibSyncFsm.Event("download_mib"); err != nil {
+					logger.Errorw("MibDownloadFsm: Can't go to state 'download_mib'", log.Fields{"err": err})
+					return errors.New("Can't go to state 'download_mib'")
+				} else {
+					//some further processing ???
+					logger.Debug("state of MibDownloadFsm", log.Fields{"state": string(dh.GetOnuDeviceEntry().MibDownloadFsm.Current())})
+					//some further processing ???
+				}
+			}
+		}
+		but by now we shortcut the download here and immediately fake the ONU-active state to get the state indication on ONUS!!!:
+		*/
+		//shortcut code to fake download-done!!!:
+		go dh.GetOnuDeviceEntry().transferSystemEvent(MibDownloadDone)
+	} else if dev_Event == MibDownloadDone {
+		logger.Debug("MibDownloadDone event: update dev state to 'Oper.Active'")
+		//initiate DevStateUpdate
+		if err := dh.coreProxy.DeviceStateUpdate(context.TODO(), dh.deviceID,
+			voltha.ConnectStatus_REACHABLE, voltha.OperStatus_ACTIVE); err != nil {
+			logger.Errorw("error-updating-device-state", log.Fields{"deviceID": dh.deviceID, "error": err})
+		}
+		logger.Debug("MibDownloadDone Event: update dev reasone to 'initial-mib-downloaded'")
+		if err := dh.coreProxy.DeviceReasonUpdate(context.TODO(), dh.deviceID, "initial-mib-downloaded"); err != nil {
+			logger.Errorw("error-DeviceReasonUpdate to 'initial-mib-downloaded'",
+				log.Fields{"deviceID": dh.deviceID, "error": err})
+		}
+
+		//TODO !!! following activities according to python code:
+		/*
+			yield self.enable_ports()
+			self._mib_download_task = None
+			yield self.onu_active_event()  -> with 'OnuActiveEvent' !!! might be this is required for ONOS visibility??
+		*/
+	} else {
+		logger.Warnw("unhandled-device-event", log.Fields{"event": dev_Event})
+	}
+}
+
+func (dh *DeviceHandler) addUniPort(a_uniInstNo uint16, a_uniId uint16, a_portType UniPortType) {
+	if _, present := dh.uniEntityMap[a_uniInstNo]; present {
+		logger.Warnw("onuUniPort-add: Port already exists", log.Fields{"for InstanceId": a_uniInstNo})
+	} else {
+		//TODO: need to find the ONU intfId and OnuId, using hard coded values for single ONU test!!!
+		// parameters are IntfId, OnuId, uniId
+		uni_no := MkUniPortNum(0, 1, uint32(a_uniId))
+		//with arguments a_uniId, a_portNo, a_portType
+		pUniPort := NewOnuUniPort(a_uniId, uni_no, a_uniInstNo, a_portType)
+		if pUniPort == nil {
+			logger.Warnw("onuUniPort-add: Could not create Port", log.Fields{"for InstanceId": a_uniInstNo})
+		} else {
+			dh.uniEntityMap[a_uniInstNo] = pUniPort
+			// create announce the UniPort to the core as VOLTHA Port object
+			if err := pUniPort.CreateVolthaPort(dh); err != nil {
+				logger.Infow("onuUniPort-added", log.Fields{"for InstanceId": a_uniInstNo})
+			} //error looging already within UniPort method
+		}
+	}
+}
diff --git a/adaptercoreont/messageTypes.go b/internal/pkg/onuadaptercore/messageTypes.go
similarity index 93%
rename from adaptercoreont/messageTypes.go
rename to internal/pkg/onuadaptercore/messageTypes.go
index e2261ed..05c5fff 100644
--- a/adaptercoreont/messageTypes.go
+++ b/internal/pkg/onuadaptercore/messageTypes.go
@@ -14,8 +14,8 @@
  * limitations under the License.
  */
 
-//Package adaptercoreont provides the utility for onu devices, flows and statistics
-package adaptercoreont
+//Package adaptercoreonu provides the utility for onu devices, flows and statistics
+package adaptercoreonu
 
 import (
 	gp "github.com/google/gopacket"
diff --git a/internal/pkg/onuadaptercore/mib_sync.go b/internal/pkg/onuadaptercore/mib_sync.go
new file mode 100644
index 0000000..45c771b
--- /dev/null
+++ b/internal/pkg/onuadaptercore/mib_sync.go
@@ -0,0 +1,216 @@
+/*
+ * 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 adaptercoreonu provides the utility for onu devices, flows and statistics
+package adaptercoreonu
+
+import (
+	"context"
+	"errors"
+
+	"github.com/looplab/fsm"
+
+	//"sync"
+	//"time"
+
+	//"github.com/opencord/voltha-lib-go/v3/pkg/kafka"
+	"github.com/opencord/omci-lib-go"
+	me "github.com/opencord/omci-lib-go/generated"
+	"github.com/opencord/voltha-lib-go/v3/pkg/log"
+	//ic "github.com/opencord/voltha-protos/v3/go/inter_container"
+	//"github.com/opencord/voltha-protos/v3/go/openflow_13"
+	//"github.com/opencord/voltha-protos/v3/go/voltha"
+)
+
+func (onuDeviceEntry *OnuDeviceEntry) logStateChange(e *fsm.Event) {
+	logger.Debugw("MibSync FSM", log.Fields{"event name": string(e.Event), "src state": string(e.Src), "dst state": string(e.Dst), "device-id": onuDeviceEntry.deviceID})
+}
+
+func (onuDeviceEntry *OnuDeviceEntry) enterStartingState(e *fsm.Event) {
+	logger.Debugw("MibSync FSM", log.Fields{"Start processing MibSync-msgs in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
+
+	// create channel and start go routine for processing of MibSync messages
+	onuDeviceEntry.MibSyncChan = make(chan Message, 2048)
+	go onuDeviceEntry.ProcessMibSyncMessages()
+}
+
+func (onuDeviceEntry *OnuDeviceEntry) enterLoadingMibTemplateState(e *fsm.Event) {
+	logger.Debugw("MibSync FSM", log.Fields{"Start MibTemplate processing in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
+	logger.Debug("function not implemented yet")
+}
+
+func (onuDeviceEntry *OnuDeviceEntry) enterUploadingState(e *fsm.Event) {
+	logger.Debugw("MibSync FSM", log.Fields{"send mibReset in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
+	onuDeviceEntry.PDevOmciCC.sendMibReset(context.TODO(), ConstDefaultOmciTimeout, true)
+}
+
+func (onuDeviceEntry *OnuDeviceEntry) enterInSyncState(e *fsm.Event) {
+	logger.Debugw("MibSync FSM", log.Fields{"send notification to core in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
+	onuDeviceEntry.transferSystemEvent(MibDatabaseSync)
+}
+
+func (onuDeviceEntry *OnuDeviceEntry) enterExaminingMdsState(e *fsm.Event) {
+	logger.Debugw("MibSync FSM", log.Fields{"Start GetMds processing in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
+	logger.Debug("function not implemented yet")
+}
+
+func (onuDeviceEntry *OnuDeviceEntry) enterResynchronizingState(e *fsm.Event) {
+	logger.Debugw("MibSync FSM", log.Fields{"Start MibResync processing in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
+	logger.Debug("function not implemented yet")
+}
+
+func (onuDeviceEntry *OnuDeviceEntry) enterAuditingState(e *fsm.Event) {
+	logger.Debugw("MibSync FSM", log.Fields{"Start MibResync processing in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
+	logger.Debug("function not implemented yet")
+}
+
+func (onuDeviceEntry *OnuDeviceEntry) enterOutOfSyncState(e *fsm.Event) {
+	logger.Debugw("MibSync FSM", log.Fields{"Start  MibReconcile processing in State": e.FSM.Current(), "device-id": onuDeviceEntry.deviceID})
+	logger.Debug("function not implemented yet")
+}
+
+func (onuDeviceEntry *OnuDeviceEntry) ProcessMibSyncMessages( /*ctx context.Context*/ ) {
+	logger.Debugw("MibSync Msg", log.Fields{"Start routine to process OMCI-messages for device-id": onuDeviceEntry.deviceID})
+loop:
+	for {
+		select {
+		// case <-ctx.Done():
+		// 	logger.Info("MibSync Msg", log.Fields{"Message handling canceled via context for device-id": onuDeviceEntry.deviceID})
+		// 	break loop
+		case message, ok := <-onuDeviceEntry.MibSyncChan:
+			if !ok {
+				logger.Info("MibSync Msg", log.Fields{"Message couldn't be read from channel for device-id": onuDeviceEntry.deviceID})
+				break loop
+			}
+			logger.Debugw("MibSync Msg", log.Fields{"Received message on ONU MibSyncChan for device-id": onuDeviceEntry.deviceID})
+
+			switch message.Type {
+			case TestMsg:
+				msg, _ := message.Data.(TestMessage)
+				onuDeviceEntry.handleTestMsg(msg)
+			case OMCI:
+				msg, _ := message.Data.(OmciMessage)
+				onuDeviceEntry.handleOmciMessage(msg)
+			default:
+				logger.Warn("MibSync Msg", log.Fields{"Unknown message type received for device-id": onuDeviceEntry.deviceID, "message.Type": message.Type})
+			}
+		}
+	}
+	logger.Info("MibSync Msg", log.Fields{"Stopped handling of MibSyncChan for device-id": onuDeviceEntry.deviceID})
+	// TODO: only this action?
+	onuDeviceEntry.MibSyncFsm.Event("stop")
+}
+
+func (onuDeviceEntry *OnuDeviceEntry) handleTestMsg(msg TestMessage) {
+
+	logger.Debugw("MibSync Msg", log.Fields{"TestMessage received for device-id": onuDeviceEntry.deviceID, "msg.TestMessageVal": msg.TestMessageVal})
+
+	switch msg.TestMessageVal {
+	case AnyTriggerForMibSyncUploadMib:
+		onuDeviceEntry.MibSyncFsm.Event("upload_mib")
+		logger.Debugw("MibSync Msg", log.Fields{"state": string(onuDeviceEntry.MibSyncFsm.Current())})
+	default:
+		logger.Warn("MibSync Msg", log.Fields{"Unknown message type received for device-id": onuDeviceEntry.deviceID, "msg.TestMessageVal": msg.TestMessageVal})
+	}
+}
+
+func (onuDeviceEntry *OnuDeviceEntry) handleOmciMessage(msg OmciMessage) {
+
+	logger.Debugw("MibSync Msg", log.Fields{"OmciMessage received for device-id": onuDeviceEntry.deviceID,
+		"msgType": msg.OmciMsg.MessageType})
+
+	//further analysis could be done here based on msg.OmciMsg.Payload, e.g. verification of error code ...
+	switch msg.OmciMsg.MessageType {
+	case omci.MibResetResponseType:
+		msgLayer := (*msg.OmciPacket).Layer(omci.LayerTypeMibResetResponse)
+		if msgLayer == nil {
+			logger.Error("Omci Msg layer could not be detected")
+			return
+		}
+		msgObj, msgOk := msgLayer.(*omci.MibResetResponse)
+		if !msgOk {
+			logger.Error("Omci Msg layer could not be assigned")
+			return
+		}
+		logger.Debugw("MibResetResponse Data", log.Fields{"data-fields": msgObj})
+		if msgObj.Result != me.Success {
+			logger.Errorw("Omci MibResetResponse Error - strange - what to do?", log.Fields{"Error": msgObj.Result})
+			return
+		}
+		onuDeviceEntry.PDevOmciCC.sendMibUpload(context.TODO(), ConstDefaultOmciTimeout, true)
+	case omci.MibUploadResponseType:
+		msgLayer := (*msg.OmciPacket).Layer(omci.LayerTypeMibUploadResponse)
+		if msgLayer == nil {
+			logger.Error("Omci Msg layer could not be detected")
+			return
+		}
+		msgObj, msgOk := msgLayer.(*omci.MibUploadResponse)
+		if !msgOk {
+			logger.Error("Omci Msg layer could not be assigned")
+			return
+		}
+		logger.Debugw("MibUploadResponse Data for:", log.Fields{"deviceId": onuDeviceEntry.PDevOmciCC.deviceID, "data-fields": msgObj})
+		/* to be verified / reworked !!! */
+		onuDeviceEntry.PDevOmciCC.uploadNoOfCmds = msgObj.NumberOfCommands
+		if onuDeviceEntry.PDevOmciCC.uploadSequNo < onuDeviceEntry.PDevOmciCC.uploadNoOfCmds {
+			onuDeviceEntry.PDevOmciCC.sendMibUploadNext(context.TODO(), ConstDefaultOmciTimeout, true)
+		} else {
+			logger.Error("Invalid number of commands received for:", log.Fields{"deviceId": onuDeviceEntry.PDevOmciCC.deviceID, "uploadNoOfCmds": onuDeviceEntry.PDevOmciCC.uploadNoOfCmds})
+			//TODO right action?
+			onuDeviceEntry.MibSyncFsm.Event("timeout")
+		}
+	case omci.MibUploadNextResponseType:
+		msgLayer := (*msg.OmciPacket).Layer(omci.LayerTypeMibUploadNextResponse)
+		if msgLayer == nil {
+			logger.Error("Omci Msg layer could not be detected")
+			return
+		}
+		msgObj, msgOk := msgLayer.(*omci.MibUploadNextResponse)
+		if !msgOk {
+			logger.Error("Omci Msg layer could not be assigned")
+			return
+		}
+		logger.Debugw("MibUploadNextResponse Data for:", log.Fields{"deviceId": onuDeviceEntry.PDevOmciCC.deviceID, "data-fields": msgObj})
+		// TODO !!! content evaluation ?????
+		if onuDeviceEntry.PDevOmciCC.uploadSequNo < onuDeviceEntry.PDevOmciCC.uploadNoOfCmds {
+			onuDeviceEntry.PDevOmciCC.sendMibUploadNext(context.TODO(), ConstDefaultOmciTimeout, true)
+		} else {
+			//TODO
+			onuDeviceEntry.MibSyncFsm.Event("success")
+		}
+	}
+}
+
+func (onuDeviceEntry *OnuDeviceEntry) MibDbVolatileDict() error {
+	logger.Debug("MibVolatileDict- running from default Entry code")
+	return errors.New("not_implemented")
+}
+
+// func (onuDeviceEntry *OnuDeviceEntry) MibTemplateTask() error {
+// 	return errors.New("not_implemented")
+// }
+// func (onuDeviceEntry *OnuDeviceEntry) MibUploadTask() error {
+// 	return errors.New("not_implemented")
+// }
+// func (onuDeviceEntry *OnuDeviceEntry) GetMdsTask() error {
+// 	return errors.New("not_implemented")
+// }
+// func (onuDeviceEntry *OnuDeviceEntry) MibResyncTask() error {
+// 	return errors.New("not_implemented")
+// }
+// func (onuDeviceEntry *OnuDeviceEntry) MibReconcileTask() error {
+// 	return errors.New("not_implemented")
+// }
diff --git a/adaptercoreont/omci_cc.go b/internal/pkg/onuadaptercore/omci_cc.go
similarity index 82%
rename from adaptercoreont/omci_cc.go
rename to internal/pkg/onuadaptercore/omci_cc.go
index 17e7e40..0d40873 100644
--- a/adaptercoreont/omci_cc.go
+++ b/internal/pkg/onuadaptercore/omci_cc.go
@@ -14,8 +14,8 @@
  * limitations under the License.
  */
 
-//Package adaptercoreont provides the utility for onu devices, flows and statistics
-package adaptercoreont
+//Package adaptercoreonu provides the utility for onu devices, flows and statistics
+package adaptercoreonu
 
 import (
 	"container/list"
@@ -91,7 +91,7 @@
 func NewOmciCC(ctx context.Context, onu_device_entry *OnuDeviceEntry,
 	device_id string, device_handler *DeviceHandler,
 	core_proxy adapterif.CoreProxy, adapter_proxy adapterif.AdapterProxy) *OmciCC {
-	log.Infow("init-omciCC", log.Fields{"deviceId": device_id})
+	logger.Infow("init-omciCC", log.Fields{"deviceId": device_id})
 	var omciCC OmciCC
 	omciCC.enabled = false
 	omciCC.pOnuDeviceEntry = onu_device_entry
@@ -118,7 +118,7 @@
 
 // Rx handler for omci messages
 func (oo *OmciCC) ReceiveOnuMessage(ctx context.Context, omciMsg *omci.OMCI) error {
-	log.Debugw("rx-onu-autonomous-message", log.Fields{"omciMsgType": omciMsg.MessageType,
+	logger.Debugw("rx-onu-autonomous-message", log.Fields{"omciMsgType": omciMsg.MessageType,
 		"payload": hex.EncodeToString(omciMsg.Payload)})
 	/*
 			msgType = rxFrame.fields["message_type"] //assumed OmciOperationsValue
@@ -127,7 +127,7 @@
 			switch msgType {
 			case AlarmNotification:
 				{
-					log.Info("Unhandled: received-onu-alarm-message")
+					logger.Info("Unhandled: received-onu-alarm-message")
 					// python code was:
 					//if msg_type == EntityOperations.AlarmNotification.value:
 					//	topic = OMCI_CC.event_bus_topic(self._device_id, RxEvent.Alarm_Notification)
@@ -137,7 +137,7 @@
 				}
 			case AttributeValueChange:
 				{
-					log.Info("Unhandled: received-attribute-value-change")
+					logger.Info("Unhandled: received-attribute-value-change")
 					// python code was:
 					//elif msg_type == EntityOperations.AttributeValueChange.value:
 					//	topic = OMCI_CC.event_bus_topic(self._device_id, RxEvent.AVC_Notification)
@@ -147,7 +147,7 @@
 				}
 			case TestResult:
 				{
-					log.Info("Unhandled: received-test-result")
+					logger.Info("Unhandled: received-test-result")
 					// python code was:
 					//elif msg_type == EntityOperations.TestResult.value:
 					//	topic = OMCI_CC.event_bus_topic(self._device_id, RxEvent.Test_Result)
@@ -157,7 +157,7 @@
 				}
 			default:
 				{
-					log.Errorw("rx-onu-unsupported-autonomous-message", log.Fields{"msgType": msgType})
+					logger.Errorw("rx-onu-unsupported-autonomous-message", log.Fields{"msgType": msgType})
 					rxOnuDiscards++
 					return errors.New("RxOnuMsgType unimplemented")
 				}
@@ -169,51 +169,51 @@
 // Rx handler for onu messages
 //    e.g. would call ReceiveOnuMessage() in case of TID=0 or Action=test ...
 func (oo *OmciCC) ReceiveMessage(ctx context.Context, rxMsg []byte) error {
-	//log.Debugw("cc-receive-omci-message", log.Fields{"RxOmciMessage-x2s": hex.EncodeToString(rxMsg)})
+	//logger.Debugw("cc-receive-omci-message", log.Fields{"RxOmciMessage-x2s": hex.EncodeToString(rxMsg)})
 	if len(rxMsg) >= 44 { // then it should normally include the BaseFormat trailer Len
 		// NOTE: autocorrection only valid for OmciBaseFormat, which is not specifically verified here!!!
 		//  (am extendedFormat message could be destroyed this way!)
 		trailerLenData := rxMsg[42:44]
 		trailerLen := binary.BigEndian.Uint16(trailerLenData)
-		log.Infow("omci-received-trailer-len", log.Fields{"Length": trailerLen})
+		logger.Infow("omci-received-trailer-len", log.Fields{"Length": trailerLen})
 		if trailerLen != 40 { // invalid base Format entry -> autocorrect
 			binary.BigEndian.PutUint16(rxMsg[42:44], 40)
-			log.Debug("cc-corrected-omci-message: trailer len inserted")
+			logger.Debug("cc-corrected-omci-message: trailer len inserted")
 		}
 	} else {
-		log.Errorw("received omci-message to small for OmciBaseFormat - abort", log.Fields{"Length": len(rxMsg)})
+		logger.Errorw("received omci-message to small for OmciBaseFormat - abort", log.Fields{"Length": len(rxMsg)})
 		return errors.New("RxOmciMessage to small for BaseFormat")
 	}
 
 	packet := gopacket.NewPacket(rxMsg, omci.LayerTypeOMCI, gopacket.NoCopy)
 	if packet == nil {
-		log.Error("omci-message could not be decoded")
+		logger.Error("omci-message could not be decoded")
 		return errors.New("could not decode rxMsg as OMCI")
 	}
 	omciLayer := packet.Layer(omci.LayerTypeOMCI)
 	if omciLayer == nil {
-		log.Error("omci-message could not decode omci layer")
+		logger.Error("omci-message could not decode omci layer")
 		return errors.New("could not decode omci layer")
 	}
 	omciMsg, ok := omciLayer.(*omci.OMCI)
 	if !ok {
-		log.Error("omci-message could not assign omci layer")
+		logger.Error("omci-message could not assign omci layer")
 		return errors.New("could not assign omci layer")
 	}
-	log.Debugw("omci-message-decoded:", log.Fields{"omciMsgType": omciMsg.MessageType,
+	logger.Debugw("omci-message-decoded:", log.Fields{"omciMsgType": omciMsg.MessageType,
 		"transCorrId": omciMsg.TransactionID, "DeviceIdent": omciMsg.DeviceIdentifier})
 	if byte(omciMsg.MessageType) & ^me.AK == 0 {
 		// Not a response
-		log.Debug("RxMsg is no Omci Response Message")
+		logger.Debug("RxMsg is no Omci Response Message")
 		if omciMsg.TransactionID == 0 {
 			return oo.ReceiveOnuMessage(ctx, omciMsg)
 		} else {
-			log.Errorw("Unexpected TransCorrId != 0  not accepted for autonomous messages",
+			logger.Errorw("Unexpected TransCorrId != 0  not accepted for autonomous messages",
 				log.Fields{"msgType": omciMsg.MessageType, "payload": hex.EncodeToString(omciMsg.Payload)})
 			return errors.New("Autonomous Omci Message with TranSCorrId != 0 not acccepted")
 		}
 	} else {
-		log.Debug("RxMsg is a Omci Response Message: try to schedule it to the requester")
+		logger.Debug("RxMsg is a Omci Response Message: try to schedule it to the requester")
 		oo.mutexRxSchedMap.Lock()
 		rxCallback, ok := oo.rxSchedulerMap[omciMsg.TransactionID]
 		if ok && rxCallback != nil {
@@ -225,7 +225,7 @@
 			oo.mutexRxSchedMap.Unlock()
 		} else {
 			oo.mutexRxSchedMap.Unlock()
-			log.Error("omci-message-response for not registered transCorrId")
+			logger.Error("omci-message-response for not registered transCorrId")
 			return errors.New("could not find registered response handler tor transCorrId")
 		}
 	}
@@ -253,15 +253,15 @@
 
 	               try:
 	                   rx_frame = msg if isinstance(msg, OmciFrame) else OmciFrame(msg)
-	                   self.log.debug('recv-omci-msg', omci_msg=hexlify(msg))
+	                   self.logger.debug('recv-omci-msg', omci_msg=hexlify(msg))
 	               except KeyError as e:
 	                   # Unknown, Unsupported, or vendor-specific ME. Key is the unknown classID
-	                   self.log.debug('frame-decode-key-error', omci_msg=hexlify(msg), e=e)
+	                   self.logger.debug('frame-decode-key-error', omci_msg=hexlify(msg), e=e)
 	                   rx_frame = self._decode_unknown_me(msg)
 	                   self._rx_unknown_me += 1
 
 	               except Exception as e:
-	                   self.log.exception('frame-decode', omci_msg=hexlify(msg), e=e)
+	                   self.logger.exception('frame-decode', omci_msg=hexlify(msg), e=e)
 	                   return
 
 	               finally:
@@ -269,17 +269,17 @@
 
 	               rx_tid = rx_frame.fields['transaction_id']
 	               msg_type = rx_frame.fields['message_type']
-	               self.log.debug('Received message for rx_tid', rx_tid = rx_tid, msg_type = msg_type)
+	               self.logger.debug('Received message for rx_tid', rx_tid = rx_tid, msg_type = msg_type)
 	               # Filter the Test Result frame and route through receive onu
 	               # message method.
 	               if rx_tid == 0 or msg_type == EntityOperations.TestResult.value:
-	                   self.log.debug('Receive ONU message', rx_tid=0)
+	                   self.logger.debug('Receive ONU message', rx_tid=0)
 	                   return self._receive_onu_message(rx_frame)
 
 	               # Previously unreachable if this is the very first round-trip Rx or we
 	               # have been running consecutive errors
 	               if self._rx_frames == 0 or self._consecutive_errors != 0:
-	                   self.log.debug('Consecutive errors for rx', err = self._consecutive_errors)
+	                   self.logger.debug('Consecutive errors for rx', err = self._consecutive_errors)
 	                   self.reactor.callLater(0, self._publish_connectivity_event, True)
 
 	               self._rx_frames += 1
@@ -296,7 +296,7 @@
 	                           last_tx_tuple[OMCI_CC.REQUEST_FRAME].fields.get('transaction_id') != rx_tid:
 	                       # Possible late Rx on a message that timed-out
 	                       if last_tx_tuple:
-	                           self.log.debug('Unknown message', rx_tid=rx_tid,
+	                           self.logger.debug('Unknown message', rx_tid=rx_tid,
 	                                          tx_id=last_tx_tuple[OMCI_CC.REQUEST_FRAME].fields.get('transaction_id'))
 	                       self._rx_unknown_tid += 1
 	                       self._rx_late += 1
@@ -311,17 +311,17 @@
 	                   # Late arrival already serviced by a timeout?
 	                   if d.called:
 	                       self._rx_late += 1
-	                       self.log.debug('Serviced by timeout. Late arrival', rx_late = self._rx_late)
+	                       self.logger.debug('Serviced by timeout. Late arrival', rx_late = self._rx_late)
 	                       return
 
 	               except Exception as e:
-	                   self.log.exception('frame-match', msg=hexlify(msg), e=e)
+	                   self.logger.exception('frame-match', msg=hexlify(msg), e=e)
 	                   if d is not None:
 	                       return d.errback(failure.Failure(e))
 	                   return
 
 	               # Publish Rx event to listeners in a different task
-	               self.log.debug('Publish rx event', rx_tid = rx_tid,
+	               self.logger.debug('Publish rx event', rx_tid = rx_tid,
 	                              tx_tid = tx_frame.fields['transaction_id'])
 	               reactor.callLater(0, self._publish_rx_frame, tx_frame, rx_frame)
 
@@ -330,7 +330,7 @@
 	               d.callback(rx_frame)
 
 	           except Exception as e:
-	   			self.log.exception('rx-msg', e=e)
+	   			self.logger.exception('rx-msg', e=e)
 	*/
 }
 
@@ -345,7 +345,7 @@
 func (oo *OmciCC) Send(ctx context.Context, txFrame []byte, timeout int, retry int, highPrio bool,
 	receiveCallbackPair CallbackPair) error {
 
-	log.Debugw("register-response-callback:", log.Fields{"for TansCorrId": receiveCallbackPair.cbKey})
+	logger.Debugw("register-response-callback:", log.Fields{"for TansCorrId": receiveCallbackPair.cbKey})
 	// it could be checked, if the callback keay is already registered - but simply overwrite may be acceptable ...
 	oo.mutexRxSchedMap.Lock()
 	oo.rxSchedulerMap[receiveCallbackPair.cbKey] = receiveCallbackPair.cbFunction
@@ -381,7 +381,7 @@
 		omciTxRequest := queueElement.Value.(omciTransferStructure)
 		/* compare olt device handler code:
 		func (dh *DeviceHandler) omciIndication(omciInd *oop.OmciIndication) {
-			log.Debugw("omci indication", log.Fields{"intfID": omciInd.IntfId, "onuID": omciInd.OnuId})
+			logger.Debugw("omci indication", log.Fields{"intfID": omciInd.IntfId, "onuID": omciInd.OnuId})
 			var deviceType string
 			var deviceID string
 			var proxyDeviceID string
@@ -390,7 +390,7 @@
 
 			if onuInCache, ok := dh.onus.Load(onuKey); !ok {
 
-				log.Debugw("omci indication for a device not in cache.", log.Fields{"intfID": omciInd.IntfId, "onuID": omciInd.OnuId})
+				logger.Debugw("omci indication for a device not in cache.", log.Fields{"intfID": omciInd.IntfId, "onuID": omciInd.OnuId})
 				ponPort := IntfIDToPortNo(omciInd.GetIntfId(), voltha.Port_PON_OLT)
 				kwargs := make(map[string]interface{})
 				kwargs["onu_id"] = omciInd.OnuId
@@ -398,7 +398,7 @@
 
 				onuDevice, err := dh.coreProxy.GetChildDevice(context.TODO(), dh.device.Id, kwargs)
 				if err != nil {
-					log.Errorw("onu not found", log.Fields{"intfID": omciInd.IntfId, "onuID": omciInd.OnuId, "error": err})
+					logger.Errorw("onu not found", log.Fields{"intfID": omciInd.IntfId, "onuID": omciInd.OnuId, "error": err})
 					return
 				}
 				deviceType = onuDevice.Type
@@ -408,7 +408,7 @@
 				dh.onus.Store(onuKey, NewOnuDevice(deviceID, deviceType, onuDevice.SerialNumber, omciInd.OnuId, omciInd.IntfId, proxyDeviceID))
 			} else {
 				//found in cache
-				log.Debugw("omci indication for a device in cache.", log.Fields{"intfID": omciInd.IntfId, "onuID": omciInd.OnuId})
+				logger.Debugw("omci indication for a device in cache.", log.Fields{"intfID": omciInd.IntfId, "onuID": omciInd.OnuId})
 				deviceType = onuInCache.(*OnuDevice).deviceType
 				deviceID = onuInCache.(*OnuDevice).deviceID
 				proxyDeviceID = onuInCache.(*OnuDevice).proxyDeviceID
@@ -420,7 +420,7 @@
 			proxy_address=self._proxy_address,
 			connect_status=self._device.connect_status)
 
-		self.log.debug('sent-omci-msg', tid=tx_tid, omci_msg=hexlify(bytes(frame)))
+		self.logger.debug('sent-omci-msg', tid=tx_tid, omci_msg=hexlify(bytes(frame)))
 
 		yield self._adapter_proxy.send_inter_adapter_message(
 			msg=omci_msg,
@@ -435,15 +435,15 @@
 			oo.pBaseDeviceHandler.deviceID, oo.deviceID) //parent, child
 		if err != nil || device == nil {
 			/*TODO: needs to handle error scenarios */
-			log.Errorw("Failed to fetch device", log.Fields{"err": err, "ParentId": oo.pBaseDeviceHandler.deviceID,
+			logger.Errorw("Failed to fetch device", log.Fields{"err": err, "ParentId": oo.pBaseDeviceHandler.deviceID,
 				"ChildId": oo.deviceID})
 			return errors.New("failed to fetch device")
 		}
 
-		log.Debugw("omci-message-sending", log.Fields{"fromDeviceType": oo.pBaseDeviceHandler.DeviceType,
+		logger.Debugw("omci-message-sending", log.Fields{"fromDeviceType": oo.pBaseDeviceHandler.DeviceType,
 			"toDeviceType": oo.pBaseDeviceHandler.ProxyAddressType,
 			"onuDeviceID":  oo.deviceID, "proxyDeviceID": oo.pBaseDeviceHandler.ProxyAddressID})
-		log.Debugw("omci-message-to-send:",
+		logger.Debugw("omci-message-to-send:",
 			log.Fields{"TxOmciMessage": hex.EncodeToString(omciTxRequest.txFrame)})
 
 		omciMsg := &ic.InterAdapterOmciMessage{Message: omciTxRequest.txFrame}
@@ -452,7 +452,7 @@
 			//fromType,toType,toDevId, ProxyDevId
 			oo.pBaseDeviceHandler.DeviceType, oo.pBaseDeviceHandler.ProxyAddressType,
 			oo.deviceID, oo.pBaseDeviceHandler.ProxyAddressID, ""); sendErr != nil {
-			log.Errorw("send omci request error", log.Fields{"error": sendErr})
+			logger.Errorw("send omci request error", log.Fields{"error": sendErr})
 			return sendErr
 		}
 		oo.txQueue.Remove(queueElement) // Dequeue
@@ -513,11 +513,11 @@
 //supply a response handler for the MibSync omci response messages
 func (oo *OmciCC) receiveMibSyncResponse(omciMsg *omci.OMCI, packet *gp.Packet) error {
 
-	log.Debugw("mib-sync-omci-message-response received:", log.Fields{"omciMsgType": omciMsg.MessageType,
+	logger.Debugw("mib-sync-omci-message-response received:", log.Fields{"omciMsgType": omciMsg.MessageType,
 		"transCorrId": omciMsg.TransactionID, "deviceId": oo.deviceID})
 
 	if oo.pOnuDeviceEntry == nil {
-		log.Error("Abort Receive MibSync OMCI, DeviceEntryPointer is nil")
+		logger.Error("Abort Receive MibSync OMCI, DeviceEntryPointer is nil")
 		return errors.New("DeviceEntryPointer is nil")
 	}
 
@@ -530,7 +530,7 @@
 			OmciPacket: packet,
 		},
 	}
-	//log.Debugw("Message to be sent into channel:", log.Fields{"mibSyncMsg": mibSyncMsg})
+	//logger.Debugw("Message to be sent into channel:", log.Fields{"mibSyncMsg": mibSyncMsg})
 	(*oo.pOnuDeviceEntry).MibSyncChan <- mibSyncMsg
 
 	return nil
@@ -538,7 +538,7 @@
 
 func (oo *OmciCC) sendMibReset(ctx context.Context, timeout int, highPrio bool) error {
 
-	log.Debugw("send MibReset-msg to:", log.Fields{"deviceId": oo.deviceID})
+	logger.Debugw("send MibReset-msg to:", log.Fields{"deviceId": oo.deviceID})
 	request := &omci.MibResetRequest{
 		MeBasePacket: omci.MeBasePacket{
 			EntityClass: me.OnuDataClassID,
@@ -547,7 +547,7 @@
 	tid := oo.GetNextTid(highPrio)
 	pkt, err := serialize(omci.MibResetRequestType, request, tid)
 	if err != nil {
-		log.Errorw("Cannot serialize MibResetRequest", log.Fields{"Err": err})
+		logger.Errorw("Cannot serialize MibResetRequest", log.Fields{"Err": err})
 		return err
 	}
 	omciRxCallbackPair := CallbackPair{tid, oo.receiveMibSyncResponse}
@@ -556,7 +556,7 @@
 
 func (oo *OmciCC) sendMibUpload(ctx context.Context, timeout int, highPrio bool) error {
 
-	log.Debugw("send MibUpload-msg to:", log.Fields{"deviceId": oo.deviceID})
+	logger.Debugw("send MibUpload-msg to:", log.Fields{"deviceId": oo.deviceID})
 	request := &omci.MibUploadRequest{
 		MeBasePacket: omci.MeBasePacket{
 			EntityClass: me.OnuDataClassID,
@@ -565,7 +565,7 @@
 	tid := oo.GetNextTid(highPrio)
 	pkt, err := serialize(omci.MibUploadRequestType, request, tid)
 	if err != nil {
-		log.Errorw("Cannot serialize MibUploadRequest", log.Fields{"Err": err})
+		logger.Errorw("Cannot serialize MibUploadRequest", log.Fields{"Err": err})
 		return err
 	}
 	oo.uploadSequNo = 0
@@ -577,7 +577,7 @@
 
 func (oo *OmciCC) sendMibUploadNext(ctx context.Context, timeout int, highPrio bool) error {
 
-	log.Debugw("send MibUploadNext-msg to:", log.Fields{"deviceId": oo.deviceID, "uploadSequNo": oo.uploadSequNo})
+	logger.Debugw("send MibUploadNext-msg to:", log.Fields{"deviceId": oo.deviceID, "uploadSequNo": oo.uploadSequNo})
 	request := &omci.MibUploadNextRequest{
 		MeBasePacket: omci.MeBasePacket{
 			EntityClass: me.OnuDataClassID,
@@ -587,7 +587,7 @@
 	tid := oo.GetNextTid(highPrio)
 	pkt, err := serialize(omci.MibUploadNextRequestType, request, tid)
 	if err != nil {
-		log.Errorw("Cannot serialize MibUploadNextRequest", log.Fields{"Err": err})
+		logger.Errorw("Cannot serialize MibUploadNextRequest", log.Fields{"Err": err})
 		return err
 	}
 	oo.uploadSequNo++
diff --git a/adaptercoreont/omci_test_request.go b/internal/pkg/onuadaptercore/omci_test_request.go
similarity index 80%
rename from adaptercoreont/omci_test_request.go
rename to internal/pkg/onuadaptercore/omci_test_request.go
index a4c6eda..a79ad21 100644
--- a/adaptercoreont/omci_test_request.go
+++ b/internal/pkg/onuadaptercore/omci_test_request.go
@@ -14,8 +14,8 @@
  * limitations under the License.
  */
 
-//Package adaptercoreont provides the utility for onu devices, flows and statistics
-package adaptercoreont
+//Package adaptercoreonu provides the utility for onu devices, flows and statistics
+package adaptercoreonu
 
 import (
 	"context"
@@ -51,7 +51,7 @@
 func NewOmciTestRequest(ctx context.Context,
 	device_id string, omci_cc *OmciCC,
 	exclusive bool, allow_failure bool) *OmciTestRequest {
-	log.Debug("omciTestRequest-init")
+	logger.Debug("omciTestRequest-init")
 	var omciTestRequest OmciTestRequest
 	omciTestRequest.deviceID = device_id
 	omciTestRequest.pDevOmciCC = omci_cc
@@ -65,7 +65,7 @@
 
 //
 func (oo *OmciTestRequest) PerformOmciTest(ctx context.Context, exec_Channel chan<- bool) {
-	log.Debug("omciTestRequest-start-test")
+	logger.Debug("omciTestRequest-start-test")
 
 	if oo.pDevOmciCC != nil {
 		oo.verifyDone = exec_Channel
@@ -75,12 +75,12 @@
 		onu2gBaseGet, _ := oo.CreateOnu2gBaseGet(tid)
 		omciRxCallbackPair := CallbackPair{tid, oo.ReceiveOmciVerifyResponse}
 
-		log.Debugw("performOmciTest-start sending frame", log.Fields{"for deviceId": oo.deviceID})
+		logger.Debugw("performOmciTest-start sending frame", log.Fields{"for deviceId": oo.deviceID})
 		// send with default timeout and normal prio
 		go oo.pDevOmciCC.Send(ctx, onu2gBaseGet, ConstDefaultOmciTimeout, 0, false, omciRxCallbackPair)
 
 	} else {
-		log.Errorw("performOmciTest: Device does not exist", log.Fields{"for deviceId": oo.deviceID})
+		logger.Errorw("performOmciTest: Device does not exist", log.Fields{"for deviceId": oo.deviceID})
 	}
 }
 
@@ -101,7 +101,7 @@
 	pkt, err := serialize(omci.GetRequestType, request, tid)
 	if err != nil {
 		//omciLogger.WithFields(log.Fields{ ...
-		log.Errorw("Cannot serialize Onu2-G GetRequest", log.Fields{"Err": err})
+		logger.Errorw("Cannot serialize Onu2-G GetRequest", log.Fields{"Err": err})
 		return nil, err
 	}
 	// hexEncode would probably work as well, but not needed and leads to wrong logs on OltAdapter frame
@@ -112,21 +112,21 @@
 //supply a response handler
 func (oo *OmciTestRequest) ReceiveOmciVerifyResponse(omciMsg *omci.OMCI, packet *gp.Packet) error {
 
-	log.Debugw("verify-omci-message-response received:", log.Fields{"omciMsgType": omciMsg.MessageType,
+	logger.Debugw("verify-omci-message-response received:", log.Fields{"omciMsgType": omciMsg.MessageType,
 		"transCorrId": omciMsg.TransactionID, "DeviceIdent": omciMsg.DeviceIdentifier})
 
 	if omciMsg.TransactionID == oo.txSeqNo {
-		log.Debugw("verify-omci-message-response", log.Fields{"correct TransCorrId": omciMsg.TransactionID})
+		logger.Debugw("verify-omci-message-response", log.Fields{"correct TransCorrId": omciMsg.TransactionID})
 	} else {
-		log.Debugw("verify-omci-message-response error", log.Fields{"incorrect TransCorrId": omciMsg.TransactionID,
+		logger.Debugw("verify-omci-message-response error", log.Fields{"incorrect TransCorrId": omciMsg.TransactionID,
 			"expected": oo.txSeqNo})
 		oo.verifyDone <- false
 		return errors.New("Unexpected TransCorrId")
 	}
 	if omciMsg.MessageType == omci.GetResponseType {
-		log.Debugw("verify-omci-message-response", log.Fields{"correct RespType": omciMsg.MessageType})
+		logger.Debugw("verify-omci-message-response", log.Fields{"correct RespType": omciMsg.MessageType})
 	} else {
-		log.Debugw("verify-omci-message-response error", log.Fields{"incorrect RespType": omciMsg.MessageType,
+		logger.Debugw("verify-omci-message-response error", log.Fields{"incorrect RespType": omciMsg.MessageType,
 			"expected": omci.GetResponseType})
 		oo.verifyDone <- false
 		return errors.New("Unexpected MessageType")
diff --git a/adaptercoreont/ont_device_entry.go b/internal/pkg/onuadaptercore/onu_device_entry.go
similarity index 83%
rename from adaptercoreont/ont_device_entry.go
rename to internal/pkg/onuadaptercore/onu_device_entry.go
index 27b71e2..837dadb 100644
--- a/adaptercoreont/ont_device_entry.go
+++ b/internal/pkg/onuadaptercore/onu_device_entry.go
@@ -14,8 +14,8 @@
  * limitations under the License.
  */
 
-//Package adaptercoreont provides the utility for onu devices, flows and statistics
-package adaptercoreont
+//Package adaptercoreonu provides the utility for onu devices, flows and statistics
+package adaptercoreonu
 
 import (
 	"context"
@@ -40,15 +40,16 @@
 	DeviceStatusInit     OnuDeviceEvent = 0 // OnuDeviceEntry default start state
 	MibDatabaseSync      OnuDeviceEvent = 1 // MIB database sync (upload done)
 	OmciCapabilitiesDone OnuDeviceEvent = 2 // OMCI ME and message type capabilities known
-	PortLinkUp           OnuDeviceEvent = 3 // Port link state change
-	PortLinkDw           OnuDeviceEvent = 4 // Port link state change
+	MibDownloadDone      OnuDeviceEvent = 3 // MIB database sync (upload done)
+	PortLinkUp           OnuDeviceEvent = 4 // Port link state change
+	PortLinkDw           OnuDeviceEvent = 5 // Port link state change
 	// Add other events here as needed (alarms separate???)
 )
 
 type activityDescr struct {
 	databaseClass   func() error
 	advertiseEvents bool
-	auditDelay      int
+	auditDelay      uint16
 	//tasks           map[string]func() error
 }
 type OmciDeviceFsms map[string]activityDescr
@@ -67,6 +68,7 @@
 	MibSyncFsm    *fsm.FSM
 	MibSyncChan   chan Message
 	devState      OnuDeviceEvent
+	mibAuditDelay uint16
 }
 
 //OnuDeviceEntry returns a new instance of a OnuDeviceEntry
@@ -74,8 +76,8 @@
 func NewOnuDeviceEntry(ctx context.Context,
 	device_id string, device_Handler *DeviceHandler,
 	core_proxy adapterif.CoreProxy, adapter_proxy adapterif.AdapterProxy,
-	mib_db func() error, supported_Fsms_Ptr *OmciDeviceFsms) *OnuDeviceEntry {
-	log.Infow("init-onuDeviceEntry", log.Fields{"deviceId": device_id})
+	supported_Fsms_Ptr *OmciDeviceFsms) *OnuDeviceEntry {
+	logger.Infow("init-onuDeviceEntry", log.Fields{"deviceId": device_id})
 	var onuDeviceEntry OnuDeviceEntry
 	onuDeviceEntry.started = false
 	onuDeviceEntry.deviceID = device_id
@@ -92,6 +94,7 @@
 		onuDeviceEntry.supportedFsms = *supported_Fsms_Ptr
 	} else {
 		//var mibSyncFsm = NewMibSynchronizer()
+		// use some internaö defaults, if not defined from outside
 		onuDeviceEntry.supportedFsms = OmciDeviceFsms{
 			"mib-synchronizer": {
 				//mibSyncFsm,        // Implements the MIB synchronization state machine
@@ -110,8 +113,10 @@
 		}
 	}
 	onuDeviceEntry.mibDbClass = onuDeviceEntry.supportedFsms["mib-synchronizer"].databaseClass
-	log.Debug("access2mibDbClass")
+	logger.Debug("access2mibDbClass")
 	go onuDeviceEntry.mibDbClass()
+	onuDeviceEntry.mibAuditDelay = onuDeviceEntry.supportedFsms["mib-synchronizer"].auditDelay
+	logger.Debugw("MibAudit is set to", log.Fields{"Delay": onuDeviceEntry.mibAuditDelay})
 
 	// Omci related Mib sync state machine
 	onuDeviceEntry.MibSyncFsm = fsm.NewFSM(
@@ -168,7 +173,7 @@
 
 //Start starts (logs) the omci agent
 func (oo *OnuDeviceEntry) Start(ctx context.Context) error {
-	log.Info("starting-OnuDeviceEntry")
+	logger.Info("starting-OnuDeviceEntry")
 
 	oo.PDevOmciCC = NewOmciCC(ctx, oo, oo.deviceID, oo.baseDeviceHandler,
 		oo.coreProxy, oo.adapterProxy)
@@ -176,22 +181,22 @@
 	//TODO .....
 	//mib_db.start()
 	oo.started = true
-	log.Info("OnuDeviceEntry-started, but not yet mib_db!!!")
+	logger.Info("OnuDeviceEntry-started, but not yet mib_db!!!")
 	return nil
 }
 
 //Stop terminates the session
 func (oo *OnuDeviceEntry) Stop(ctx context.Context) error {
-	log.Info("stopping-OnuDeviceEntry")
+	logger.Info("stopping-OnuDeviceEntry")
 	oo.started = false
 	//oo.exitChannel <- 1
-	log.Info("OnuDeviceEntry-stopped")
+	logger.Info("OnuDeviceEntry-stopped")
 	return nil
 }
 
 //Relay the InSync message via Handler to Rw core - Status update
 func (oo *OnuDeviceEntry) transferSystemEvent(dev_Event OnuDeviceEvent) error {
-	log.Debugw("relaying system-event", log.Fields{"Event": dev_Event})
+	logger.Debugw("relaying system-event", log.Fields{"Event": dev_Event})
 	// decouple the handler transfer from further processing here
 	// TODO!!! check if really no synch is required within the system e.g. to ensure following steps ..
 	if dev_Event == MibDatabaseSync {
@@ -200,10 +205,17 @@
 			go oo.baseDeviceHandler.DeviceStateUpdate(dev_Event)
 			//TODO!!! device control: next step: start MIB capability verification from here ?!!!
 		} else {
-			log.Debugw("mib-in-sync-event in some already synced state - ignored", log.Fields{"state": oo.devState})
+			logger.Debugw("mibinsync-event in some already synced state - ignored", log.Fields{"state": oo.devState})
+		}
+	} else if dev_Event == MibDownloadDone {
+		if oo.devState < MibDownloadDone { //devState has not been synced yet
+			oo.devState = MibDownloadDone
+			go oo.baseDeviceHandler.DeviceStateUpdate(dev_Event)
+		} else {
+			logger.Debugw("mibdownloaddone-event was already seen - ignored", log.Fields{"state": oo.devState})
 		}
 	} else {
-		log.Warnw("device-event not yet handled", log.Fields{"state": dev_Event})
+		logger.Warnw("device-event not yet handled", log.Fields{"state": dev_Event})
 	}
 	return nil
 }
diff --git a/internal/pkg/onuadaptercore/onu_uni_port.go b/internal/pkg/onuadaptercore/onu_uni_port.go
new file mode 100644
index 0000000..adf2b6c
--- /dev/null
+++ b/internal/pkg/onuadaptercore/onu_uni_port.go
@@ -0,0 +1,107 @@
+/*
+ * 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 adaptercoreonu provides the utility for onu devices, flows and statistics
+package adaptercoreonu
+
+import (
+	"context"
+	"errors"
+	"strconv"
+
+	//"sync"
+	//"time"
+
+	//"github.com/opencord/voltha-lib-go/v3/pkg/kafka"
+	"github.com/opencord/voltha-lib-go/v3/pkg/log"
+	vc "github.com/opencord/voltha-protos/v3/go/common"
+	"github.com/opencord/voltha-protos/v3/go/voltha"
+	//ic "github.com/opencord/voltha-protos/v3/go/inter_container"
+	//"github.com/opencord/voltha-protos/v3/go/openflow_13"
+	//"github.com/opencord/voltha-protos/v3/go/voltha"
+)
+
+type UniPortType uint8
+
+const (
+	// Uni Interface type
+	UniPPTP UniPortType = 0 // relates to PPTP
+	UniVEIP UniPortType = 1 // relates to VEIP
+)
+
+//OntDeviceEntry structure holds information about the attached FSM'as and their communication
+type OnuUniPort struct {
+	enabled    bool
+	name       string
+	portNo     uint32
+	portType   UniPortType
+	ofpPortNo  string
+	uniId      uint16
+	macBpNo    uint16
+	entityId   uint16
+	adminState vc.AdminState_Types
+	operState  vc.OperStatus_Types
+	pPort      *voltha.Port
+}
+
+//NewOnuUniPort returns a new instance of a OnuUniPort
+func NewOnuUniPort(a_uniId uint16, a_portNo uint32, a_InstNo uint16,
+	a_portType UniPortType) *OnuUniPort {
+	logger.Infow("init-onuUniPort", log.Fields{"uniId": a_uniId,
+		"portNo": a_portNo, "InstNo": a_InstNo, "type": a_portType})
+	var onuUniPort OnuUniPort
+	onuUniPort.enabled = false
+	onuUniPort.name = "uni-" + strconv.FormatUint(uint64(a_portNo), 10)
+	onuUniPort.portNo = a_portNo
+	onuUniPort.portType = a_portType
+	// so far it seems as here ofpPortNo/Name ist the same as the original port name ...??
+	onuUniPort.ofpPortNo = onuUniPort.name
+	onuUniPort.uniId = a_uniId
+	onuUniPort.macBpNo = a_uniId + 1 //ensure >0 instanceNo
+	onuUniPort.entityId = a_InstNo
+	onuUniPort.adminState = vc.AdminState_ENABLED //enabled per create
+	onuUniPort.operState = vc.OperStatus_UNKNOWN
+	onuUniPort.pPort = nil // to be set on create
+	return &onuUniPort
+}
+
+//Start starts (logs) the omci agent
+func (oo *OnuUniPort) CreateVolthaPort(a_pDeviceHandler *DeviceHandler) error {
+	logger.Debug("adding-uni-port")
+	pUniPort := &voltha.Port{
+		PortNo:     oo.portNo,
+		Label:      oo.name,
+		Type:       voltha.Port_ETHERNET_UNI,
+		AdminState: oo.adminState,
+		OperStatus: oo.operState,
+		// obviously empty peer setting
+	}
+	if pUniPort != nil {
+		if err := a_pDeviceHandler.coreProxy.PortCreated(context.TODO(),
+			a_pDeviceHandler.deviceID, pUniPort); err != nil {
+			logger.Fatalf("adding-uni-port: create-VOLTHA-Port-failed-%s", err)
+			return err
+		}
+		logger.Infow("Voltha onuUniPort-added", log.Fields{"for PortNo": oo.portNo})
+		oo.pPort = pUniPort
+		oo.operState = vc.OperStatus_DISCOVERED
+	} else {
+		logger.Warnw("could not create Voltha UniPort - nil pointer",
+			log.Fields{"for PortNo": oo.portNo})
+		return errors.New("create Voltha UniPort failed")
+	}
+	return nil
+}
diff --git a/adaptercoreont/openonu.go b/internal/pkg/onuadaptercore/openonu.go
similarity index 84%
rename from adaptercoreont/openonu.go
rename to internal/pkg/onuadaptercore/openonu.go
index 20ba5c5..93584c2 100644
--- a/adaptercoreont/openonu.go
+++ b/internal/pkg/onuadaptercore/openonu.go
@@ -14,8 +14,8 @@
  * limitations under the License.
  */
 
-//Package adaptercoreont provides the utility for onu devices, flows and statistics
-package adaptercoreont
+//Package adaptercoreonu provides the utility for onu devices, flows and statistics
+package adaptercoreonu
 
 import (
 	"context"
@@ -31,7 +31,7 @@
 	"github.com/opencord/voltha-protos/v3/go/openflow_13"
 	"github.com/opencord/voltha-protos/v3/go/voltha"
 
-	"test.internal/openadapter/config"
+	"test.internal/openadapter/internal/pkg/config"
 )
 
 //OpenONUAC structure holds the ONU core information
@@ -51,6 +51,7 @@
 	HeartbeatFailReportInterval time.Duration
 	//GrpcTimeoutInterval         time.Duration
 	lockDeviceHandlersMap sync.RWMutex
+	pSupportedFsms        *OmciDeviceFsms
 }
 
 //NewOpenONUAC returns a new instance of OpenONU_AC
@@ -73,21 +74,39 @@
 	openOnuAc.HeartbeatFailReportInterval = cfg.HeartbeatFailReportInterval
 	//openOnuAc.GrpcTimeoutInterval = cfg.GrpcTimeoutInterval
 	openOnuAc.lockDeviceHandlersMap = sync.RWMutex{}
+
+	openOnuAc.pSupportedFsms = &OmciDeviceFsms{
+		"mib-synchronizer": {
+			//mibSyncFsm,        // Implements the MIB synchronization state machine
+			MibDbVolatileDictImpl, // Implements volatile ME MIB database
+			true,                  // Advertise events on OpenOMCI event bus
+			cMibAuditDelayImpl,    // Time to wait between MIB audits.  0 to disable audits.
+			// map[string]func() error{
+			// 	"mib-upload":    onuDeviceEntry.MibUploadTask,
+			// 	"mib-template":  onuDeviceEntry.MibTemplateTask,
+			// 	"get-mds":       onuDeviceEntry.GetMdsTask,
+			// 	"mib-audit":     onuDeviceEntry.GetMdsTask,
+			// 	"mib-resync":    onuDeviceEntry.MibResyncTask,
+			// 	"mib-reconcile": onuDeviceEntry.MibReconcileTask,
+			// },
+		},
+	}
+
 	return &openOnuAc
 }
 
 //Start starts (logs) the adapter
 func (oo *OpenONUAC) Start(ctx context.Context) error {
-	log.Info("starting-openonu-adapter")
-	log.Info("openonu-adapter-started")
+	logger.Info("starting-openonu-adapter")
+	logger.Info("openonu-adapter-started")
 	return nil
 }
 
 //Stop terminates the session
 func (oo *OpenONUAC) Stop(ctx context.Context) error {
-	log.Info("stopping-device-manager")
+	logger.Info("stopping-device-manager")
 	oo.exitChannel <- 1
-	log.Info("device-manager-stopped")
+	logger.Info("device-manager-stopped")
 	return nil
 }
 
@@ -124,11 +143,11 @@
 // Adopt_device creates a new device handler if not present already and then adopts the device
 func (oo *OpenONUAC) Adopt_device(device *voltha.Device) error {
 	if device == nil {
-		log.Warn("voltha-device-is-nil")
+		logger.Warn("voltha-device-is-nil")
 		return errors.New("nil-device")
 	}
 	ctx := context.Background()
-	log.Infow("adopt-device", log.Fields{"deviceId": device.Id})
+	logger.Infow("adopt-device", log.Fields{"deviceId": device.Id})
 	var handler *DeviceHandler
 	if handler = oo.getDeviceHandler(device.Id); handler == nil {
 		handler := NewDeviceHandler(oo.coreProxy, oo.adapterProxy, oo.eventProxy, device, oo)
@@ -142,27 +161,32 @@
 
 //Get_ofp_device_info returns OFP information for the given device
 func (oo *OpenONUAC) Get_ofp_device_info(device *voltha.Device) (*ic.SwitchCapability, error) {
-	log.Errorw("device-handler-not-set", log.Fields{"deviceId": device.Id})
+	logger.Errorw("device-handler-not-set", log.Fields{"deviceId": device.Id})
 	return nil, errors.New("device-handler-not-set")
 }
 
 //Get_ofp_port_info returns OFP port information for the given device
 func (oo *OpenONUAC) Get_ofp_port_info(device *voltha.Device, portNo int64) (*ic.PortCapability, error) {
-	log.Errorw("device-handler-not-set", log.Fields{"deviceId": device.Id})
+	logger.Errorw("device-handler-not-set", log.Fields{"deviceId": device.Id})
 	return nil, errors.New("device-handler-not-set")
 }
 
 //Process_inter_adapter_message sends messages to a target device (between adapters)
 func (oo *OpenONUAC) Process_inter_adapter_message(msg *ic.InterAdapterMessage) error {
-	log.Debugw("Process_inter_adapter_message", log.Fields{"msgId": msg.Header.Id,
+	logger.Debugw("Process_inter_adapter_message", log.Fields{"msgId": msg.Header.Id,
 		"msgProxyDeviceId": msg.Header.ProxyDeviceId, "msgToDeviceId": msg.Header.ToDeviceId})
 
 	targetDevice := msg.Header.ToDeviceId
 	//ToDeviceId should address an DeviceHandler instance
 	if handler := oo.getDeviceHandler(targetDevice); handler != nil {
-		return handler.ProcessInterAdapterMessage(msg)
+		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
 	}
-	log.Warn("no handler found for received Inter-Proxy-message 'ToDeviceId'")
+	logger.Warn("no handler found for received Inter-Proxy-message 'ToDeviceId'")
 	return fmt.Errorf(fmt.Sprintf("handler-not-found-%s", targetDevice))
 }
 
@@ -291,6 +315,7 @@
 	return errors.New("unImplemented")
 }
 
+//needed for if update >= 3.1.x
 func (oo *OpenONUAC) Child_device_lost(deviceID string, pPortNo uint32, onuID uint32) error {
 	return errors.New("unImplemented")
 }
diff --git a/internal/pkg/onuadaptercore/openonuimpl.go b/internal/pkg/onuadaptercore/openonuimpl.go
new file mode 100644
index 0000000..0678c0c
--- /dev/null
+++ b/internal/pkg/onuadaptercore/openonuimpl.go
@@ -0,0 +1,100 @@
+/*
+ * 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 adaptercoreonu provides the utility for onu devices, flows and statistics
+package adaptercoreonu
+
+import (
+	"errors"
+	//"github.com/opencord/voltha-lib-go/v3/pkg/log"
+)
+
+/*
+OpenOmciAgentDefaults = {
+    'mib-synchronizer': {
+        'state-machine': MibSynchronizer,  # Implements the MIB synchronization state machine
+        'database': MibDbVolatileDict,     # Implements volatile ME MIB database
+        # 'database': MibDbExternal,         # Implements persistent ME MIB database
+        'advertise-events': True,          # Advertise events on OpenOMCI event bus
+        'audit-delay': 60,                 # Time to wait between MIB audits.  0 to disable audits.
+        'tasks': {
+            'mib-upload': MibUploadTask,
+            'mib-template': MibTemplateTask,
+            'get-mds': GetMdsTask,
+            'mib-audit': GetMdsTask,
+            'mib-resync': MibResyncTask,
+            'mib-reconcile': MibReconcileTask
+        }
+    },
+    'omci-capabilities': {
+        'state-machine': OnuOmciCapabilities,   # Implements OMCI capabilities state machine
+        'advertise-events': False,              # Advertise events on OpenOMCI event bus
+        'tasks': {
+            'get-capabilities': OnuCapabilitiesTask # Get supported ME and Commands
+        }
+    },
+    'performance-intervals': {
+        'state-machine': PerformanceIntervals,  # Implements PM Intervals State machine
+        'advertise-events': False,              # Advertise events on OpenOMCI event bus
+        'tasks': {
+            'sync-time': SyncTimeTask,
+            'collect-data': IntervalDataTask,
+            'create-pm': OmciCreatePMRequest,
+            'delete-pm': OmciDeletePMRequest,
+        },
+    },
+    'alarm-synchronizer': {
+        'state-machine': AlarmSynchronizer,    # Implements the Alarm sync state machine
+        'database': AlarmDbExternal,           # For any State storage needs
+        'advertise-events': True,              # Advertise events on OpenOMCI event bus
+        'tasks': {
+            'alarm-resync': AlarmResyncTask
+        }
+     },
+    'image_downloader': {
+        'state-machine': ImageDownloadeSTM,
+        'advertise-event': True,
+        'tasks': {
+            'download-file': FileDownloadTask
+        }
+    },
+    'image_upgrader': {
+        'state-machine': OmciSoftwareImageDownloadSTM,
+        'advertise-event': True,
+        'tasks': {
+            'omci_upgrade_task': OmciSwImageUpgradeTask
+        }
+    }
+    # 'image_activator': {
+    #     'state-machine': OmciSoftwareImageActivateSTM,
+    #     'advertise-event': True,
+    # }
+}
+*/
+
+// do not use MibAudit
+const cMibAuditDelayImpl = 0
+
+//suppose global methods per adapter ...
+func MibDbVolatileDictImpl() error {
+	logger.Debug("MibVolatileDict-called")
+	return errors.New("not_implemented")
+}
+
+func AlarmDbDictImpl() error {
+	logger.Debug("AlarmDb-called")
+	return errors.New("not_implemented")
+}
diff --git a/internal/pkg/onuadaptercore/platform.go b/internal/pkg/onuadaptercore/platform.go
new file mode 100644
index 0000000..8edf4d1
--- /dev/null
+++ b/internal/pkg/onuadaptercore/platform.go
@@ -0,0 +1,292 @@
+/*
+ * 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 core provides the utility for olt devices, flows and statistics
+package adaptercoreonu
+
+//Attention: this file is more or less a coopy of file olt_platform.go from the voltha-openolt-adapter
+// which includes system wide definitions and thus normally should be stored more centrally (within some voltha libs)!!
+
+import (
+	"errors"
+
+	"github.com/opencord/voltha-lib-go/v3/pkg/flows"
+	"github.com/opencord/voltha-lib-go/v3/pkg/log"
+
+	//"github.com/opencord/voltha-openolt-adapter/internal/pkg/olterrors"
+	ofp "github.com/opencord/voltha-protos/v3/go/openflow_13"
+	"github.com/opencord/voltha-protos/v3/go/voltha"
+)
+
+/*=====================================================================
+
+@TODO: Looks like this Flow id concept below is not used anywhere
+       Propose to remove the below documentation of Flow Id on confirmation
+       of the same
+
+Flow id
+
+    Identifies a flow within a single OLT
+    Flow Id is unique per OLT
+    Multiple GEM ports can map to same flow id
+
+     13    11              4      0
+    +--------+--------------+------+
+    | pon id |    onu id    | Flow |
+    |        |              | idx  |
+    +--------+--------------+------+
+
+    14 bits = 16384 flows (per OLT).
+
+    pon id = 4 bits = 16 PON ports
+    onu id = 7 bits = 128 ONUss per PON port
+    Flow index = 3 bits = 4 bi-directional flows per ONU
+                        = 8 uni-directional flows per ONU
+
+
+Logical (OF) UNI port number
+
+    OpenFlow port number corresponding to PON UNI
+
+     20        12              4      0
+    +--+--------+--------------+------+
+    |0 | pon id |    onu id    |uni id|
+    +--+--------+--------------+------+
+
+    pon id = 8 bits = 256 PON ports
+    onu id = 8 bits = 256 ONUs per PON port
+
+Logical (OF) NNI port number
+
+    OpenFlow port number corresponding to PON NNI
+
+     20                             0
+    +--+----------------------------+
+    |1 |                    intf_id |
+    +--+----------------------------+
+
+    No overlap with UNI port number space
+
+
+PON OLT (OF) port number
+
+    OpenFlow port number corresponding to PON OLT ports
+
+     31     28                                 0
+    +--------+------------------------~~~------+
+    |  0x2   |          pon intf id            |
+    +--------+------------------------~~~------+
+*/
+
+const (
+	// Number of bits for the physical UNI of the ONUs
+	bitsForUniID = 4
+	// Number of bits for the ONU ID
+	bitsForONUID = 8
+	// Number of bits for PON ID
+	bitsForPONID = 8
+	// Number of bits to differentiate between UNI and NNI Logical Port
+	bitsForUNINNIDiff = 1
+	//MaxOnusPerPon is Max number of ONUs on any PON port
+	MaxOnusPerPon = (1 << bitsForONUID)
+	//MaxPonsPerOlt is Max number of PON ports on any OLT
+	MaxPonsPerOlt = (1 << bitsForPONID)
+	//MaxUnisPerOnu is the Max number of UNI ports on any ONU
+	MaxUnisPerOnu = (1 << bitsForUniID)
+	//Bit position where the differentiation bit is located
+	nniUniDiffPos = (bitsForUniID + bitsForONUID + bitsForPONID)
+	//Bit position where the marker for PON port type of OF port is present
+	ponIntfMarkerPos = 28
+	//Value of marker used to distinguish PON port type of OF port
+	ponIntfMarkerValue = 0x2
+	// Number of bits for NNI ID
+	bitsforNNIID = 20
+	// minNniIntPortNum is used to store start range of nni port number (1 << 20) 1048576
+	minNniIntPortNum = (1 << bitsforNNIID)
+	// maxNniPortNum is used to store the maximum range of nni port number ((1 << 21)-1) 2097151
+	maxNniPortNum = ((1 << (bitsforNNIID + 1)) - 1)
+)
+
+//MinUpstreamPortID value
+var MinUpstreamPortID = 0xfffd
+
+//MaxUpstreamPortID value
+var MaxUpstreamPortID = 0xfffffffd
+
+var controllerPorts = []uint32{0xfffd, 0x7ffffffd, 0xfffffffd}
+
+//MkUniPortNum returns new UNIportNum based on intfID, inuID and uniID
+func MkUniPortNum(intfID, onuID, uniID uint32) uint32 {
+	//extended for checks available in the python onu adapter:!!
+	var limit = int(intfID)
+	if limit > MaxPonsPerOlt {
+		logger.Warn("Warning: exceeded the MAX pons per OLT")
+	}
+	limit = int(onuID)
+	if limit > MaxOnusPerPon {
+		logger.Warn("Warning: exceeded the MAX ONUS per PON")
+	}
+	limit = int(uniID)
+	if limit > MaxUnisPerOnu {
+		logger.Warn("Warning: exceeded the MAX UNIS per ONU")
+	}
+	return (intfID << (bitsForUniID + bitsForONUID)) | (onuID << bitsForUniID) | uniID
+}
+
+//OnuIDFromPortNum returns ONUID derived from portNumber
+func OnuIDFromPortNum(portNum uint32) uint32 {
+	return (portNum >> bitsForUniID) & (MaxOnusPerPon - 1)
+}
+
+//IntfIDFromUniPortNum returns IntfID derived from portNum
+func IntfIDFromUniPortNum(portNum uint32) uint32 {
+	return (portNum >> (bitsForUniID + bitsForONUID)) & (MaxPonsPerOlt - 1)
+}
+
+//UniIDFromPortNum return UniID derived from portNum
+func UniIDFromPortNum(portNum uint32) uint32 {
+	return (portNum) & (MaxUnisPerOnu - 1)
+}
+
+//IntfIDToPortNo returns portId derived from intftype, intfId and portType
+func IntfIDToPortNo(intfID uint32, intfType voltha.Port_PortType) uint32 {
+	if (intfType) == voltha.Port_ETHERNET_NNI {
+		return (1 << nniUniDiffPos) | intfID
+	}
+	if (intfType) == voltha.Port_PON_OLT {
+		return (ponIntfMarkerValue << ponIntfMarkerPos) | intfID
+	}
+	return 0
+}
+
+//PortNoToIntfID returns portnumber derived from interfaceID
+func PortNoToIntfID(portno uint32, intfType voltha.Port_PortType) uint32 {
+	if (intfType) == voltha.Port_ETHERNET_NNI {
+		return (1 << nniUniDiffPos) ^ portno
+	}
+	if (intfType) == voltha.Port_PON_OLT {
+		return (ponIntfMarkerValue << ponIntfMarkerPos) ^ portno
+	}
+	return 0
+}
+
+//IntfIDFromNniPortNum returns Intf ID derived from portNum
+func IntfIDFromNniPortNum(portNum uint32) (uint32, error) {
+	if portNum < minNniIntPortNum || portNum > maxNniPortNum {
+		logger.Errorw("NNIPortNumber is not in valid range", log.Fields{"portNum": portNum})
+		return uint32(0), errors.New("invalid-port-range") //olterrors.ErrInvalidPortRange
+	}
+	return (portNum & 0xFFFF), nil
+}
+
+//IntfIDToPortTypeName returns port type derived from the intfId
+func IntfIDToPortTypeName(intfID uint32) voltha.Port_PortType {
+	if ((ponIntfMarkerValue << ponIntfMarkerPos) ^ intfID) < MaxPonsPerOlt {
+		return voltha.Port_PON_OLT
+	}
+	if (intfID & (1 << nniUniDiffPos)) == (1 << nniUniDiffPos) {
+		return voltha.Port_ETHERNET_NNI
+	}
+	return voltha.Port_ETHERNET_UNI
+}
+
+//ExtractAccessFromFlow returns AccessDevice information
+func ExtractAccessFromFlow(inPort, outPort uint32) (uint32, uint32, uint32, uint32) {
+	if IsUpstream(outPort) {
+		return inPort, IntfIDFromUniPortNum(inPort), OnuIDFromPortNum(inPort), UniIDFromPortNum(inPort)
+	}
+	return outPort, IntfIDFromUniPortNum(outPort), OnuIDFromPortNum(outPort), UniIDFromPortNum(outPort)
+}
+
+//IsUpstream returns true for Upstream and false for downstream
+func IsUpstream(outPort uint32) bool {
+	for _, port := range controllerPorts {
+		if port == outPort {
+			return true
+		}
+	}
+	return (outPort & (1 << nniUniDiffPos)) == (1 << nniUniDiffPos)
+}
+
+//IsControllerBoundFlow returns true/false
+func IsControllerBoundFlow(outPort uint32) bool {
+	for _, port := range controllerPorts {
+		if port == outPort {
+			return true
+		}
+	}
+	return false
+}
+
+//OnuIDFromUniPortNum returns onuId from give portNum information.
+func OnuIDFromUniPortNum(portNum uint32) uint32 {
+	return (portNum >> bitsForUniID) & (MaxOnusPerPon - 1)
+}
+
+//FlowExtractInfo fetches uniport from the flow, based on which it gets and returns ponInf, onuID, uniID, inPort and ethType
+func FlowExtractInfo(flow *ofp.OfpFlowStats, flowDirection string) (uint32, uint32, uint32, uint32, uint32, uint32, error) {
+	var uniPortNo uint32
+	var ponIntf uint32
+	var onuID uint32
+	var uniID uint32
+	var inPort uint32
+	var ethType uint32
+
+	if flowDirection == "upstream" {
+		if uniPortNo = flows.GetChildPortFromTunnelId(flow); uniPortNo == 0 {
+			for _, field := range flows.GetOfbFields(flow) {
+				if field.GetType() == flows.IN_PORT {
+					uniPortNo = field.GetPort()
+					break
+				}
+			}
+		}
+	} else if flowDirection == "downstream" {
+		if uniPortNo = flows.GetChildPortFromTunnelId(flow); uniPortNo == 0 {
+			for _, field := range flows.GetOfbFields(flow) {
+				if field.GetType() == flows.METADATA {
+					for _, action := range flows.GetActions(flow) {
+						if action.Type == flows.OUTPUT {
+							if out := action.GetOutput(); out != nil {
+								uniPortNo = out.GetPort()
+							}
+							break
+						}
+					}
+				} else if field.GetType() == flows.IN_PORT {
+					inPort = field.GetPort()
+				} else if field.GetType() == flows.ETH_TYPE {
+					ethType = field.GetEthType()
+				}
+			}
+		}
+	}
+
+	if uniPortNo == 0 {
+		return 0, 0, 0, 0, 0, 0, errors.New("NotFound: pon-interface (flowDirection)")
+		// olterrors.NewErrNotFound("pon-interface", log.Fields{"flow-direction": flowDirection}, nil)
+	}
+
+	ponIntf = IntfIDFromUniPortNum(uniPortNo)
+	onuID = OnuIDFromUniPortNum(uniPortNo)
+	uniID = UniIDFromPortNum(uniPortNo)
+
+	logger.Debugw("flow extract info result",
+		log.Fields{"uniPortNo": uniPortNo, "ponIntf": ponIntf,
+			"onuID": onuID, "uniID": uniID, "inPort": inPort, "ethType": ethType})
+
+	return uniPortNo, ponIntf, onuID, uniID, inPort, ethType, nil
+}
diff --git a/pkg/mocks/common.go b/pkg/mocks/common.go
new file mode 100644
index 0000000..3b2df29
--- /dev/null
+++ b/pkg/mocks/common.go
@@ -0,0 +1,33 @@
+/*
+ * 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 mocks Common Logger initialization
+package mocks
+
+import (
+	"github.com/opencord/voltha-lib-go/v3/pkg/log"
+)
+
+var logger log.Logger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.AddPackage(log.JSON, log.ErrorLevel, log.Fields{"pkg": "mocks"})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/vendor/github.com/cevaris/ordered_map/.gitignore b/vendor/github.com/cevaris/ordered_map/.gitignore
new file mode 100644
index 0000000..4af4e59
--- /dev/null
+++ b/vendor/github.com/cevaris/ordered_map/.gitignore
@@ -0,0 +1,5 @@
+*.test
+*~
+
+.idea
+*.iml
diff --git a/vendor/github.com/cevaris/ordered_map/.travis.yml b/vendor/github.com/cevaris/ordered_map/.travis.yml
new file mode 100644
index 0000000..193242f
--- /dev/null
+++ b/vendor/github.com/cevaris/ordered_map/.travis.yml
@@ -0,0 +1,19 @@
+---
+language: go
+
+go:
+  - tip
+  - 1.12
+  - 1.11
+  - 1.10
+  - 1.9
+  - 1.8
+  - 1.7
+  - 1.6
+  - 1.5
+  - 1.4
+  - 1.3
+
+install:
+  - make
+  - make test
diff --git a/vendor/github.com/cevaris/ordered_map/LICENSE.md b/vendor/github.com/cevaris/ordered_map/LICENSE.md
new file mode 100644
index 0000000..4cb9b14
--- /dev/null
+++ b/vendor/github.com/cevaris/ordered_map/LICENSE.md
@@ -0,0 +1,21 @@
+The MIT License (MIT)
+
+Copyright (c) 2015-2016 Adam Cardenas
+
+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/cevaris/ordered_map/Makefile b/vendor/github.com/cevaris/ordered_map/Makefile
new file mode 100644
index 0000000..099e53b
--- /dev/null
+++ b/vendor/github.com/cevaris/ordered_map/Makefile
@@ -0,0 +1,10 @@
+all: build install 
+
+build:
+	go build
+
+install:
+	go install
+
+test:
+	go test -v *.go
diff --git a/vendor/github.com/cevaris/ordered_map/README.md b/vendor/github.com/cevaris/ordered_map/README.md
new file mode 100644
index 0000000..bc3e366
--- /dev/null
+++ b/vendor/github.com/cevaris/ordered_map/README.md
@@ -0,0 +1,113 @@
+# Ordered Map for golang
+
+[![Build Status](https://travis-ci.org/cevaris/ordered_map.svg?branch=master)](https://travis-ci.org/cevaris/ordered_map)
+
+**OrderedMap** is a Python port of OrderedDict implemented in golang. Golang's builtin `map` purposefully randomizes the iteration of stored key/values. **OrderedMap** struct preserves inserted key/value pairs; such that on iteration, key/value pairs are received in inserted (first in, first out) order.
+
+
+## Features
+- Full support Key/Value for all data types
+- Exposes an Iterator that iterates in order of insertion
+- Full Get/Set/Delete map interface
+- Supports Golang v1.3 through v1.12
+
+## Download and Install 
+  
+`go get https://github.com/cevaris/ordered_map.git`
+
+
+## Examples
+
+### Create, Get, Set, Delete
+
+```go
+package main
+
+import (
+    "fmt"
+    "github.com/cevaris/ordered_map"
+)
+
+func main() {
+
+    // Init new OrderedMap
+    om := ordered_map.NewOrderedMap()
+
+    // Set key
+    om.Set("a", 1)
+    om.Set("b", 2)
+    om.Set("c", 3)
+    om.Set("d", 4)
+
+    // Same interface as builtin map
+    if val, ok := om.Get("b"); ok == true {
+        // Found key "b"
+        fmt.Println(val)
+    }
+
+    // Delete a key
+    om.Delete("c")
+
+    // Failed Get lookup becase we deleted "c"
+    if _, ok := om.Get("c"); ok == false {
+        // Did not find key "c"
+        fmt.Println("c not found")
+    }
+    
+    fmt.Println(om)
+}
+```
+
+
+### Iterator
+
+```go
+n := 100
+om := ordered_map.NewOrderedMap()
+
+for i := 0; i < n; i++ {
+    // Insert data into OrderedMap
+    om.Set(i, fmt.Sprintf("%d", i * i))
+}
+
+// Iterate though values
+// - Values iteration are in insert order
+// - Returned in a key/value pair struct
+iter := om.IterFunc()
+for kv, ok := iter(); ok; kv, ok = iter() {
+    fmt.Println(kv, kv.Key, kv.Value)
+}
+```
+
+### Custom Structs
+
+```go
+om := ordered_map.NewOrderedMap()
+om.Set("one", &MyStruct{1, 1.1})
+om.Set("two", &MyStruct{2, 2.2})
+om.Set("three", &MyStruct{3, 3.3})
+
+fmt.Println(om)
+// Ouput: OrderedMap[one:&{1 1.1},  two:&{2 2.2},  three:&{3 3.3}, ]
+```
+  
+## For Development
+
+Git clone project 
+
+`git clone https://github.com/cevaris/ordered_map.git`  
+  
+Build and install project
+
+`make`
+
+Run tests 
+
+`make test`
+
+
+
+
+
+
+
diff --git a/vendor/github.com/cevaris/ordered_map/key_pair.go b/vendor/github.com/cevaris/ordered_map/key_pair.go
new file mode 100644
index 0000000..88afbcf
--- /dev/null
+++ b/vendor/github.com/cevaris/ordered_map/key_pair.go
@@ -0,0 +1,16 @@
+package ordered_map
+
+import "fmt"
+
+type KVPair struct {
+	Key   interface{}
+	Value interface{}
+}
+
+func (k *KVPair) String() string {
+	return fmt.Sprintf("%v:%v", k.Key, k.Value)
+}
+
+func (kv1 *KVPair) Compare(kv2 *KVPair) bool {
+	return kv1.Key == kv2.Key && kv1.Value == kv2.Value
+}
\ No newline at end of file
diff --git a/vendor/github.com/cevaris/ordered_map/node.go b/vendor/github.com/cevaris/ordered_map/node.go
new file mode 100644
index 0000000..ad0d142
--- /dev/null
+++ b/vendor/github.com/cevaris/ordered_map/node.go
@@ -0,0 +1,62 @@
+package ordered_map
+
+import (
+	"fmt"
+	"bytes"
+)
+
+type node struct {
+	Prev  *node
+	Next  *node
+	Value interface{}
+}
+
+func newRootNode() *node {
+	root := &node{}
+	root.Prev = root
+	root.Next = root
+	return root
+}
+
+func newNode(prev *node, next *node, key interface{}) *node {
+	return &node{Prev: prev, Next: next, Value: key}
+}
+
+func (n *node) Add(value string) {
+	root := n
+	last := root.Prev
+	last.Next = newNode(last, n, value)
+	root.Prev = last.Next
+}
+
+func (n *node) String() string {
+	var buffer bytes.Buffer
+	if n.Value == "" {
+		// Need to sentinel
+		var curr *node
+		root := n
+		curr = root.Next
+		for curr != root {
+			buffer.WriteString(fmt.Sprintf("%s, ", curr.Value))
+			curr = curr.Next
+		}
+	} else {
+		// Else, print pointer value
+		buffer.WriteString(fmt.Sprintf("%p, ", &n))
+	}
+	return fmt.Sprintf("LinkList[%v]", buffer.String())
+}
+
+func (n *node) IterFunc() func() (string, bool) {
+	var curr *node
+	root := n
+	curr = root.Next
+	return func() (string, bool) {
+		for curr != root {
+			tmp := curr.Value.(string)
+			curr = curr.Next
+			return tmp, true
+		}
+		return "", false
+	}
+}
diff --git a/vendor/github.com/cevaris/ordered_map/ordered_map.go b/vendor/github.com/cevaris/ordered_map/ordered_map.go
new file mode 100644
index 0000000..4116cd8
--- /dev/null
+++ b/vendor/github.com/cevaris/ordered_map/ordered_map.go
@@ -0,0 +1,121 @@
+package ordered_map
+
+import (
+	"fmt"
+)
+
+type OrderedMap struct {
+	store  map[interface{}]interface{}
+	mapper map[interface{}]*node
+	root   *node
+}
+
+func NewOrderedMap() *OrderedMap {
+	om := &OrderedMap{
+		store:  make(map[interface{}]interface{}),
+		mapper: make(map[interface{}]*node),
+		root:   newRootNode(),
+	}
+	return om
+}
+
+func NewOrderedMapWithArgs(args []*KVPair) *OrderedMap {
+	om := NewOrderedMap()
+	om.update(args)
+	return om
+}
+
+func (om *OrderedMap) update(args []*KVPair) {
+	for _, pair := range args {
+		om.Set(pair.Key, pair.Value)
+	}
+}
+
+func (om *OrderedMap) Set(key interface{}, value interface{}) {
+	if _, ok := om.store[key]; ok == false {
+		root := om.root
+		last := root.Prev
+		last.Next = newNode(last, root, key)
+		root.Prev = last.Next
+		om.mapper[key] = last.Next
+	}
+	om.store[key] = value
+}
+
+func (om *OrderedMap) Get(key interface{}) (interface{}, bool) {
+	val, ok := om.store[key]
+	return val, ok
+}
+
+func (om *OrderedMap) Delete(key interface{}) {
+	_, ok := om.store[key]
+	if ok {
+		delete(om.store, key)
+	}
+	root, rootFound := om.mapper[key]
+	if rootFound {
+		prev := root.Prev
+		next := root.Next
+		prev.Next = next
+		next.Prev = prev
+		delete(om.mapper, key)
+	}
+}
+
+func (om *OrderedMap) String() string {
+	builder := make([]string, len(om.store))
+
+	var index int = 0
+	iter := om.IterFunc()
+	for kv, ok := iter(); ok; kv, ok = iter() {
+		val, _ := om.Get(kv.Key)
+		builder[index] = fmt.Sprintf("%v:%v, ", kv.Key, val)
+		index++
+	}
+	return fmt.Sprintf("OrderedMap%v", builder)
+}
+
+func (om *OrderedMap) Iter() <-chan *KVPair {
+	println("Iter() method is deprecated!. Use IterFunc() instead.")
+	return om.UnsafeIter()
+}
+
+/*
+Beware, Iterator leaks goroutines if we do not fully traverse the map.
+For most cases, `IterFunc()` should work as an iterator.
+ */
+func (om *OrderedMap) UnsafeIter() <-chan *KVPair {
+	keys := make(chan *KVPair)
+	go func() {
+		defer close(keys)
+		var curr *node
+		root := om.root
+		curr = root.Next
+		for curr != root {
+			v, _ := om.store[curr.Value]
+			keys <- &KVPair{curr.Value, v}
+			curr = curr.Next
+		}
+	}()
+	return keys
+}
+
+func (om *OrderedMap) IterFunc() func() (*KVPair, bool) {
+	var curr *node
+	root := om.root
+	curr = root.Next
+	return func() (*KVPair, bool) {
+		for curr != root {
+			tmp := curr
+			curr = curr.Next
+			v, _ := om.store[tmp.Value]
+			return &KVPair{tmp.Value, v}, true
+		}
+		return nil, false
+	}
+}
+
+func (om *OrderedMap) Len() int {
+	return len(om.store)
+}
+
diff --git a/vendor/github.com/google/gopacket/layers/gen.go b/vendor/github.com/google/gopacket/layers/gen.go
new file mode 100644
index 0000000..ab7a0c0
--- /dev/null
+++ b/vendor/github.com/google/gopacket/layers/gen.go
@@ -0,0 +1,109 @@
+// Copyright 2012 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 in the root of the source
+// tree.
+
+// +build ignore
+
+// This binary pulls known ports from IANA, and uses them to populate
+// iana_ports.go's TCPPortNames and UDPPortNames maps.
+//
+//  go run gen.go | gofmt > iana_ports.go
+package main
+
+import (
+	"bytes"
+	"encoding/xml"
+	"flag"
+	"fmt"
+	"io/ioutil"
+	"net/http"
+	"os"
+	"strconv"
+	"time"
+)
+
+const fmtString = `// Copyright 2012 Google, Inc. All rights reserved.
+
+package layers
+
+// Created by gen.go, don't edit manually
+// Generated at %s
+// Fetched from %q
+
+// TCPPortNames contains the port names for all TCP ports.
+var TCPPortNames = tcpPortNames
+
+// UDPPortNames contains the port names for all UDP ports.
+var UDPPortNames = udpPortNames
+
+// SCTPPortNames contains the port names for all SCTP ports.
+var SCTPPortNames = sctpPortNames
+
+var tcpPortNames = map[TCPPort]string{
+%s}
+var udpPortNames = map[UDPPort]string{
+%s}
+var sctpPortNames = map[SCTPPort]string{
+%s}
+`
+
+var url = flag.String("url", "http://www.iana.org/assignments/service-names-port-numbers/service-names-port-numbers.xml", "URL to grab port numbers from")
+
+func main() {
+	fmt.Fprintf(os.Stderr, "Fetching ports from %q\n", *url)
+	resp, err := http.Get(*url)
+	if err != nil {
+		panic(err)
+	}
+	defer resp.Body.Close()
+	body, err := ioutil.ReadAll(resp.Body)
+	if err != nil {
+		panic(err)
+	}
+	fmt.Fprintln(os.Stderr, "Parsing XML")
+	var registry struct {
+		Records []struct {
+			Protocol string `xml:"protocol"`
+			Number   string `xml:"number"`
+			Name     string `xml:"name"`
+		} `xml:"record"`
+	}
+	xml.Unmarshal(body, &registry)
+	var tcpPorts bytes.Buffer
+	var udpPorts bytes.Buffer
+	var sctpPorts bytes.Buffer
+	done := map[string]map[int]bool{
+		"tcp":  map[int]bool{},
+		"udp":  map[int]bool{},
+		"sctp": map[int]bool{},
+	}
+	for _, r := range registry.Records {
+		port, err := strconv.Atoi(r.Number)
+		if err != nil {
+			continue
+		}
+		if r.Name == "" {
+			continue
+		}
+		var b *bytes.Buffer
+		switch r.Protocol {
+		case "tcp":
+			b = &tcpPorts
+		case "udp":
+			b = &udpPorts
+		case "sctp":
+			b = &sctpPorts
+		default:
+			continue
+		}
+		if done[r.Protocol][port] {
+			continue
+		}
+		done[r.Protocol][port] = true
+		fmt.Fprintf(b, "\t%d: %q,\n", port, r.Name)
+	}
+	fmt.Fprintln(os.Stderr, "Writing results to stdout")
+	fmt.Printf(fmtString, time.Now(), *url, tcpPorts.String(), udpPorts.String(), sctpPorts.String())
+}
diff --git a/vendor/github.com/google/gopacket/layers/gen2.go b/vendor/github.com/google/gopacket/layers/gen2.go
new file mode 100644
index 0000000..150cad7
--- /dev/null
+++ b/vendor/github.com/google/gopacket/layers/gen2.go
@@ -0,0 +1,104 @@
+// Copyright 2012 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 in the root of the source
+// tree.
+
+// +build ignore
+
+// This binary handles creating string constants and function templates for enums.
+//
+//  go run gen.go | gofmt > enums_generated.go
+package main
+
+import (
+	"fmt"
+	"log"
+	"os"
+	"text/template"
+	"time"
+)
+
+const fmtString = `// Copyright 2012 Google, Inc. All rights reserved.
+
+package layers
+
+// Created by gen2.go, don't edit manually
+// Generated at %s
+
+import (
+  "fmt"
+
+  "github.com/google/gopacket"
+)
+
+`
+
+var funcsTmpl = template.Must(template.New("foo").Parse(`
+// Decoder calls {{.Name}}Metadata.DecodeWith's decoder.
+func (a {{.Name}}) Decode(data []byte, p gopacket.PacketBuilder) error {
+	return {{.Name}}Metadata[a].DecodeWith.Decode(data, p)
+}
+// String returns {{.Name}}Metadata.Name.
+func (a {{.Name}}) String() string {
+	return {{.Name}}Metadata[a].Name
+}
+// LayerType returns {{.Name}}Metadata.LayerType.
+func (a {{.Name}}) LayerType() gopacket.LayerType {
+	return {{.Name}}Metadata[a].LayerType
+}
+
+type errorDecoderFor{{.Name}} int
+func (a *errorDecoderFor{{.Name}}) Decode(data []byte, p gopacket.PacketBuilder) error {
+  return a
+}
+func (a *errorDecoderFor{{.Name}}) Error() string {
+  return fmt.Sprintf("Unable to decode {{.Name}} %d", int(*a))
+}
+
+var errorDecodersFor{{.Name}} [{{.Num}}]errorDecoderFor{{.Name}}
+var {{.Name}}Metadata [{{.Num}}]EnumMetadata
+
+func initUnknownTypesFor{{.Name}}() {
+  for i := 0; i < {{.Num}}; i++ {
+    errorDecodersFor{{.Name}}[i] = errorDecoderFor{{.Name}}(i)
+    {{.Name}}Metadata[i] = EnumMetadata{
+      DecodeWith: &errorDecodersFor{{.Name}}[i],
+      Name: "Unknown{{.Name}}",
+    }
+  }
+}
+`))
+
+func main() {
+	fmt.Fprintf(os.Stderr, "Writing results to stdout\n")
+	fmt.Printf(fmtString, time.Now())
+	types := []struct {
+		Name string
+		Num  int
+	}{
+		{"LinkType", 256},
+		{"EthernetType", 65536},
+		{"PPPType", 65536},
+		{"IPProtocol", 256},
+		{"SCTPChunkType", 256},
+		{"PPPoECode", 256},
+		{"FDDIFrameControl", 256},
+		{"EAPOLType", 256},
+		{"ProtocolFamily", 256},
+		{"Dot11Type", 256},
+		{"USBTransportType", 256},
+	}
+
+	fmt.Println("func init() {")
+	for _, t := range types {
+		fmt.Printf("initUnknownTypesFor%s()\n", t.Name)
+	}
+	fmt.Println("initActualTypeData()")
+	fmt.Println("}")
+	for _, t := range types {
+		if err := funcsTmpl.Execute(os.Stdout, t); err != nil {
+			log.Fatalf("Failed to execute template %s: %v", t.Name, err)
+		}
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v3/pkg/config/common.go b/vendor/github.com/opencord/voltha-lib-go/v3/pkg/config/common.go
new file mode 100644
index 0000000..37e05fd
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v3/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/v3/pkg/log"
+)
+
+var logger log.Logger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.AddPackage(log.JSON, log.ErrorLevel, log.Fields{"pkg": "config"})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v3/pkg/config/configmanager.go b/vendor/github.com/opencord/voltha-lib-go/v3/pkg/config/configmanager.go
new file mode 100644
index 0000000..724ad32
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v3/pkg/config/configmanager.go
@@ -0,0 +1,271 @@
+/*
+ * 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"
+	"github.com/opencord/voltha-lib-go/v3/pkg/db"
+	"github.com/opencord/voltha-lib-go/v3/pkg/db/kvstore"
+	"github.com/opencord/voltha-lib-go/v3/pkg/log"
+	"strings"
+)
+
+const (
+	defaultkvStoreConfigPath = "config"
+	kvStoreDataPathPrefix    = "service/voltha"
+	kvStorePathSeparator     = "/"
+)
+
+// ConfigType represents the type for which config is created inside the kvstore
+// For example, loglevel
+type ConfigType int
+
+const (
+	ConfigTypeLogLevel ConfigType = iota
+	ConfigTypeKafka
+)
+
+func (c ConfigType) String() string {
+	return [...]string{"loglevel", "kafka"}[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
+}
+
+// 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(kvClient kvstore.Client, kvStoreType, kvStoreHost string, kvStorePort, kvStoreTimeout int) *ConfigManager {
+
+	return &ConfigManager{
+		KvStoreConfigPrefix: defaultkvStoreConfigPath,
+		Backend: &db.Backend{
+			Client:     kvClient,
+			StoreType:  kvStoreType,
+			Host:       kvStoreHost,
+			Port:       kvStorePort,
+			Timeout:    kvStoreTimeout,
+			PathPrefix: kvStoreDataPathPrefix,
+		},
+	}
+}
+
+// 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 := 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("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()
+
+	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() {
+
+	ccKeyPrefix := c.makeConfigPath()
+
+	logger.Debugw("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("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("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("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("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("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("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/v3/pkg/config/logcontroller.go b/vendor/github.com/opencord/voltha-lib-go/v3/pkg/config/logcontroller.go
new file mode 100644
index 0000000..9c36241
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v3/pkg/config/logcontroller.go
@@ -0,0 +1,361 @@
+/*
+ * 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/v3/pkg/log"
+	"os"
+	"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
+)
+
+// 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(cm *ConfigManager) (*ComponentLogController, error) {
+
+	logger.Debug("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(cm)
+	if err != nil {
+		logger.Errorw("unable-to-construct-component-log-controller-instance-for-log-config-monitoring", log.Fields{"error": err})
+		return
+	}
+
+	cc.GlobalConfig = cm.InitComponentConfig(globalConfigRootNode, ConfigTypeLogLevel)
+	logger.Debugw("global-log-config", log.Fields{"cc-global-config": cc.GlobalConfig})
+
+	cc.componentNameConfig = cm.InitComponentConfig(cc.ComponentName, ConfigTypeLogLevel)
+	logger.Debugw("component-log-config", log.Fields{"cc-component-name-config": cc.componentNameConfig})
+
+	cc.persistInitialDefaultLogConfigs(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("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("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("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("failed-to-persist-component-default-log-config-at-startup", log.Fields{"error": err, "loglevel": c.initialLogLevel})
+		}
+	}
+}
+
+// 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("unable-to-load-log-config-at-startup", log.Fields{"error": err})
+	} else {
+		if err := c.loadAndApplyLogConfig(initialLogConfig); err != nil {
+			logger.Warnw("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("processing-log-config-change", log.Fields{"ChangeType": configEvent.ChangeType, "Package": configEvent.ConfigAttribute})
+
+		updatedLogConfig, err := c.buildUpdatedLogConfig(ctx)
+		if err != nil {
+			logger.Warnw("unable-to-fetch-updated-log-config", log.Fields{"error": err})
+			continue
+		}
+
+		logger.Debugw("applying-updated-log-config", log.Fields{"updated-log-config": updatedLogConfig})
+
+		if err := c.loadAndApplyLogConfig(updatedLogConfig); err != nil {
+			logger.Warnw("unable-to-load-and-apply-log-config", log.Fields{"error": err})
+		}
+	}
+
+}
+
+// get active loglevel from the zap logger
+func getActiveLogLevels() 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("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("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("global-default-loglevel-not-found-in-config-store")
+		globalDefaultLogLevel = initialGlobalDefaultLogLevelValue
+	}
+
+	if _, err := log.StringToLogLevel(globalDefaultLogLevel); err != nil {
+		logger.Warnw("unsupported-loglevel-config-defined-at-global-default", log.Fields{"log-level": globalDefaultLogLevel})
+		globalDefaultLogLevel = initialGlobalDefaultLogLevelValue
+	}
+
+	logger.Debugw("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("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("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("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(logConfig map[string]string) error {
+	currentLogHash, err := GenerateLogConfigHash(logConfig)
+	if err != nil {
+		return err
+	}
+
+	if c.logHash != currentLogHash {
+		UpdateLogLevels(logConfig)
+		c.logHash = currentLogHash
+	} else {
+		logger.Debug("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(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("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(updatedLogConfig map[string]string) {
+
+	activeLogLevels := getActiveLogLevels()
+	changedLogLevels := createModifiedLogLevels(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("no-change-in-effective-loglevel-config")
+		return
+	}
+
+	logger.Debugw("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/v3/pkg/flows/common.go b/vendor/github.com/opencord/voltha-lib-go/v3/pkg/flows/common.go
new file mode 100644
index 0000000..557de3f
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v3/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/v3/pkg/log"
+)
+
+var logger log.Logger
+
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	var err error
+	logger, err = log.AddPackage(log.JSON, log.ErrorLevel, log.Fields{"pkg": "flowsUtils"})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v3/pkg/flows/flow_utils.go b/vendor/github.com/opencord/voltha-lib-go/v3/pkg/flows/flow_utils.go
new file mode 100644
index 0000000..b2086cd
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v3/pkg/flows/flow_utils.go
@@ -0,0 +1,1371 @@
+/*
+ * 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"
+	"crypto/md5"
+	"errors"
+	"fmt"
+	"github.com/cevaris/ordered_map"
+	"github.com/gogo/protobuf/proto"
+	"github.com/opencord/voltha-lib-go/v3/pkg/log"
+	ofp "github.com/opencord/voltha-protos/v3/go/openflow_13"
+	"math/big"
+	"strings"
+)
+
+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 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(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("No-metadata-present")
+	return 0
+}
+
+func GetMetaData64Bit(flow *ofp.OfpFlowStats) uint64 {
+	if flow == nil {
+		return 0
+	}
+	for _, field := range GetOfbFields(flow) {
+		if field.Type == METADATA {
+			return field.GetTableMetadata()
+		}
+	}
+	logger.Debug("No-metadata-present")
+	return 0
+}
+
+// function returns write metadata value from write_metadata action field
+func GetMetadataFromWriteMetadataAction(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("No-write-metadata-present", log.Fields{"flow": flow})
+	return 0
+}
+
+func GetTechProfileIDFromWriteMetaData(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("Write metadata value for Techprofile ID", log.Fields{"metadata": metadata})
+	if metadata != 0 {
+		tpId = uint16((metadata >> 32) & 0xFFFF)
+		logger.Debugw("Found techprofile ID from write metadata action", log.Fields{"tpid": tpId})
+	}
+	return tpId
+}
+
+func GetEgressPortNumberFromWriteMetadata(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(flow)
+	logger.Debugw("Metadata found for egress/uni port ", log.Fields{"metadata": md})
+	if md != 0 {
+		uniPort = uint32(md & 0xFFFFFFFF)
+		logger.Debugw("Found EgressPort from write metadata action", log.Fields{"egress_port": uniPort})
+	}
+	return uniPort
+
+}
+
+func GetInnerTagFromMetaData(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(flow)
+	if md != 0 {
+		innerTag = uint16((md >> 48) & 0xFFFF)
+		logger.Debugw("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("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
+}
+
+// Return unique 64-bit integer hash for flow covering the following attributes:
+// 'table_id', 'priority', 'flags', 'cookie', 'match', '_instruction_string'
+func HashFlowStats(flow *ofp.OfpFlowStats) (uint64, error) {
+	if flow == nil { // Should never happen
+		return 0, errors.New("hash-flow-stats-nil-flow")
+	}
+	// Create string with the instructions field first
+	var instructionString bytes.Buffer
+	for _, instruction := range flow.Instructions {
+		instructionString.WriteString(instruction.String())
+	}
+	var flowString = fmt.Sprintf("%d%d%d%d%s%s", flow.TableId, flow.Priority, flow.Flags, flow.Cookie, flow.Match.String(), instructionString.String())
+	h := md5.New()
+	if _, err := h.Write([]byte(flowString)); err != nil {
+		return 0, fmt.Errorf("hash-flow-stats-failed-hash: %v", err)
+	}
+	hash := big.NewInt(0)
+	hash.SetBytes(h.Sum(nil))
+	return hash.Uint64(), 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(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("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("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 FlowMatch(f, flow) {
+			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 {
+	if f1 == nil || f2 == nil {
+		return false
+	}
+	keysMatter := []string{"TableId", "Priority", "Flags", "Cookie", "Match"}
+	for _, key := range keysMatter {
+		switch key {
+		case "TableId":
+			if f1.TableId != f2.TableId {
+				return false
+			}
+		case "Priority":
+			if f1.Priority != f2.Priority {
+				return false
+			}
+		case "Flags":
+			if f1.Flags != f2.Flags {
+				return false
+			}
+		case "Cookie":
+			if f1.Cookie != f2.Cookie {
+				return false
+			}
+		case "Match":
+			if strings.Compare(f1.Match.String(), f2.Match.String()) != 0 {
+				return false
+			}
+		}
+	}
+	return true
+}
+
+//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()
+}
diff --git a/config/version/version.go b/vendor/github.com/opencord/voltha-lib-go/v3/pkg/version/version.go
similarity index 63%
rename from config/version/version.go
rename to vendor/github.com/opencord/voltha-lib-go/v3/pkg/version/version.go
index ea2441d..49c0b10 100644
--- a/config/version/version.go
+++ b/vendor/github.com/opencord/voltha-lib-go/v3/pkg/version/version.go
@@ -1,12 +1,23 @@
-// Package version is used to inject build time information via -X variables
+/*
+ * 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 version
 
 import (
 	"fmt"
-	"io/ioutil"
 	"strings"
-
-	"github.com/opencord/voltha-lib-go/v3/pkg/log"
 )
 
 // Default build-time variable.
@@ -22,8 +33,7 @@
 	arch      = "unknown-arch"
 )
 
-// InfoType is a collection of build time environment variables
-type InfoType struct {
+type VersionInfoType struct {
 	Version   string `json:"version"`
 	GoVersion string `json:"goversion"`
 	VcsRef    string `json:"vcsref"`
@@ -33,11 +43,10 @@
 	Arch      string `json:"arch"`
 }
 
-// VersionInfo is an instance of build time environment variables populated at build time via -X arguments
-var VersionInfo InfoType
+var VersionInfo VersionInfoType
 
 func init() {
-	VersionInfo = InfoType{
+	VersionInfo = VersionInfoType{
 		Version:   version,
 		VcsRef:    vcsRef,
 		VcsDirty:  vcsDirty,
@@ -46,10 +55,9 @@
 		Arch:      arch,
 		BuildTime: buildTime,
 	}
-	_, _ = log.AddPackage(log.CONSOLE, log.DebugLevel, nil)
 }
 
-func (v InfoType) String(indent string) string {
+func (v VersionInfoType) String(indent string) string {
 	builder := strings.Builder{}
 
 	builder.WriteString(fmt.Sprintf("%sVersion:      %s\n", indent, VersionInfo.Version))
@@ -60,17 +68,3 @@
 	builder.WriteString(fmt.Sprintf("%sOS/Arch:      %s/%s\n", indent, VersionInfo.Os, VersionInfo.Arch))
 	return builder.String()
 }
-
-func GetCodeVersion() string {
-	if VersionInfo.Version == "unknown-version" {
-		content, err := ioutil.ReadFile("VERSION")
-		if err == nil {
-			return (string(content))
-		} else {
-			log.Error("VERSION-file not readable")
-			return VersionInfo.Version
-		}
-	} else {
-		return VersionInfo.Version
-	}
-}
diff --git a/vendor/golang.org/x/sys/unix/mkasm_darwin.go b/vendor/golang.org/x/sys/unix/mkasm_darwin.go
new file mode 100644
index 0000000..6f7bb6e
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/mkasm_darwin.go
@@ -0,0 +1,78 @@
+// 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 ignore
+
+// mkasm_darwin.go generates assembly trampolines to call libSystem routines from Go.
+//This program must be run after mksyscall.go.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"log"
+	"os"
+	"strings"
+)
+
+func writeASMFile(in string, fileName string, buildTags string) {
+	trampolines := map[string]bool{}
+
+	var out bytes.Buffer
+
+	fmt.Fprintf(&out, "// go run mkasm_darwin.go %s\n", strings.Join(os.Args[1:], " "))
+	fmt.Fprintf(&out, "// Code generated by the command above; DO NOT EDIT.\n")
+	fmt.Fprintf(&out, "\n")
+	fmt.Fprintf(&out, "// +build %s\n", buildTags)
+	fmt.Fprintf(&out, "\n")
+	fmt.Fprintf(&out, "#include \"textflag.h\"\n")
+	for _, line := range strings.Split(in, "\n") {
+		if !strings.HasPrefix(line, "func ") || !strings.HasSuffix(line, "_trampoline()") {
+			continue
+		}
+		fn := line[5 : len(line)-13]
+		if !trampolines[fn] {
+			trampolines[fn] = true
+			fmt.Fprintf(&out, "TEXT ·%s_trampoline(SB),NOSPLIT,$0-0\n", fn)
+			fmt.Fprintf(&out, "\tJMP\t%s(SB)\n", fn)
+		}
+	}
+	err := ioutil.WriteFile(fileName, out.Bytes(), 0644)
+	if err != nil {
+		log.Fatalf("can't write %s: %s", fileName, err)
+	}
+}
+
+func main() {
+	in1, err := ioutil.ReadFile("syscall_darwin.go")
+	if err != nil {
+		log.Fatalf("can't open syscall_darwin.go: %s", err)
+	}
+	arch := os.Args[1]
+	in2, err := ioutil.ReadFile(fmt.Sprintf("syscall_darwin_%s.go", arch))
+	if err != nil {
+		log.Fatalf("can't open syscall_darwin_%s.go: %s", arch, err)
+	}
+	in3, err := ioutil.ReadFile(fmt.Sprintf("zsyscall_darwin_%s.go", arch))
+	if err != nil {
+		log.Fatalf("can't open zsyscall_darwin_%s.go: %s", arch, err)
+	}
+	in := string(in1) + string(in2) + string(in3)
+
+	writeASMFile(in, fmt.Sprintf("zsyscall_darwin_%s.s", arch), "go1.12")
+
+	in1, err = ioutil.ReadFile("syscall_darwin.1_13.go")
+	if err != nil {
+		log.Fatalf("can't open syscall_darwin.1_13.go: %s", err)
+	}
+	in2, err = ioutil.ReadFile(fmt.Sprintf("zsyscall_darwin_%s.1_13.go", arch))
+	if err != nil {
+		log.Fatalf("can't open zsyscall_darwin_%s.1_13.go: %s", arch, err)
+	}
+
+	in = string(in1) + string(in2)
+
+	writeASMFile(in, fmt.Sprintf("zsyscall_darwin_%s.1_13.s", arch), "go1.13")
+}
diff --git a/vendor/golang.org/x/sys/unix/mkpost.go b/vendor/golang.org/x/sys/unix/mkpost.go
new file mode 100644
index 0000000..5ee1a16
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/mkpost.go
@@ -0,0 +1,127 @@
+// 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 ignore
+
+// mkpost processes the output of cgo -godefs to
+// modify the generated types. It is used to clean up
+// the sys API in an architecture specific manner.
+//
+// mkpost is run after cgo -godefs; see README.md.
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"go/format"
+	"io/ioutil"
+	"log"
+	"os"
+	"regexp"
+)
+
+func main() {
+	// Get the OS and architecture (using GOARCH_TARGET if it exists)
+	goos := os.Getenv("GOOS")
+	goarch := os.Getenv("GOARCH_TARGET")
+	if goarch == "" {
+		goarch = os.Getenv("GOARCH")
+	}
+	// Check that we are using the Docker-based build system if we should be.
+	if goos == "linux" {
+		if os.Getenv("GOLANG_SYS_BUILD") != "docker" {
+			os.Stderr.WriteString("In the Docker-based build system, mkpost should not be called directly.\n")
+			os.Stderr.WriteString("See README.md\n")
+			os.Exit(1)
+		}
+	}
+
+	b, err := ioutil.ReadAll(os.Stdin)
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	if goos == "aix" {
+		// Replace type of Atim, Mtim and Ctim by Timespec in Stat_t
+		// to avoid having both StTimespec and Timespec.
+		sttimespec := regexp.MustCompile(`_Ctype_struct_st_timespec`)
+		b = sttimespec.ReplaceAll(b, []byte("Timespec"))
+	}
+
+	// Intentionally export __val fields in Fsid and Sigset_t
+	valRegex := regexp.MustCompile(`type (Fsid|Sigset_t) struct {(\s+)X__(bits|val)(\s+\S+\s+)}`)
+	b = valRegex.ReplaceAll(b, []byte("type $1 struct {${2}Val$4}"))
+
+	// Intentionally export __fds_bits field in FdSet
+	fdSetRegex := regexp.MustCompile(`type (FdSet) struct {(\s+)X__fds_bits(\s+\S+\s+)}`)
+	b = fdSetRegex.ReplaceAll(b, []byte("type $1 struct {${2}Bits$3}"))
+
+	// If we have empty Ptrace structs, we should delete them. Only s390x emits
+	// nonempty Ptrace structs.
+	ptraceRexexp := regexp.MustCompile(`type Ptrace((Psw|Fpregs|Per) struct {\s*})`)
+	b = ptraceRexexp.ReplaceAll(b, nil)
+
+	// Replace the control_regs union with a blank identifier for now.
+	controlRegsRegex := regexp.MustCompile(`(Control_regs)\s+\[0\]uint64`)
+	b = controlRegsRegex.ReplaceAll(b, []byte("_ [0]uint64"))
+
+	// Remove fields that are added by glibc
+	// Note that this is unstable as the identifers are private.
+	removeFieldsRegex := regexp.MustCompile(`X__glibc\S*`)
+	b = removeFieldsRegex.ReplaceAll(b, []byte("_"))
+
+	// Convert [65]int8 to [65]byte in Utsname members to simplify
+	// conversion to string; see golang.org/issue/20753
+	convertUtsnameRegex := regexp.MustCompile(`((Sys|Node|Domain)name|Release|Version|Machine)(\s+)\[(\d+)\]u?int8`)
+	b = convertUtsnameRegex.ReplaceAll(b, []byte("$1$3[$4]byte"))
+
+	// Convert [n]int8 to [n]byte in Statvfs_t members to simplify
+	// conversion to string.
+	convertStatvfsRegex := regexp.MustCompile(`((Fstype|Mnton|Mntfrom)name)(\s+)\[(\d+)\]int8`)
+	b = convertStatvfsRegex.ReplaceAll(b, []byte("$1$3[$4]byte"))
+
+	// Convert [1024]int8 to [1024]byte in Ptmget members
+	convertPtmget := regexp.MustCompile(`([SC]n)(\s+)\[(\d+)\]u?int8`)
+	b = convertPtmget.ReplaceAll(b, []byte("$1[$3]byte"))
+
+	// Remove spare fields (e.g. in Statx_t)
+	spareFieldsRegex := regexp.MustCompile(`X__spare\S*`)
+	b = spareFieldsRegex.ReplaceAll(b, []byte("_"))
+
+	// Remove cgo padding fields
+	removePaddingFieldsRegex := regexp.MustCompile(`Pad_cgo_\d+`)
+	b = removePaddingFieldsRegex.ReplaceAll(b, []byte("_"))
+
+	// Remove padding, hidden, or unused fields
+	removeFieldsRegex = regexp.MustCompile(`\b(X_\S+|Padding)`)
+	b = removeFieldsRegex.ReplaceAll(b, []byte("_"))
+
+	// Remove the first line of warning from cgo
+	b = b[bytes.IndexByte(b, '\n')+1:]
+	// Modify the command in the header to include:
+	//  mkpost, our own warning, and a build tag.
+	replacement := fmt.Sprintf(`$1 | go run mkpost.go
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+// +build %s,%s`, goarch, goos)
+	cgoCommandRegex := regexp.MustCompile(`(cgo -godefs .*)`)
+	b = cgoCommandRegex.ReplaceAll(b, []byte(replacement))
+
+	// Rename Stat_t time fields
+	if goos == "freebsd" && goarch == "386" {
+		// Hide Stat_t.[AMCB]tim_ext fields
+		renameStatTimeExtFieldsRegex := regexp.MustCompile(`[AMCB]tim_ext`)
+		b = renameStatTimeExtFieldsRegex.ReplaceAll(b, []byte("_"))
+	}
+	renameStatTimeFieldsRegex := regexp.MustCompile(`([AMCB])(?:irth)?time?(?:spec)?\s+(Timespec|StTimespec)`)
+	b = renameStatTimeFieldsRegex.ReplaceAll(b, []byte("${1}tim ${2}"))
+
+	// gofmt
+	b, err = format.Source(b)
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	os.Stdout.Write(b)
+}
diff --git a/vendor/golang.org/x/sys/unix/mksyscall.go b/vendor/golang.org/x/sys/unix/mksyscall.go
new file mode 100644
index 0000000..9e540cc
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/mksyscall.go
@@ -0,0 +1,402 @@
+// 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 ignore
+
+/*
+This program reads a file containing function prototypes
+(like syscall_darwin.go) and generates system call bodies.
+The prototypes are marked by lines beginning with "//sys"
+and read like func declarations if //sys is replaced by func, but:
+	* The parameter lists must give a name for each argument.
+	  This includes return parameters.
+	* The parameter lists must give a type for each argument:
+	  the (x, y, z int) shorthand is not allowed.
+	* If the return parameter is an error number, it must be named errno.
+
+A line beginning with //sysnb is like //sys, except that the
+goroutine will not be suspended during the execution of the system
+call.  This must only be used for system calls which can never
+block, as otherwise the system call could cause all goroutines to
+hang.
+*/
+package main
+
+import (
+	"bufio"
+	"flag"
+	"fmt"
+	"os"
+	"regexp"
+	"strings"
+)
+
+var (
+	b32       = flag.Bool("b32", false, "32bit big-endian")
+	l32       = flag.Bool("l32", false, "32bit little-endian")
+	plan9     = flag.Bool("plan9", false, "plan9")
+	openbsd   = flag.Bool("openbsd", false, "openbsd")
+	netbsd    = flag.Bool("netbsd", false, "netbsd")
+	dragonfly = flag.Bool("dragonfly", false, "dragonfly")
+	arm       = flag.Bool("arm", false, "arm") // 64-bit value should use (even, odd)-pair
+	tags      = flag.String("tags", "", "build tags")
+	filename  = flag.String("output", "", "output file name (standard output if omitted)")
+)
+
+// cmdLine returns this programs's commandline arguments
+func cmdLine() string {
+	return "go run mksyscall.go " + strings.Join(os.Args[1:], " ")
+}
+
+// buildTags returns build tags
+func buildTags() string {
+	return *tags
+}
+
+// Param is function parameter
+type Param struct {
+	Name string
+	Type string
+}
+
+// usage prints the program usage
+func usage() {
+	fmt.Fprintf(os.Stderr, "usage: go run mksyscall.go [-b32 | -l32] [-tags x,y] [file ...]\n")
+	os.Exit(1)
+}
+
+// parseParamList parses parameter list and returns a slice of parameters
+func parseParamList(list string) []string {
+	list = strings.TrimSpace(list)
+	if list == "" {
+		return []string{}
+	}
+	return regexp.MustCompile(`\s*,\s*`).Split(list, -1)
+}
+
+// parseParam splits a parameter into name and type
+func parseParam(p string) Param {
+	ps := regexp.MustCompile(`^(\S*) (\S*)$`).FindStringSubmatch(p)
+	if ps == nil {
+		fmt.Fprintf(os.Stderr, "malformed parameter: %s\n", p)
+		os.Exit(1)
+	}
+	return Param{ps[1], ps[2]}
+}
+
+func main() {
+	// Get the OS and architecture (using GOARCH_TARGET if it exists)
+	goos := os.Getenv("GOOS")
+	if goos == "" {
+		fmt.Fprintln(os.Stderr, "GOOS not defined in environment")
+		os.Exit(1)
+	}
+	goarch := os.Getenv("GOARCH_TARGET")
+	if goarch == "" {
+		goarch = os.Getenv("GOARCH")
+	}
+
+	// Check that we are using the Docker-based build system if we should
+	if goos == "linux" {
+		if os.Getenv("GOLANG_SYS_BUILD") != "docker" {
+			fmt.Fprintf(os.Stderr, "In the Docker-based build system, mksyscall should not be called directly.\n")
+			fmt.Fprintf(os.Stderr, "See README.md\n")
+			os.Exit(1)
+		}
+	}
+
+	flag.Usage = usage
+	flag.Parse()
+	if len(flag.Args()) <= 0 {
+		fmt.Fprintf(os.Stderr, "no files to parse provided\n")
+		usage()
+	}
+
+	endianness := ""
+	if *b32 {
+		endianness = "big-endian"
+	} else if *l32 {
+		endianness = "little-endian"
+	}
+
+	libc := false
+	if goos == "darwin" && (strings.Contains(buildTags(), ",go1.12") || strings.Contains(buildTags(), ",go1.13")) {
+		libc = true
+	}
+	trampolines := map[string]bool{}
+
+	text := ""
+	for _, path := range flag.Args() {
+		file, err := os.Open(path)
+		if err != nil {
+			fmt.Fprintf(os.Stderr, err.Error())
+			os.Exit(1)
+		}
+		s := bufio.NewScanner(file)
+		for s.Scan() {
+			t := s.Text()
+			t = strings.TrimSpace(t)
+			t = regexp.MustCompile(`\s+`).ReplaceAllString(t, ` `)
+			nonblock := regexp.MustCompile(`^\/\/sysnb `).FindStringSubmatch(t)
+			if regexp.MustCompile(`^\/\/sys `).FindStringSubmatch(t) == nil && nonblock == nil {
+				continue
+			}
+
+			// Line must be of the form
+			//	func Open(path string, mode int, perm int) (fd int, errno error)
+			// Split into name, in params, out params.
+			f := regexp.MustCompile(`^\/\/sys(nb)? (\w+)\(([^()]*)\)\s*(?:\(([^()]+)\))?\s*(?:=\s*((?i)SYS_[A-Z0-9_]+))?$`).FindStringSubmatch(t)
+			if f == nil {
+				fmt.Fprintf(os.Stderr, "%s:%s\nmalformed //sys declaration\n", path, t)
+				os.Exit(1)
+			}
+			funct, inps, outps, sysname := f[2], f[3], f[4], f[5]
+
+			// ClockGettime doesn't have a syscall number on Darwin, only generate libc wrappers.
+			if goos == "darwin" && !libc && funct == "ClockGettime" {
+				continue
+			}
+
+			// Split argument lists on comma.
+			in := parseParamList(inps)
+			out := parseParamList(outps)
+
+			// Try in vain to keep people from editing this file.
+			// The theory is that they jump into the middle of the file
+			// without reading the header.
+			text += "// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT\n\n"
+
+			// Go function header.
+			outDecl := ""
+			if len(out) > 0 {
+				outDecl = fmt.Sprintf(" (%s)", strings.Join(out, ", "))
+			}
+			text += fmt.Sprintf("func %s(%s)%s {\n", funct, strings.Join(in, ", "), outDecl)
+
+			// Check if err return available
+			errvar := ""
+			for _, param := range out {
+				p := parseParam(param)
+				if p.Type == "error" {
+					errvar = p.Name
+					break
+				}
+			}
+
+			// Prepare arguments to Syscall.
+			var args []string
+			n := 0
+			for _, param := range in {
+				p := parseParam(param)
+				if regexp.MustCompile(`^\*`).FindStringSubmatch(p.Type) != nil {
+					args = append(args, "uintptr(unsafe.Pointer("+p.Name+"))")
+				} else if p.Type == "string" && errvar != "" {
+					text += fmt.Sprintf("\tvar _p%d *byte\n", n)
+					text += fmt.Sprintf("\t_p%d, %s = BytePtrFromString(%s)\n", n, errvar, p.Name)
+					text += fmt.Sprintf("\tif %s != nil {\n\t\treturn\n\t}\n", errvar)
+					args = append(args, fmt.Sprintf("uintptr(unsafe.Pointer(_p%d))", n))
+					n++
+				} else if p.Type == "string" {
+					fmt.Fprintf(os.Stderr, path+":"+funct+" uses string arguments, but has no error return\n")
+					text += fmt.Sprintf("\tvar _p%d *byte\n", n)
+					text += fmt.Sprintf("\t_p%d, _ = BytePtrFromString(%s)\n", n, p.Name)
+					args = append(args, fmt.Sprintf("uintptr(unsafe.Pointer(_p%d))", n))
+					n++
+				} else if regexp.MustCompile(`^\[\](.*)`).FindStringSubmatch(p.Type) != nil {
+					// Convert slice into pointer, length.
+					// Have to be careful not to take address of &a[0] if len == 0:
+					// pass dummy pointer in that case.
+					// Used to pass nil, but some OSes or simulators reject write(fd, nil, 0).
+					text += fmt.Sprintf("\tvar _p%d unsafe.Pointer\n", n)
+					text += fmt.Sprintf("\tif len(%s) > 0 {\n\t\t_p%d = unsafe.Pointer(&%s[0])\n\t}", p.Name, n, p.Name)
+					text += fmt.Sprintf(" else {\n\t\t_p%d = unsafe.Pointer(&_zero)\n\t}\n", n)
+					args = append(args, fmt.Sprintf("uintptr(_p%d)", n), fmt.Sprintf("uintptr(len(%s))", p.Name))
+					n++
+				} else if p.Type == "int64" && (*openbsd || *netbsd) {
+					args = append(args, "0")
+					if endianness == "big-endian" {
+						args = append(args, fmt.Sprintf("uintptr(%s>>32)", p.Name), fmt.Sprintf("uintptr(%s)", p.Name))
+					} else if endianness == "little-endian" {
+						args = append(args, fmt.Sprintf("uintptr(%s)", p.Name), fmt.Sprintf("uintptr(%s>>32)", p.Name))
+					} else {
+						args = append(args, fmt.Sprintf("uintptr(%s)", p.Name))
+					}
+				} else if p.Type == "int64" && *dragonfly {
+					if regexp.MustCompile(`^(?i)extp(read|write)`).FindStringSubmatch(funct) == nil {
+						args = append(args, "0")
+					}
+					if endianness == "big-endian" {
+						args = append(args, fmt.Sprintf("uintptr(%s>>32)", p.Name), fmt.Sprintf("uintptr(%s)", p.Name))
+					} else if endianness == "little-endian" {
+						args = append(args, fmt.Sprintf("uintptr(%s)", p.Name), fmt.Sprintf("uintptr(%s>>32)", p.Name))
+					} else {
+						args = append(args, fmt.Sprintf("uintptr(%s)", p.Name))
+					}
+				} else if (p.Type == "int64" || p.Type == "uint64") && endianness != "" {
+					if len(args)%2 == 1 && *arm {
+						// arm abi specifies 64-bit argument uses
+						// (even, odd) pair
+						args = append(args, "0")
+					}
+					if endianness == "big-endian" {
+						args = append(args, fmt.Sprintf("uintptr(%s>>32)", p.Name), fmt.Sprintf("uintptr(%s)", p.Name))
+					} else {
+						args = append(args, fmt.Sprintf("uintptr(%s)", p.Name), fmt.Sprintf("uintptr(%s>>32)", p.Name))
+					}
+				} else {
+					args = append(args, fmt.Sprintf("uintptr(%s)", p.Name))
+				}
+			}
+
+			// Determine which form to use; pad args with zeros.
+			asm := "Syscall"
+			if nonblock != nil {
+				if errvar == "" && goos == "linux" {
+					asm = "RawSyscallNoError"
+				} else {
+					asm = "RawSyscall"
+				}
+			} else {
+				if errvar == "" && goos == "linux" {
+					asm = "SyscallNoError"
+				}
+			}
+			if len(args) <= 3 {
+				for len(args) < 3 {
+					args = append(args, "0")
+				}
+			} else if len(args) <= 6 {
+				asm += "6"
+				for len(args) < 6 {
+					args = append(args, "0")
+				}
+			} else if len(args) <= 9 {
+				asm += "9"
+				for len(args) < 9 {
+					args = append(args, "0")
+				}
+			} else {
+				fmt.Fprintf(os.Stderr, "%s:%s too many arguments to system call\n", path, funct)
+			}
+
+			// System call number.
+			if sysname == "" {
+				sysname = "SYS_" + funct
+				sysname = regexp.MustCompile(`([a-z])([A-Z])`).ReplaceAllString(sysname, `${1}_$2`)
+				sysname = strings.ToUpper(sysname)
+			}
+
+			var libcFn string
+			if libc {
+				asm = "syscall_" + strings.ToLower(asm[:1]) + asm[1:] // internal syscall call
+				sysname = strings.TrimPrefix(sysname, "SYS_")         // remove SYS_
+				sysname = strings.ToLower(sysname)                    // lowercase
+				libcFn = sysname
+				sysname = "funcPC(libc_" + sysname + "_trampoline)"
+			}
+
+			// Actual call.
+			arglist := strings.Join(args, ", ")
+			call := fmt.Sprintf("%s(%s, %s)", asm, sysname, arglist)
+
+			// Assign return values.
+			body := ""
+			ret := []string{"_", "_", "_"}
+			doErrno := false
+			for i := 0; i < len(out); i++ {
+				p := parseParam(out[i])
+				reg := ""
+				if p.Name == "err" && !*plan9 {
+					reg = "e1"
+					ret[2] = reg
+					doErrno = true
+				} else if p.Name == "err" && *plan9 {
+					ret[0] = "r0"
+					ret[2] = "e1"
+					break
+				} else {
+					reg = fmt.Sprintf("r%d", i)
+					ret[i] = reg
+				}
+				if p.Type == "bool" {
+					reg = fmt.Sprintf("%s != 0", reg)
+				}
+				if p.Type == "int64" && endianness != "" {
+					// 64-bit number in r1:r0 or r0:r1.
+					if i+2 > len(out) {
+						fmt.Fprintf(os.Stderr, "%s:%s not enough registers for int64 return\n", path, funct)
+					}
+					if endianness == "big-endian" {
+						reg = fmt.Sprintf("int64(r%d)<<32 | int64(r%d)", i, i+1)
+					} else {
+						reg = fmt.Sprintf("int64(r%d)<<32 | int64(r%d)", i+1, i)
+					}
+					ret[i] = fmt.Sprintf("r%d", i)
+					ret[i+1] = fmt.Sprintf("r%d", i+1)
+				}
+				if reg != "e1" || *plan9 {
+					body += fmt.Sprintf("\t%s = %s(%s)\n", p.Name, p.Type, reg)
+				}
+			}
+			if ret[0] == "_" && ret[1] == "_" && ret[2] == "_" {
+				text += fmt.Sprintf("\t%s\n", call)
+			} else {
+				if errvar == "" && goos == "linux" {
+					// raw syscall without error on Linux, see golang.org/issue/22924
+					text += fmt.Sprintf("\t%s, %s := %s\n", ret[0], ret[1], call)
+				} else {
+					text += fmt.Sprintf("\t%s, %s, %s := %s\n", ret[0], ret[1], ret[2], call)
+				}
+			}
+			text += body
+
+			if *plan9 && ret[2] == "e1" {
+				text += "\tif int32(r0) == -1 {\n"
+				text += "\t\terr = e1\n"
+				text += "\t}\n"
+			} else if doErrno {
+				text += "\tif e1 != 0 {\n"
+				text += "\t\terr = errnoErr(e1)\n"
+				text += "\t}\n"
+			}
+			text += "\treturn\n"
+			text += "}\n\n"
+
+			if libc && !trampolines[libcFn] {
+				// some system calls share a trampoline, like read and readlen.
+				trampolines[libcFn] = true
+				// Declare assembly trampoline.
+				text += fmt.Sprintf("func libc_%s_trampoline()\n", libcFn)
+				// Assembly trampoline calls the libc_* function, which this magic
+				// redirects to use the function from libSystem.
+				text += fmt.Sprintf("//go:linkname libc_%s libc_%s\n", libcFn, libcFn)
+				text += fmt.Sprintf("//go:cgo_import_dynamic libc_%s %s \"/usr/lib/libSystem.B.dylib\"\n", libcFn, libcFn)
+				text += "\n"
+			}
+		}
+		if err := s.Err(); err != nil {
+			fmt.Fprintf(os.Stderr, err.Error())
+			os.Exit(1)
+		}
+		file.Close()
+	}
+	fmt.Printf(srcTemplate, cmdLine(), buildTags(), text)
+}
+
+const srcTemplate = `// %s
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+// +build %s
+
+package unix
+
+import (
+	"syscall"
+	"unsafe"
+)
+
+var _ syscall.Errno
+
+%s
+`
diff --git a/vendor/golang.org/x/sys/unix/mksyscall_aix_ppc.go b/vendor/golang.org/x/sys/unix/mksyscall_aix_ppc.go
new file mode 100644
index 0000000..3be3cdf
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/mksyscall_aix_ppc.go
@@ -0,0 +1,415 @@
+// 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 ignore
+
+/*
+This program reads a file containing function prototypes
+(like syscall_aix.go) and generates system call bodies.
+The prototypes are marked by lines beginning with "//sys"
+and read like func declarations if //sys is replaced by func, but:
+	* The parameter lists must give a name for each argument.
+	  This includes return parameters.
+	* The parameter lists must give a type for each argument:
+	  the (x, y, z int) shorthand is not allowed.
+	* If the return parameter is an error number, it must be named err.
+	* If go func name needs to be different than its libc name,
+	* or the function is not in libc, name could be specified
+	* at the end, after "=" sign, like
+	  //sys getsockopt(s int, level int, name int, val uintptr, vallen *_Socklen) (err error) = libsocket.getsockopt
+*/
+package main
+
+import (
+	"bufio"
+	"flag"
+	"fmt"
+	"os"
+	"regexp"
+	"strings"
+)
+
+var (
+	b32  = flag.Bool("b32", false, "32bit big-endian")
+	l32  = flag.Bool("l32", false, "32bit little-endian")
+	aix  = flag.Bool("aix", false, "aix")
+	tags = flag.String("tags", "", "build tags")
+)
+
+// cmdLine returns this programs's commandline arguments
+func cmdLine() string {
+	return "go run mksyscall_aix_ppc.go " + strings.Join(os.Args[1:], " ")
+}
+
+// buildTags returns build tags
+func buildTags() string {
+	return *tags
+}
+
+// Param is function parameter
+type Param struct {
+	Name string
+	Type string
+}
+
+// usage prints the program usage
+func usage() {
+	fmt.Fprintf(os.Stderr, "usage: go run mksyscall_aix_ppc.go [-b32 | -l32] [-tags x,y] [file ...]\n")
+	os.Exit(1)
+}
+
+// parseParamList parses parameter list and returns a slice of parameters
+func parseParamList(list string) []string {
+	list = strings.TrimSpace(list)
+	if list == "" {
+		return []string{}
+	}
+	return regexp.MustCompile(`\s*,\s*`).Split(list, -1)
+}
+
+// parseParam splits a parameter into name and type
+func parseParam(p string) Param {
+	ps := regexp.MustCompile(`^(\S*) (\S*)$`).FindStringSubmatch(p)
+	if ps == nil {
+		fmt.Fprintf(os.Stderr, "malformed parameter: %s\n", p)
+		os.Exit(1)
+	}
+	return Param{ps[1], ps[2]}
+}
+
+func main() {
+	flag.Usage = usage
+	flag.Parse()
+	if len(flag.Args()) <= 0 {
+		fmt.Fprintf(os.Stderr, "no files to parse provided\n")
+		usage()
+	}
+
+	endianness := ""
+	if *b32 {
+		endianness = "big-endian"
+	} else if *l32 {
+		endianness = "little-endian"
+	}
+
+	pack := ""
+	text := ""
+	cExtern := "/*\n#include <stdint.h>\n#include <stddef.h>\n"
+	for _, path := range flag.Args() {
+		file, err := os.Open(path)
+		if err != nil {
+			fmt.Fprintf(os.Stderr, err.Error())
+			os.Exit(1)
+		}
+		s := bufio.NewScanner(file)
+		for s.Scan() {
+			t := s.Text()
+			t = strings.TrimSpace(t)
+			t = regexp.MustCompile(`\s+`).ReplaceAllString(t, ` `)
+			if p := regexp.MustCompile(`^package (\S+)$`).FindStringSubmatch(t); p != nil && pack == "" {
+				pack = p[1]
+			}
+			nonblock := regexp.MustCompile(`^\/\/sysnb `).FindStringSubmatch(t)
+			if regexp.MustCompile(`^\/\/sys `).FindStringSubmatch(t) == nil && nonblock == nil {
+				continue
+			}
+
+			// Line must be of the form
+			//	func Open(path string, mode int, perm int) (fd int, err error)
+			// Split into name, in params, out params.
+			f := regexp.MustCompile(`^\/\/sys(nb)? (\w+)\(([^()]*)\)\s*(?:\(([^()]+)\))?\s*(?:=\s*(?:(\w*)\.)?(\w*))?$`).FindStringSubmatch(t)
+			if f == nil {
+				fmt.Fprintf(os.Stderr, "%s:%s\nmalformed //sys declaration\n", path, t)
+				os.Exit(1)
+			}
+			funct, inps, outps, modname, sysname := f[2], f[3], f[4], f[5], f[6]
+
+			// Split argument lists on comma.
+			in := parseParamList(inps)
+			out := parseParamList(outps)
+
+			inps = strings.Join(in, ", ")
+			outps = strings.Join(out, ", ")
+
+			// Try in vain to keep people from editing this file.
+			// The theory is that they jump into the middle of the file
+			// without reading the header.
+			text += "// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT\n\n"
+
+			// Check if value return, err return available
+			errvar := ""
+			retvar := ""
+			rettype := ""
+			for _, param := range out {
+				p := parseParam(param)
+				if p.Type == "error" {
+					errvar = p.Name
+				} else {
+					retvar = p.Name
+					rettype = p.Type
+				}
+			}
+
+			// System call name.
+			if sysname == "" {
+				sysname = funct
+			}
+			sysname = regexp.MustCompile(`([a-z])([A-Z])`).ReplaceAllString(sysname, `${1}_$2`)
+			sysname = strings.ToLower(sysname) // All libc functions are lowercase.
+
+			cRettype := ""
+			if rettype == "unsafe.Pointer" {
+				cRettype = "uintptr_t"
+			} else if rettype == "uintptr" {
+				cRettype = "uintptr_t"
+			} else if regexp.MustCompile(`^_`).FindStringSubmatch(rettype) != nil {
+				cRettype = "uintptr_t"
+			} else if rettype == "int" {
+				cRettype = "int"
+			} else if rettype == "int32" {
+				cRettype = "int"
+			} else if rettype == "int64" {
+				cRettype = "long long"
+			} else if rettype == "uint32" {
+				cRettype = "unsigned int"
+			} else if rettype == "uint64" {
+				cRettype = "unsigned long long"
+			} else {
+				cRettype = "int"
+			}
+			if sysname == "exit" {
+				cRettype = "void"
+			}
+
+			// Change p.Types to c
+			var cIn []string
+			for _, param := range in {
+				p := parseParam(param)
+				if regexp.MustCompile(`^\*`).FindStringSubmatch(p.Type) != nil {
+					cIn = append(cIn, "uintptr_t")
+				} else if p.Type == "string" {
+					cIn = append(cIn, "uintptr_t")
+				} else if regexp.MustCompile(`^\[\](.*)`).FindStringSubmatch(p.Type) != nil {
+					cIn = append(cIn, "uintptr_t", "size_t")
+				} else if p.Type == "unsafe.Pointer" {
+					cIn = append(cIn, "uintptr_t")
+				} else if p.Type == "uintptr" {
+					cIn = append(cIn, "uintptr_t")
+				} else if regexp.MustCompile(`^_`).FindStringSubmatch(p.Type) != nil {
+					cIn = append(cIn, "uintptr_t")
+				} else if p.Type == "int" {
+					cIn = append(cIn, "int")
+				} else if p.Type == "int32" {
+					cIn = append(cIn, "int")
+				} else if p.Type == "int64" {
+					cIn = append(cIn, "long long")
+				} else if p.Type == "uint32" {
+					cIn = append(cIn, "unsigned int")
+				} else if p.Type == "uint64" {
+					cIn = append(cIn, "unsigned long long")
+				} else {
+					cIn = append(cIn, "int")
+				}
+			}
+
+			if funct != "fcntl" && funct != "FcntlInt" && funct != "readlen" && funct != "writelen" {
+				if sysname == "select" {
+					// select is a keyword of Go. Its name is
+					// changed to c_select.
+					cExtern += "#define c_select select\n"
+				}
+				// Imports of system calls from libc
+				cExtern += fmt.Sprintf("%s %s", cRettype, sysname)
+				cIn := strings.Join(cIn, ", ")
+				cExtern += fmt.Sprintf("(%s);\n", cIn)
+			}
+
+			// So file name.
+			if *aix {
+				if modname == "" {
+					modname = "libc.a/shr_64.o"
+				} else {
+					fmt.Fprintf(os.Stderr, "%s: only syscall using libc are available\n", funct)
+					os.Exit(1)
+				}
+			}
+
+			strconvfunc := "C.CString"
+
+			// Go function header.
+			if outps != "" {
+				outps = fmt.Sprintf(" (%s)", outps)
+			}
+			if text != "" {
+				text += "\n"
+			}
+
+			text += fmt.Sprintf("func %s(%s)%s {\n", funct, strings.Join(in, ", "), outps)
+
+			// Prepare arguments to Syscall.
+			var args []string
+			n := 0
+			argN := 0
+			for _, param := range in {
+				p := parseParam(param)
+				if regexp.MustCompile(`^\*`).FindStringSubmatch(p.Type) != nil {
+					args = append(args, "C.uintptr_t(uintptr(unsafe.Pointer("+p.Name+")))")
+				} else if p.Type == "string" && errvar != "" {
+					text += fmt.Sprintf("\t_p%d := uintptr(unsafe.Pointer(%s(%s)))\n", n, strconvfunc, p.Name)
+					args = append(args, fmt.Sprintf("C.uintptr_t(_p%d)", n))
+					n++
+				} else if p.Type == "string" {
+					fmt.Fprintf(os.Stderr, path+":"+funct+" uses string arguments, but has no error return\n")
+					text += fmt.Sprintf("\t_p%d := uintptr(unsafe.Pointer(%s(%s)))\n", n, strconvfunc, p.Name)
+					args = append(args, fmt.Sprintf("C.uintptr_t(_p%d)", n))
+					n++
+				} else if m := regexp.MustCompile(`^\[\](.*)`).FindStringSubmatch(p.Type); m != nil {
+					// Convert slice into pointer, length.
+					// Have to be careful not to take address of &a[0] if len == 0:
+					// pass nil in that case.
+					text += fmt.Sprintf("\tvar _p%d *%s\n", n, m[1])
+					text += fmt.Sprintf("\tif len(%s) > 0 {\n\t\t_p%d = &%s[0]\n\t}\n", p.Name, n, p.Name)
+					args = append(args, fmt.Sprintf("C.uintptr_t(uintptr(unsafe.Pointer(_p%d)))", n))
+					n++
+					text += fmt.Sprintf("\tvar _p%d int\n", n)
+					text += fmt.Sprintf("\t_p%d = len(%s)\n", n, p.Name)
+					args = append(args, fmt.Sprintf("C.size_t(_p%d)", n))
+					n++
+				} else if p.Type == "int64" && endianness != "" {
+					if endianness == "big-endian" {
+						args = append(args, fmt.Sprintf("uintptr(%s>>32)", p.Name), fmt.Sprintf("uintptr(%s)", p.Name))
+					} else {
+						args = append(args, fmt.Sprintf("uintptr(%s)", p.Name), fmt.Sprintf("uintptr(%s>>32)", p.Name))
+					}
+					n++
+				} else if p.Type == "bool" {
+					text += fmt.Sprintf("\tvar _p%d uint32\n", n)
+					text += fmt.Sprintf("\tif %s {\n\t\t_p%d = 1\n\t} else {\n\t\t_p%d = 0\n\t}\n", p.Name, n, n)
+					args = append(args, fmt.Sprintf("_p%d", n))
+				} else if regexp.MustCompile(`^_`).FindStringSubmatch(p.Type) != nil {
+					args = append(args, fmt.Sprintf("C.uintptr_t(uintptr(%s))", p.Name))
+				} else if p.Type == "unsafe.Pointer" {
+					args = append(args, fmt.Sprintf("C.uintptr_t(uintptr(%s))", p.Name))
+				} else if p.Type == "int" {
+					if (argN == 2) && ((funct == "readlen") || (funct == "writelen")) {
+						args = append(args, fmt.Sprintf("C.size_t(%s)", p.Name))
+					} else if argN == 0 && funct == "fcntl" {
+						args = append(args, fmt.Sprintf("C.uintptr_t(%s)", p.Name))
+					} else if (argN == 2) && ((funct == "fcntl") || (funct == "FcntlInt")) {
+						args = append(args, fmt.Sprintf("C.uintptr_t(%s)", p.Name))
+					} else {
+						args = append(args, fmt.Sprintf("C.int(%s)", p.Name))
+					}
+				} else if p.Type == "int32" {
+					args = append(args, fmt.Sprintf("C.int(%s)", p.Name))
+				} else if p.Type == "int64" {
+					args = append(args, fmt.Sprintf("C.longlong(%s)", p.Name))
+				} else if p.Type == "uint32" {
+					args = append(args, fmt.Sprintf("C.uint(%s)", p.Name))
+				} else if p.Type == "uint64" {
+					args = append(args, fmt.Sprintf("C.ulonglong(%s)", p.Name))
+				} else if p.Type == "uintptr" {
+					args = append(args, fmt.Sprintf("C.uintptr_t(%s)", p.Name))
+				} else {
+					args = append(args, fmt.Sprintf("C.int(%s)", p.Name))
+				}
+				argN++
+			}
+
+			// Actual call.
+			arglist := strings.Join(args, ", ")
+			call := ""
+			if sysname == "exit" {
+				if errvar != "" {
+					call += "er :="
+				} else {
+					call += ""
+				}
+			} else if errvar != "" {
+				call += "r0,er :="
+			} else if retvar != "" {
+				call += "r0,_ :="
+			} else {
+				call += ""
+			}
+			if sysname == "select" {
+				// select is a keyword of Go. Its name is
+				// changed to c_select.
+				call += fmt.Sprintf("C.c_%s(%s)", sysname, arglist)
+			} else {
+				call += fmt.Sprintf("C.%s(%s)", sysname, arglist)
+			}
+
+			// Assign return values.
+			body := ""
+			for i := 0; i < len(out); i++ {
+				p := parseParam(out[i])
+				reg := ""
+				if p.Name == "err" {
+					reg = "e1"
+				} else {
+					reg = "r0"
+				}
+				if reg != "e1" {
+					body += fmt.Sprintf("\t%s = %s(%s)\n", p.Name, p.Type, reg)
+				}
+			}
+
+			// verify return
+			if sysname != "exit" && errvar != "" {
+				if regexp.MustCompile(`^uintptr`).FindStringSubmatch(cRettype) != nil {
+					body += "\tif (uintptr(r0) ==^uintptr(0) && er != nil) {\n"
+					body += fmt.Sprintf("\t\t%s = er\n", errvar)
+					body += "\t}\n"
+				} else {
+					body += "\tif (r0 ==-1 && er != nil) {\n"
+					body += fmt.Sprintf("\t\t%s = er\n", errvar)
+					body += "\t}\n"
+				}
+			} else if errvar != "" {
+				body += "\tif (er != nil) {\n"
+				body += fmt.Sprintf("\t\t%s = er\n", errvar)
+				body += "\t}\n"
+			}
+
+			text += fmt.Sprintf("\t%s\n", call)
+			text += body
+
+			text += "\treturn\n"
+			text += "}\n"
+		}
+		if err := s.Err(); err != nil {
+			fmt.Fprintf(os.Stderr, err.Error())
+			os.Exit(1)
+		}
+		file.Close()
+	}
+	imp := ""
+	if pack != "unix" {
+		imp = "import \"golang.org/x/sys/unix\"\n"
+
+	}
+	fmt.Printf(srcTemplate, cmdLine(), buildTags(), pack, cExtern, imp, text)
+}
+
+const srcTemplate = `// %s
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+// +build %s
+
+package %s
+
+
+%s
+*/
+import "C"
+import (
+	"unsafe"
+)
+
+
+%s
+
+%s
+`
diff --git a/vendor/golang.org/x/sys/unix/mksyscall_aix_ppc64.go b/vendor/golang.org/x/sys/unix/mksyscall_aix_ppc64.go
new file mode 100644
index 0000000..c960099
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/mksyscall_aix_ppc64.go
@@ -0,0 +1,614 @@
+// 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 ignore
+
+/*
+This program reads a file containing function prototypes
+(like syscall_aix.go) and generates system call bodies.
+The prototypes are marked by lines beginning with "//sys"
+and read like func declarations if //sys is replaced by func, but:
+	* The parameter lists must give a name for each argument.
+	  This includes return parameters.
+	* The parameter lists must give a type for each argument:
+	  the (x, y, z int) shorthand is not allowed.
+	* If the return parameter is an error number, it must be named err.
+	* If go func name needs to be different than its libc name,
+	* or the function is not in libc, name could be specified
+	* at the end, after "=" sign, like
+	  //sys getsockopt(s int, level int, name int, val uintptr, vallen *_Socklen) (err error) = libsocket.getsockopt
+
+
+This program will generate three files and handle both gc and gccgo implementation:
+  - zsyscall_aix_ppc64.go: the common part of each implementation (error handler, pointer creation)
+  - zsyscall_aix_ppc64_gc.go: gc part with //go_cgo_import_dynamic and a call to syscall6
+  - zsyscall_aix_ppc64_gccgo.go: gccgo part with C function and conversion to C type.
+
+ The generated code looks like this
+
+zsyscall_aix_ppc64.go
+func asyscall(...) (n int, err error) {
+	 // Pointer Creation
+	 r1, e1 := callasyscall(...)
+	 // Type Conversion
+	 // Error Handler
+	 return
+}
+
+zsyscall_aix_ppc64_gc.go
+//go:cgo_import_dynamic libc_asyscall asyscall "libc.a/shr_64.o"
+//go:linkname libc_asyscall libc_asyscall
+var asyscall syscallFunc
+
+func callasyscall(...) (r1 uintptr, e1 Errno) {
+	 r1, _, e1 = syscall6(uintptr(unsafe.Pointer(&libc_asyscall)), "nb_args", ... )
+	 return
+}
+
+zsyscall_aix_ppc64_ggcgo.go
+
+// int asyscall(...)
+
+import "C"
+
+func callasyscall(...) (r1 uintptr, e1 Errno) {
+	 r1 = uintptr(C.asyscall(...))
+	 e1 = syscall.GetErrno()
+	 return
+}
+*/
+
+package main
+
+import (
+	"bufio"
+	"flag"
+	"fmt"
+	"io/ioutil"
+	"os"
+	"regexp"
+	"strings"
+)
+
+var (
+	b32  = flag.Bool("b32", false, "32bit big-endian")
+	l32  = flag.Bool("l32", false, "32bit little-endian")
+	aix  = flag.Bool("aix", false, "aix")
+	tags = flag.String("tags", "", "build tags")
+)
+
+// cmdLine returns this programs's commandline arguments
+func cmdLine() string {
+	return "go run mksyscall_aix_ppc64.go " + strings.Join(os.Args[1:], " ")
+}
+
+// buildTags returns build tags
+func buildTags() string {
+	return *tags
+}
+
+// Param is function parameter
+type Param struct {
+	Name string
+	Type string
+}
+
+// usage prints the program usage
+func usage() {
+	fmt.Fprintf(os.Stderr, "usage: go run mksyscall_aix_ppc64.go [-b32 | -l32] [-tags x,y] [file ...]\n")
+	os.Exit(1)
+}
+
+// parseParamList parses parameter list and returns a slice of parameters
+func parseParamList(list string) []string {
+	list = strings.TrimSpace(list)
+	if list == "" {
+		return []string{}
+	}
+	return regexp.MustCompile(`\s*,\s*`).Split(list, -1)
+}
+
+// parseParam splits a parameter into name and type
+func parseParam(p string) Param {
+	ps := regexp.MustCompile(`^(\S*) (\S*)$`).FindStringSubmatch(p)
+	if ps == nil {
+		fmt.Fprintf(os.Stderr, "malformed parameter: %s\n", p)
+		os.Exit(1)
+	}
+	return Param{ps[1], ps[2]}
+}
+
+func main() {
+	flag.Usage = usage
+	flag.Parse()
+	if len(flag.Args()) <= 0 {
+		fmt.Fprintf(os.Stderr, "no files to parse provided\n")
+		usage()
+	}
+
+	endianness := ""
+	if *b32 {
+		endianness = "big-endian"
+	} else if *l32 {
+		endianness = "little-endian"
+	}
+
+	pack := ""
+	// GCCGO
+	textgccgo := ""
+	cExtern := "/*\n#include <stdint.h>\n"
+	// GC
+	textgc := ""
+	dynimports := ""
+	linknames := ""
+	var vars []string
+	// COMMON
+	textcommon := ""
+	for _, path := range flag.Args() {
+		file, err := os.Open(path)
+		if err != nil {
+			fmt.Fprintf(os.Stderr, err.Error())
+			os.Exit(1)
+		}
+		s := bufio.NewScanner(file)
+		for s.Scan() {
+			t := s.Text()
+			t = strings.TrimSpace(t)
+			t = regexp.MustCompile(`\s+`).ReplaceAllString(t, ` `)
+			if p := regexp.MustCompile(`^package (\S+)$`).FindStringSubmatch(t); p != nil && pack == "" {
+				pack = p[1]
+			}
+			nonblock := regexp.MustCompile(`^\/\/sysnb `).FindStringSubmatch(t)
+			if regexp.MustCompile(`^\/\/sys `).FindStringSubmatch(t) == nil && nonblock == nil {
+				continue
+			}
+
+			// Line must be of the form
+			//	func Open(path string, mode int, perm int) (fd int, err error)
+			// Split into name, in params, out params.
+			f := regexp.MustCompile(`^\/\/sys(nb)? (\w+)\(([^()]*)\)\s*(?:\(([^()]+)\))?\s*(?:=\s*(?:(\w*)\.)?(\w*))?$`).FindStringSubmatch(t)
+			if f == nil {
+				fmt.Fprintf(os.Stderr, "%s:%s\nmalformed //sys declaration\n", path, t)
+				os.Exit(1)
+			}
+			funct, inps, outps, modname, sysname := f[2], f[3], f[4], f[5], f[6]
+
+			// Split argument lists on comma.
+			in := parseParamList(inps)
+			out := parseParamList(outps)
+
+			inps = strings.Join(in, ", ")
+			outps = strings.Join(out, ", ")
+
+			if sysname == "" {
+				sysname = funct
+			}
+
+			onlyCommon := false
+			if funct == "readlen" || funct == "writelen" || funct == "FcntlInt" || funct == "FcntlFlock" {
+				// This function call another syscall which is already implemented.
+				// Therefore, the gc and gccgo part must not be generated.
+				onlyCommon = true
+			}
+
+			// Try in vain to keep people from editing this file.
+			// The theory is that they jump into the middle of the file
+			// without reading the header.
+
+			textcommon += "// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT\n\n"
+			if !onlyCommon {
+				textgccgo += "// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT\n\n"
+				textgc += "// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT\n\n"
+			}
+
+			// Check if value return, err return available
+			errvar := ""
+			rettype := ""
+			for _, param := range out {
+				p := parseParam(param)
+				if p.Type == "error" {
+					errvar = p.Name
+				} else {
+					rettype = p.Type
+				}
+			}
+
+			sysname = regexp.MustCompile(`([a-z])([A-Z])`).ReplaceAllString(sysname, `${1}_$2`)
+			sysname = strings.ToLower(sysname) // All libc functions are lowercase.
+
+			// GCCGO Prototype return type
+			cRettype := ""
+			if rettype == "unsafe.Pointer" {
+				cRettype = "uintptr_t"
+			} else if rettype == "uintptr" {
+				cRettype = "uintptr_t"
+			} else if regexp.MustCompile(`^_`).FindStringSubmatch(rettype) != nil {
+				cRettype = "uintptr_t"
+			} else if rettype == "int" {
+				cRettype = "int"
+			} else if rettype == "int32" {
+				cRettype = "int"
+			} else if rettype == "int64" {
+				cRettype = "long long"
+			} else if rettype == "uint32" {
+				cRettype = "unsigned int"
+			} else if rettype == "uint64" {
+				cRettype = "unsigned long long"
+			} else {
+				cRettype = "int"
+			}
+			if sysname == "exit" {
+				cRettype = "void"
+			}
+
+			// GCCGO Prototype arguments type
+			var cIn []string
+			for i, param := range in {
+				p := parseParam(param)
+				if regexp.MustCompile(`^\*`).FindStringSubmatch(p.Type) != nil {
+					cIn = append(cIn, "uintptr_t")
+				} else if p.Type == "string" {
+					cIn = append(cIn, "uintptr_t")
+				} else if regexp.MustCompile(`^\[\](.*)`).FindStringSubmatch(p.Type) != nil {
+					cIn = append(cIn, "uintptr_t", "size_t")
+				} else if p.Type == "unsafe.Pointer" {
+					cIn = append(cIn, "uintptr_t")
+				} else if p.Type == "uintptr" {
+					cIn = append(cIn, "uintptr_t")
+				} else if regexp.MustCompile(`^_`).FindStringSubmatch(p.Type) != nil {
+					cIn = append(cIn, "uintptr_t")
+				} else if p.Type == "int" {
+					if (i == 0 || i == 2) && funct == "fcntl" {
+						// These fcntl arguments needs to be uintptr to be able to call FcntlInt and FcntlFlock
+						cIn = append(cIn, "uintptr_t")
+					} else {
+						cIn = append(cIn, "int")
+					}
+
+				} else if p.Type == "int32" {
+					cIn = append(cIn, "int")
+				} else if p.Type == "int64" {
+					cIn = append(cIn, "long long")
+				} else if p.Type == "uint32" {
+					cIn = append(cIn, "unsigned int")
+				} else if p.Type == "uint64" {
+					cIn = append(cIn, "unsigned long long")
+				} else {
+					cIn = append(cIn, "int")
+				}
+			}
+
+			if !onlyCommon {
+				// GCCGO Prototype Generation
+				// Imports of system calls from libc
+				if sysname == "select" {
+					// select is a keyword of Go. Its name is
+					// changed to c_select.
+					cExtern += "#define c_select select\n"
+				}
+				cExtern += fmt.Sprintf("%s %s", cRettype, sysname)
+				cIn := strings.Join(cIn, ", ")
+				cExtern += fmt.Sprintf("(%s);\n", cIn)
+			}
+			// GC Library name
+			if modname == "" {
+				modname = "libc.a/shr_64.o"
+			} else {
+				fmt.Fprintf(os.Stderr, "%s: only syscall using libc are available\n", funct)
+				os.Exit(1)
+			}
+			sysvarname := fmt.Sprintf("libc_%s", sysname)
+
+			if !onlyCommon {
+				// GC Runtime import of function to allow cross-platform builds.
+				dynimports += fmt.Sprintf("//go:cgo_import_dynamic %s %s \"%s\"\n", sysvarname, sysname, modname)
+				// GC Link symbol to proc address variable.
+				linknames += fmt.Sprintf("//go:linkname %s %s\n", sysvarname, sysvarname)
+				// GC Library proc address variable.
+				vars = append(vars, sysvarname)
+			}
+
+			strconvfunc := "BytePtrFromString"
+			strconvtype := "*byte"
+
+			// Go function header.
+			if outps != "" {
+				outps = fmt.Sprintf(" (%s)", outps)
+			}
+			if textcommon != "" {
+				textcommon += "\n"
+			}
+
+			textcommon += fmt.Sprintf("func %s(%s)%s {\n", funct, strings.Join(in, ", "), outps)
+
+			// Prepare arguments tocall.
+			var argscommon []string // Arguments in the common part
+			var argscall []string   // Arguments for call prototype
+			var argsgc []string     // Arguments for gc call (with syscall6)
+			var argsgccgo []string  // Arguments for gccgo call (with C.name_of_syscall)
+			n := 0
+			argN := 0
+			for _, param := range in {
+				p := parseParam(param)
+				if regexp.MustCompile(`^\*`).FindStringSubmatch(p.Type) != nil {
+					argscommon = append(argscommon, fmt.Sprintf("uintptr(unsafe.Pointer(%s))", p.Name))
+					argscall = append(argscall, fmt.Sprintf("%s uintptr", p.Name))
+					argsgc = append(argsgc, p.Name)
+					argsgccgo = append(argsgccgo, fmt.Sprintf("C.uintptr_t(%s)", p.Name))
+				} else if p.Type == "string" && errvar != "" {
+					textcommon += fmt.Sprintf("\tvar _p%d %s\n", n, strconvtype)
+					textcommon += fmt.Sprintf("\t_p%d, %s = %s(%s)\n", n, errvar, strconvfunc, p.Name)
+					textcommon += fmt.Sprintf("\tif %s != nil {\n\t\treturn\n\t}\n", errvar)
+
+					argscommon = append(argscommon, fmt.Sprintf("uintptr(unsafe.Pointer(_p%d))", n))
+					argscall = append(argscall, fmt.Sprintf("_p%d uintptr ", n))
+					argsgc = append(argsgc, fmt.Sprintf("_p%d", n))
+					argsgccgo = append(argsgccgo, fmt.Sprintf("C.uintptr_t(_p%d)", n))
+					n++
+				} else if p.Type == "string" {
+					fmt.Fprintf(os.Stderr, path+":"+funct+" uses string arguments, but has no error return\n")
+					textcommon += fmt.Sprintf("\tvar _p%d %s\n", n, strconvtype)
+					textcommon += fmt.Sprintf("\t_p%d, %s = %s(%s)\n", n, errvar, strconvfunc, p.Name)
+					textcommon += fmt.Sprintf("\tif %s != nil {\n\t\treturn\n\t}\n", errvar)
+
+					argscommon = append(argscommon, fmt.Sprintf("uintptr(unsafe.Pointer(_p%d))", n))
+					argscall = append(argscall, fmt.Sprintf("_p%d uintptr", n))
+					argsgc = append(argsgc, fmt.Sprintf("_p%d", n))
+					argsgccgo = append(argsgccgo, fmt.Sprintf("C.uintptr_t(_p%d)", n))
+					n++
+				} else if m := regexp.MustCompile(`^\[\](.*)`).FindStringSubmatch(p.Type); m != nil {
+					// Convert slice into pointer, length.
+					// Have to be careful not to take address of &a[0] if len == 0:
+					// pass nil in that case.
+					textcommon += fmt.Sprintf("\tvar _p%d *%s\n", n, m[1])
+					textcommon += fmt.Sprintf("\tif len(%s) > 0 {\n\t\t_p%d = &%s[0]\n\t}\n", p.Name, n, p.Name)
+					argscommon = append(argscommon, fmt.Sprintf("uintptr(unsafe.Pointer(_p%d))", n), fmt.Sprintf("len(%s)", p.Name))
+					argscall = append(argscall, fmt.Sprintf("_p%d uintptr", n), fmt.Sprintf("_lenp%d int", n))
+					argsgc = append(argsgc, fmt.Sprintf("_p%d", n), fmt.Sprintf("uintptr(_lenp%d)", n))
+					argsgccgo = append(argsgccgo, fmt.Sprintf("C.uintptr_t(_p%d)", n), fmt.Sprintf("C.size_t(_lenp%d)", n))
+					n++
+				} else if p.Type == "int64" && endianness != "" {
+					fmt.Fprintf(os.Stderr, path+":"+funct+" uses int64 with 32 bits mode. Case not yet implemented\n")
+				} else if p.Type == "bool" {
+					fmt.Fprintf(os.Stderr, path+":"+funct+" uses bool. Case not yet implemented\n")
+				} else if regexp.MustCompile(`^_`).FindStringSubmatch(p.Type) != nil || p.Type == "unsafe.Pointer" {
+					argscommon = append(argscommon, fmt.Sprintf("uintptr(%s)", p.Name))
+					argscall = append(argscall, fmt.Sprintf("%s uintptr", p.Name))
+					argsgc = append(argsgc, p.Name)
+					argsgccgo = append(argsgccgo, fmt.Sprintf("C.uintptr_t(%s)", p.Name))
+				} else if p.Type == "int" {
+					if (argN == 0 || argN == 2) && ((funct == "fcntl") || (funct == "FcntlInt") || (funct == "FcntlFlock")) {
+						// These fcntl arguments need to be uintptr to be able to call FcntlInt and FcntlFlock
+						argscommon = append(argscommon, fmt.Sprintf("uintptr(%s)", p.Name))
+						argscall = append(argscall, fmt.Sprintf("%s uintptr", p.Name))
+						argsgc = append(argsgc, p.Name)
+						argsgccgo = append(argsgccgo, fmt.Sprintf("C.uintptr_t(%s)", p.Name))
+
+					} else {
+						argscommon = append(argscommon, p.Name)
+						argscall = append(argscall, fmt.Sprintf("%s int", p.Name))
+						argsgc = append(argsgc, fmt.Sprintf("uintptr(%s)", p.Name))
+						argsgccgo = append(argsgccgo, fmt.Sprintf("C.int(%s)", p.Name))
+					}
+				} else if p.Type == "int32" {
+					argscommon = append(argscommon, p.Name)
+					argscall = append(argscall, fmt.Sprintf("%s int32", p.Name))
+					argsgc = append(argsgc, fmt.Sprintf("uintptr(%s)", p.Name))
+					argsgccgo = append(argsgccgo, fmt.Sprintf("C.int(%s)", p.Name))
+				} else if p.Type == "int64" {
+					argscommon = append(argscommon, p.Name)
+					argscall = append(argscall, fmt.Sprintf("%s int64", p.Name))
+					argsgc = append(argsgc, fmt.Sprintf("uintptr(%s)", p.Name))
+					argsgccgo = append(argsgccgo, fmt.Sprintf("C.longlong(%s)", p.Name))
+				} else if p.Type == "uint32" {
+					argscommon = append(argscommon, p.Name)
+					argscall = append(argscall, fmt.Sprintf("%s uint32", p.Name))
+					argsgc = append(argsgc, fmt.Sprintf("uintptr(%s)", p.Name))
+					argsgccgo = append(argsgccgo, fmt.Sprintf("C.uint(%s)", p.Name))
+				} else if p.Type == "uint64" {
+					argscommon = append(argscommon, p.Name)
+					argscall = append(argscall, fmt.Sprintf("%s uint64", p.Name))
+					argsgc = append(argsgc, fmt.Sprintf("uintptr(%s)", p.Name))
+					argsgccgo = append(argsgccgo, fmt.Sprintf("C.ulonglong(%s)", p.Name))
+				} else if p.Type == "uintptr" {
+					argscommon = append(argscommon, p.Name)
+					argscall = append(argscall, fmt.Sprintf("%s uintptr", p.Name))
+					argsgc = append(argsgc, p.Name)
+					argsgccgo = append(argsgccgo, fmt.Sprintf("C.uintptr_t(%s)", p.Name))
+				} else {
+					argscommon = append(argscommon, fmt.Sprintf("int(%s)", p.Name))
+					argscall = append(argscall, fmt.Sprintf("%s int", p.Name))
+					argsgc = append(argsgc, fmt.Sprintf("uintptr(%s)", p.Name))
+					argsgccgo = append(argsgccgo, fmt.Sprintf("C.int(%s)", p.Name))
+				}
+				argN++
+			}
+			nargs := len(argsgc)
+
+			// COMMON function generation
+			argscommonlist := strings.Join(argscommon, ", ")
+			callcommon := fmt.Sprintf("call%s(%s)", sysname, argscommonlist)
+			ret := []string{"_", "_"}
+			body := ""
+			doErrno := false
+			for i := 0; i < len(out); i++ {
+				p := parseParam(out[i])
+				reg := ""
+				if p.Name == "err" {
+					reg = "e1"
+					ret[1] = reg
+					doErrno = true
+				} else {
+					reg = "r0"
+					ret[0] = reg
+				}
+				if p.Type == "bool" {
+					reg = fmt.Sprintf("%s != 0", reg)
+				}
+				if reg != "e1" {
+					body += fmt.Sprintf("\t%s = %s(%s)\n", p.Name, p.Type, reg)
+				}
+			}
+			if ret[0] == "_" && ret[1] == "_" {
+				textcommon += fmt.Sprintf("\t%s\n", callcommon)
+			} else {
+				textcommon += fmt.Sprintf("\t%s, %s := %s\n", ret[0], ret[1], callcommon)
+			}
+			textcommon += body
+
+			if doErrno {
+				textcommon += "\tif e1 != 0 {\n"
+				textcommon += "\t\terr = errnoErr(e1)\n"
+				textcommon += "\t}\n"
+			}
+			textcommon += "\treturn\n"
+			textcommon += "}\n"
+
+			if onlyCommon {
+				continue
+			}
+
+			// CALL Prototype
+			callProto := fmt.Sprintf("func call%s(%s) (r1 uintptr, e1 Errno) {\n", sysname, strings.Join(argscall, ", "))
+
+			// GC function generation
+			asm := "syscall6"
+			if nonblock != nil {
+				asm = "rawSyscall6"
+			}
+
+			if len(argsgc) <= 6 {
+				for len(argsgc) < 6 {
+					argsgc = append(argsgc, "0")
+				}
+			} else {
+				fmt.Fprintf(os.Stderr, "%s: too many arguments to system call", funct)
+				os.Exit(1)
+			}
+			argsgclist := strings.Join(argsgc, ", ")
+			callgc := fmt.Sprintf("%s(uintptr(unsafe.Pointer(&%s)), %d, %s)", asm, sysvarname, nargs, argsgclist)
+
+			textgc += callProto
+			textgc += fmt.Sprintf("\tr1, _, e1 = %s\n", callgc)
+			textgc += "\treturn\n}\n"
+
+			// GCCGO function generation
+			argsgccgolist := strings.Join(argsgccgo, ", ")
+			var callgccgo string
+			if sysname == "select" {
+				// select is a keyword of Go. Its name is
+				// changed to c_select.
+				callgccgo = fmt.Sprintf("C.c_%s(%s)", sysname, argsgccgolist)
+			} else {
+				callgccgo = fmt.Sprintf("C.%s(%s)", sysname, argsgccgolist)
+			}
+			textgccgo += callProto
+			textgccgo += fmt.Sprintf("\tr1 = uintptr(%s)\n", callgccgo)
+			textgccgo += "\te1 = syscall.GetErrno()\n"
+			textgccgo += "\treturn\n}\n"
+		}
+		if err := s.Err(); err != nil {
+			fmt.Fprintf(os.Stderr, err.Error())
+			os.Exit(1)
+		}
+		file.Close()
+	}
+	imp := ""
+	if pack != "unix" {
+		imp = "import \"golang.org/x/sys/unix\"\n"
+
+	}
+
+	// Print zsyscall_aix_ppc64.go
+	err := ioutil.WriteFile("zsyscall_aix_ppc64.go",
+		[]byte(fmt.Sprintf(srcTemplate1, cmdLine(), buildTags(), pack, imp, textcommon)),
+		0644)
+	if err != nil {
+		fmt.Fprintf(os.Stderr, err.Error())
+		os.Exit(1)
+	}
+
+	// Print zsyscall_aix_ppc64_gc.go
+	vardecls := "\t" + strings.Join(vars, ",\n\t")
+	vardecls += " syscallFunc"
+	err = ioutil.WriteFile("zsyscall_aix_ppc64_gc.go",
+		[]byte(fmt.Sprintf(srcTemplate2, cmdLine(), buildTags(), pack, imp, dynimports, linknames, vardecls, textgc)),
+		0644)
+	if err != nil {
+		fmt.Fprintf(os.Stderr, err.Error())
+		os.Exit(1)
+	}
+
+	// Print zsyscall_aix_ppc64_gccgo.go
+	err = ioutil.WriteFile("zsyscall_aix_ppc64_gccgo.go",
+		[]byte(fmt.Sprintf(srcTemplate3, cmdLine(), buildTags(), pack, cExtern, imp, textgccgo)),
+		0644)
+	if err != nil {
+		fmt.Fprintf(os.Stderr, err.Error())
+		os.Exit(1)
+	}
+}
+
+const srcTemplate1 = `// %s
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+// +build %s
+
+package %s
+
+import (
+	"unsafe"
+)
+
+
+%s
+
+%s
+`
+const srcTemplate2 = `// %s
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+// +build %s
+// +build !gccgo
+
+package %s
+
+import (
+	"unsafe"
+)
+%s
+%s
+%s
+type syscallFunc uintptr
+
+var (
+%s
+)
+
+// Implemented in runtime/syscall_aix.go.
+func rawSyscall6(trap, nargs, a1, a2, a3, a4, a5, a6 uintptr) (r1, r2 uintptr, err Errno)
+func syscall6(trap, nargs, a1, a2, a3, a4, a5, a6 uintptr) (r1, r2 uintptr, err Errno)
+
+%s
+`
+const srcTemplate3 = `// %s
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+// +build %s
+// +build gccgo
+
+package %s
+
+%s
+*/
+import "C"
+import (
+	"syscall"
+)
+
+
+%s
+
+%s
+`
diff --git a/vendor/golang.org/x/sys/unix/mksyscall_solaris.go b/vendor/golang.org/x/sys/unix/mksyscall_solaris.go
new file mode 100644
index 0000000..3d86473
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/mksyscall_solaris.go
@@ -0,0 +1,335 @@
+// 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 ignore
+
+/*
+ This program reads a file containing function prototypes
+ (like syscall_solaris.go) and generates system call bodies.
+ The prototypes are marked by lines beginning with "//sys"
+ and read like func declarations if //sys is replaced by func, but:
+	* The parameter lists must give a name for each argument.
+	  This includes return parameters.
+	* The parameter lists must give a type for each argument:
+	  the (x, y, z int) shorthand is not allowed.
+	* If the return parameter is an error number, it must be named err.
+	* If go func name needs to be different than its libc name,
+	* or the function is not in libc, name could be specified
+	* at the end, after "=" sign, like
+	  //sys getsockopt(s int, level int, name int, val uintptr, vallen *_Socklen) (err error) = libsocket.getsockopt
+*/
+
+package main
+
+import (
+	"bufio"
+	"flag"
+	"fmt"
+	"os"
+	"regexp"
+	"strings"
+)
+
+var (
+	b32  = flag.Bool("b32", false, "32bit big-endian")
+	l32  = flag.Bool("l32", false, "32bit little-endian")
+	tags = flag.String("tags", "", "build tags")
+)
+
+// cmdLine returns this programs's commandline arguments
+func cmdLine() string {
+	return "go run mksyscall_solaris.go " + strings.Join(os.Args[1:], " ")
+}
+
+// buildTags returns build tags
+func buildTags() string {
+	return *tags
+}
+
+// Param is function parameter
+type Param struct {
+	Name string
+	Type string
+}
+
+// usage prints the program usage
+func usage() {
+	fmt.Fprintf(os.Stderr, "usage: go run mksyscall_solaris.go [-b32 | -l32] [-tags x,y] [file ...]\n")
+	os.Exit(1)
+}
+
+// parseParamList parses parameter list and returns a slice of parameters
+func parseParamList(list string) []string {
+	list = strings.TrimSpace(list)
+	if list == "" {
+		return []string{}
+	}
+	return regexp.MustCompile(`\s*,\s*`).Split(list, -1)
+}
+
+// parseParam splits a parameter into name and type
+func parseParam(p string) Param {
+	ps := regexp.MustCompile(`^(\S*) (\S*)$`).FindStringSubmatch(p)
+	if ps == nil {
+		fmt.Fprintf(os.Stderr, "malformed parameter: %s\n", p)
+		os.Exit(1)
+	}
+	return Param{ps[1], ps[2]}
+}
+
+func main() {
+	flag.Usage = usage
+	flag.Parse()
+	if len(flag.Args()) <= 0 {
+		fmt.Fprintf(os.Stderr, "no files to parse provided\n")
+		usage()
+	}
+
+	endianness := ""
+	if *b32 {
+		endianness = "big-endian"
+	} else if *l32 {
+		endianness = "little-endian"
+	}
+
+	pack := ""
+	text := ""
+	dynimports := ""
+	linknames := ""
+	var vars []string
+	for _, path := range flag.Args() {
+		file, err := os.Open(path)
+		if err != nil {
+			fmt.Fprintf(os.Stderr, err.Error())
+			os.Exit(1)
+		}
+		s := bufio.NewScanner(file)
+		for s.Scan() {
+			t := s.Text()
+			t = strings.TrimSpace(t)
+			t = regexp.MustCompile(`\s+`).ReplaceAllString(t, ` `)
+			if p := regexp.MustCompile(`^package (\S+)$`).FindStringSubmatch(t); p != nil && pack == "" {
+				pack = p[1]
+			}
+			nonblock := regexp.MustCompile(`^\/\/sysnb `).FindStringSubmatch(t)
+			if regexp.MustCompile(`^\/\/sys `).FindStringSubmatch(t) == nil && nonblock == nil {
+				continue
+			}
+
+			// Line must be of the form
+			//	func Open(path string, mode int, perm int) (fd int, err error)
+			// Split into name, in params, out params.
+			f := regexp.MustCompile(`^\/\/sys(nb)? (\w+)\(([^()]*)\)\s*(?:\(([^()]+)\))?\s*(?:=\s*(?:(\w*)\.)?(\w*))?$`).FindStringSubmatch(t)
+			if f == nil {
+				fmt.Fprintf(os.Stderr, "%s:%s\nmalformed //sys declaration\n", path, t)
+				os.Exit(1)
+			}
+			funct, inps, outps, modname, sysname := f[2], f[3], f[4], f[5], f[6]
+
+			// Split argument lists on comma.
+			in := parseParamList(inps)
+			out := parseParamList(outps)
+
+			inps = strings.Join(in, ", ")
+			outps = strings.Join(out, ", ")
+
+			// Try in vain to keep people from editing this file.
+			// The theory is that they jump into the middle of the file
+			// without reading the header.
+			text += "// THIS FILE IS GENERATED BY THE COMMAND AT THE TOP; DO NOT EDIT\n\n"
+
+			// So file name.
+			if modname == "" {
+				modname = "libc"
+			}
+
+			// System call name.
+			if sysname == "" {
+				sysname = funct
+			}
+
+			// System call pointer variable name.
+			sysvarname := fmt.Sprintf("proc%s", sysname)
+
+			strconvfunc := "BytePtrFromString"
+			strconvtype := "*byte"
+
+			sysname = strings.ToLower(sysname) // All libc functions are lowercase.
+
+			// Runtime import of function to allow cross-platform builds.
+			dynimports += fmt.Sprintf("//go:cgo_import_dynamic libc_%s %s \"%s.so\"\n", sysname, sysname, modname)
+			// Link symbol to proc address variable.
+			linknames += fmt.Sprintf("//go:linkname %s libc_%s\n", sysvarname, sysname)
+			// Library proc address variable.
+			vars = append(vars, sysvarname)
+
+			// Go function header.
+			outlist := strings.Join(out, ", ")
+			if outlist != "" {
+				outlist = fmt.Sprintf(" (%s)", outlist)
+			}
+			if text != "" {
+				text += "\n"
+			}
+			text += fmt.Sprintf("func %s(%s)%s {\n", funct, strings.Join(in, ", "), outlist)
+
+			// Check if err return available
+			errvar := ""
+			for _, param := range out {
+				p := parseParam(param)
+				if p.Type == "error" {
+					errvar = p.Name
+					continue
+				}
+			}
+
+			// Prepare arguments to Syscall.
+			var args []string
+			n := 0
+			for _, param := range in {
+				p := parseParam(param)
+				if regexp.MustCompile(`^\*`).FindStringSubmatch(p.Type) != nil {
+					args = append(args, "uintptr(unsafe.Pointer("+p.Name+"))")
+				} else if p.Type == "string" && errvar != "" {
+					text += fmt.Sprintf("\tvar _p%d %s\n", n, strconvtype)
+					text += fmt.Sprintf("\t_p%d, %s = %s(%s)\n", n, errvar, strconvfunc, p.Name)
+					text += fmt.Sprintf("\tif %s != nil {\n\t\treturn\n\t}\n", errvar)
+					args = append(args, fmt.Sprintf("uintptr(unsafe.Pointer(_p%d))", n))
+					n++
+				} else if p.Type == "string" {
+					fmt.Fprintf(os.Stderr, path+":"+funct+" uses string arguments, but has no error return\n")
+					text += fmt.Sprintf("\tvar _p%d %s\n", n, strconvtype)
+					text += fmt.Sprintf("\t_p%d, _ = %s(%s)\n", n, strconvfunc, p.Name)
+					args = append(args, fmt.Sprintf("uintptr(unsafe.Pointer(_p%d))", n))
+					n++
+				} else if s := regexp.MustCompile(`^\[\](.*)`).FindStringSubmatch(p.Type); s != nil {
+					// Convert slice into pointer, length.
+					// Have to be careful not to take address of &a[0] if len == 0:
+					// pass nil in that case.
+					text += fmt.Sprintf("\tvar _p%d *%s\n", n, s[1])
+					text += fmt.Sprintf("\tif len(%s) > 0 {\n\t\t_p%d = &%s[0]\n\t}\n", p.Name, n, p.Name)
+					args = append(args, fmt.Sprintf("uintptr(unsafe.Pointer(_p%d))", n), fmt.Sprintf("uintptr(len(%s))", p.Name))
+					n++
+				} else if p.Type == "int64" && endianness != "" {
+					if endianness == "big-endian" {
+						args = append(args, fmt.Sprintf("uintptr(%s>>32)", p.Name), fmt.Sprintf("uintptr(%s)", p.Name))
+					} else {
+						args = append(args, fmt.Sprintf("uintptr(%s)", p.Name), fmt.Sprintf("uintptr(%s>>32)", p.Name))
+					}
+				} else if p.Type == "bool" {
+					text += fmt.Sprintf("\tvar _p%d uint32\n", n)
+					text += fmt.Sprintf("\tif %s {\n\t\t_p%d = 1\n\t} else {\n\t\t_p%d = 0\n\t}\n", p.Name, n, n)
+					args = append(args, fmt.Sprintf("uintptr(_p%d)", n))
+					n++
+				} else {
+					args = append(args, fmt.Sprintf("uintptr(%s)", p.Name))
+				}
+			}
+			nargs := len(args)
+
+			// Determine which form to use; pad args with zeros.
+			asm := "sysvicall6"
+			if nonblock != nil {
+				asm = "rawSysvicall6"
+			}
+			if len(args) <= 6 {
+				for len(args) < 6 {
+					args = append(args, "0")
+				}
+			} else {
+				fmt.Fprintf(os.Stderr, "%s: too many arguments to system call\n", path)
+				os.Exit(1)
+			}
+
+			// Actual call.
+			arglist := strings.Join(args, ", ")
+			call := fmt.Sprintf("%s(uintptr(unsafe.Pointer(&%s)), %d, %s)", asm, sysvarname, nargs, arglist)
+
+			// Assign return values.
+			body := ""
+			ret := []string{"_", "_", "_"}
+			doErrno := false
+			for i := 0; i < len(out); i++ {
+				p := parseParam(out[i])
+				reg := ""
+				if p.Name == "err" {
+					reg = "e1"
+					ret[2] = reg
+					doErrno = true
+				} else {
+					reg = fmt.Sprintf("r%d", i)
+					ret[i] = reg
+				}
+				if p.Type == "bool" {
+					reg = fmt.Sprintf("%d != 0", reg)
+				}
+				if p.Type == "int64" && endianness != "" {
+					// 64-bit number in r1:r0 or r0:r1.
+					if i+2 > len(out) {
+						fmt.Fprintf(os.Stderr, "%s: not enough registers for int64 return\n", path)
+						os.Exit(1)
+					}
+					if endianness == "big-endian" {
+						reg = fmt.Sprintf("int64(r%d)<<32 | int64(r%d)", i, i+1)
+					} else {
+						reg = fmt.Sprintf("int64(r%d)<<32 | int64(r%d)", i+1, i)
+					}
+					ret[i] = fmt.Sprintf("r%d", i)
+					ret[i+1] = fmt.Sprintf("r%d", i+1)
+				}
+				if reg != "e1" {
+					body += fmt.Sprintf("\t%s = %s(%s)\n", p.Name, p.Type, reg)
+				}
+			}
+			if ret[0] == "_" && ret[1] == "_" && ret[2] == "_" {
+				text += fmt.Sprintf("\t%s\n", call)
+			} else {
+				text += fmt.Sprintf("\t%s, %s, %s := %s\n", ret[0], ret[1], ret[2], call)
+			}
+			text += body
+
+			if doErrno {
+				text += "\tif e1 != 0 {\n"
+				text += "\t\terr = e1\n"
+				text += "\t}\n"
+			}
+			text += "\treturn\n"
+			text += "}\n"
+		}
+		if err := s.Err(); err != nil {
+			fmt.Fprintf(os.Stderr, err.Error())
+			os.Exit(1)
+		}
+		file.Close()
+	}
+	imp := ""
+	if pack != "unix" {
+		imp = "import \"golang.org/x/sys/unix\"\n"
+
+	}
+	vardecls := "\t" + strings.Join(vars, ",\n\t")
+	vardecls += " syscallFunc"
+	fmt.Printf(srcTemplate, cmdLine(), buildTags(), pack, imp, dynimports, linknames, vardecls, text)
+}
+
+const srcTemplate = `// %s
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+// +build %s
+
+package %s
+
+import (
+	"syscall"
+	"unsafe"
+)
+%s
+%s
+%s
+var (
+%s	
+)
+
+%s
+`
diff --git a/vendor/golang.org/x/sys/unix/mksysctl_openbsd.go b/vendor/golang.org/x/sys/unix/mksysctl_openbsd.go
new file mode 100644
index 0000000..b6b4099
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/mksysctl_openbsd.go
@@ -0,0 +1,355 @@
+// 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 ignore
+
+// Parse the header files for OpenBSD and generate a Go usable sysctl MIB.
+//
+// Build a MIB with each entry being an array containing the level, type and
+// a hash that will contain additional entries if the current entry is a node.
+// We then walk this MIB and create a flattened sysctl name to OID hash.
+
+package main
+
+import (
+	"bufio"
+	"fmt"
+	"os"
+	"path/filepath"
+	"regexp"
+	"sort"
+	"strings"
+)
+
+var (
+	goos, goarch string
+)
+
+// cmdLine returns this programs's commandline arguments.
+func cmdLine() string {
+	return "go run mksysctl_openbsd.go " + strings.Join(os.Args[1:], " ")
+}
+
+// buildTags returns build tags.
+func buildTags() string {
+	return fmt.Sprintf("%s,%s", goarch, goos)
+}
+
+// reMatch performs regular expression match and stores the substring slice to value pointed by m.
+func reMatch(re *regexp.Regexp, str string, m *[]string) bool {
+	*m = re.FindStringSubmatch(str)
+	if *m != nil {
+		return true
+	}
+	return false
+}
+
+type nodeElement struct {
+	n  int
+	t  string
+	pE *map[string]nodeElement
+}
+
+var (
+	debugEnabled bool
+	mib          map[string]nodeElement
+	node         *map[string]nodeElement
+	nodeMap      map[string]string
+	sysCtl       []string
+)
+
+var (
+	ctlNames1RE = regexp.MustCompile(`^#define\s+(CTL_NAMES)\s+{`)
+	ctlNames2RE = regexp.MustCompile(`^#define\s+(CTL_(.*)_NAMES)\s+{`)
+	ctlNames3RE = regexp.MustCompile(`^#define\s+((.*)CTL_NAMES)\s+{`)
+	netInetRE   = regexp.MustCompile(`^netinet/`)
+	netInet6RE  = regexp.MustCompile(`^netinet6/`)
+	netRE       = regexp.MustCompile(`^net/`)
+	bracesRE    = regexp.MustCompile(`{.*}`)
+	ctlTypeRE   = regexp.MustCompile(`{\s+"(\w+)",\s+(CTLTYPE_[A-Z]+)\s+}`)
+	fsNetKernRE = regexp.MustCompile(`^(fs|net|kern)_`)
+)
+
+func debug(s string) {
+	if debugEnabled {
+		fmt.Fprintln(os.Stderr, s)
+	}
+}
+
+// Walk the MIB and build a sysctl name to OID mapping.
+func buildSysctl(pNode *map[string]nodeElement, name string, oid []int) {
+	lNode := pNode // local copy of pointer to node
+	var keys []string
+	for k := range *lNode {
+		keys = append(keys, k)
+	}
+	sort.Strings(keys)
+
+	for _, key := range keys {
+		nodename := name
+		if name != "" {
+			nodename += "."
+		}
+		nodename += key
+
+		nodeoid := append(oid, (*pNode)[key].n)
+
+		if (*pNode)[key].t == `CTLTYPE_NODE` {
+			if _, ok := nodeMap[nodename]; ok {
+				lNode = &mib
+				ctlName := nodeMap[nodename]
+				for _, part := range strings.Split(ctlName, ".") {
+					lNode = ((*lNode)[part]).pE
+				}
+			} else {
+				lNode = (*pNode)[key].pE
+			}
+			buildSysctl(lNode, nodename, nodeoid)
+		} else if (*pNode)[key].t != "" {
+			oidStr := []string{}
+			for j := range nodeoid {
+				oidStr = append(oidStr, fmt.Sprintf("%d", nodeoid[j]))
+			}
+			text := "\t{ \"" + nodename + "\", []_C_int{ " + strings.Join(oidStr, ", ") + " } }, \n"
+			sysCtl = append(sysCtl, text)
+		}
+	}
+}
+
+func main() {
+	// Get the OS (using GOOS_TARGET if it exist)
+	goos = os.Getenv("GOOS_TARGET")
+	if goos == "" {
+		goos = os.Getenv("GOOS")
+	}
+	// Get the architecture (using GOARCH_TARGET if it exists)
+	goarch = os.Getenv("GOARCH_TARGET")
+	if goarch == "" {
+		goarch = os.Getenv("GOARCH")
+	}
+	// Check if GOOS and GOARCH environment variables are defined
+	if goarch == "" || goos == "" {
+		fmt.Fprintf(os.Stderr, "GOARCH or GOOS not defined in environment\n")
+		os.Exit(1)
+	}
+
+	mib = make(map[string]nodeElement)
+	headers := [...]string{
+		`sys/sysctl.h`,
+		`sys/socket.h`,
+		`sys/tty.h`,
+		`sys/malloc.h`,
+		`sys/mount.h`,
+		`sys/namei.h`,
+		`sys/sem.h`,
+		`sys/shm.h`,
+		`sys/vmmeter.h`,
+		`uvm/uvmexp.h`,
+		`uvm/uvm_param.h`,
+		`uvm/uvm_swap_encrypt.h`,
+		`ddb/db_var.h`,
+		`net/if.h`,
+		`net/if_pfsync.h`,
+		`net/pipex.h`,
+		`netinet/in.h`,
+		`netinet/icmp_var.h`,
+		`netinet/igmp_var.h`,
+		`netinet/ip_ah.h`,
+		`netinet/ip_carp.h`,
+		`netinet/ip_divert.h`,
+		`netinet/ip_esp.h`,
+		`netinet/ip_ether.h`,
+		`netinet/ip_gre.h`,
+		`netinet/ip_ipcomp.h`,
+		`netinet/ip_ipip.h`,
+		`netinet/pim_var.h`,
+		`netinet/tcp_var.h`,
+		`netinet/udp_var.h`,
+		`netinet6/in6.h`,
+		`netinet6/ip6_divert.h`,
+		`netinet6/pim6_var.h`,
+		`netinet/icmp6.h`,
+		`netmpls/mpls.h`,
+	}
+
+	ctls := [...]string{
+		`kern`,
+		`vm`,
+		`fs`,
+		`net`,
+		//debug			/* Special handling required */
+		`hw`,
+		//machdep		/* Arch specific */
+		`user`,
+		`ddb`,
+		//vfs			/* Special handling required */
+		`fs.posix`,
+		`kern.forkstat`,
+		`kern.intrcnt`,
+		`kern.malloc`,
+		`kern.nchstats`,
+		`kern.seminfo`,
+		`kern.shminfo`,
+		`kern.timecounter`,
+		`kern.tty`,
+		`kern.watchdog`,
+		`net.bpf`,
+		`net.ifq`,
+		`net.inet`,
+		`net.inet.ah`,
+		`net.inet.carp`,
+		`net.inet.divert`,
+		`net.inet.esp`,
+		`net.inet.etherip`,
+		`net.inet.gre`,
+		`net.inet.icmp`,
+		`net.inet.igmp`,
+		`net.inet.ip`,
+		`net.inet.ip.ifq`,
+		`net.inet.ipcomp`,
+		`net.inet.ipip`,
+		`net.inet.mobileip`,
+		`net.inet.pfsync`,
+		`net.inet.pim`,
+		`net.inet.tcp`,
+		`net.inet.udp`,
+		`net.inet6`,
+		`net.inet6.divert`,
+		`net.inet6.ip6`,
+		`net.inet6.icmp6`,
+		`net.inet6.pim6`,
+		`net.inet6.tcp6`,
+		`net.inet6.udp6`,
+		`net.mpls`,
+		`net.mpls.ifq`,
+		`net.key`,
+		`net.pflow`,
+		`net.pfsync`,
+		`net.pipex`,
+		`net.rt`,
+		`vm.swapencrypt`,
+		//vfsgenctl		/* Special handling required */
+	}
+
+	// Node name "fixups"
+	ctlMap := map[string]string{
+		"ipproto":             "net.inet",
+		"net.inet.ipproto":    "net.inet",
+		"net.inet6.ipv6proto": "net.inet6",
+		"net.inet6.ipv6":      "net.inet6.ip6",
+		"net.inet.icmpv6":     "net.inet6.icmp6",
+		"net.inet6.divert6":   "net.inet6.divert",
+		"net.inet6.tcp6":      "net.inet.tcp",
+		"net.inet6.udp6":      "net.inet.udp",
+		"mpls":                "net.mpls",
+		"swpenc":              "vm.swapencrypt",
+	}
+
+	// Node mappings
+	nodeMap = map[string]string{
+		"net.inet.ip.ifq": "net.ifq",
+		"net.inet.pfsync": "net.pfsync",
+		"net.mpls.ifq":    "net.ifq",
+	}
+
+	mCtls := make(map[string]bool)
+	for _, ctl := range ctls {
+		mCtls[ctl] = true
+	}
+
+	for _, header := range headers {
+		debug("Processing " + header)
+		file, err := os.Open(filepath.Join("/usr/include", header))
+		if err != nil {
+			fmt.Fprintf(os.Stderr, "%v\n", err)
+			os.Exit(1)
+		}
+		s := bufio.NewScanner(file)
+		for s.Scan() {
+			var sub []string
+			if reMatch(ctlNames1RE, s.Text(), &sub) ||
+				reMatch(ctlNames2RE, s.Text(), &sub) ||
+				reMatch(ctlNames3RE, s.Text(), &sub) {
+				if sub[1] == `CTL_NAMES` {
+					// Top level.
+					node = &mib
+				} else {
+					// Node.
+					nodename := strings.ToLower(sub[2])
+					ctlName := ""
+					if reMatch(netInetRE, header, &sub) {
+						ctlName = "net.inet." + nodename
+					} else if reMatch(netInet6RE, header, &sub) {
+						ctlName = "net.inet6." + nodename
+					} else if reMatch(netRE, header, &sub) {
+						ctlName = "net." + nodename
+					} else {
+						ctlName = nodename
+						ctlName = fsNetKernRE.ReplaceAllString(ctlName, `$1.`)
+					}
+
+					if val, ok := ctlMap[ctlName]; ok {
+						ctlName = val
+					}
+					if _, ok := mCtls[ctlName]; !ok {
+						debug("Ignoring " + ctlName + "...")
+						continue
+					}
+
+					// Walk down from the top of the MIB.
+					node = &mib
+					for _, part := range strings.Split(ctlName, ".") {
+						if _, ok := (*node)[part]; !ok {
+							debug("Missing node " + part)
+							(*node)[part] = nodeElement{n: 0, t: "", pE: &map[string]nodeElement{}}
+						}
+						node = (*node)[part].pE
+					}
+				}
+
+				// Populate current node with entries.
+				i := -1
+				for !strings.HasPrefix(s.Text(), "}") {
+					s.Scan()
+					if reMatch(bracesRE, s.Text(), &sub) {
+						i++
+					}
+					if !reMatch(ctlTypeRE, s.Text(), &sub) {
+						continue
+					}
+					(*node)[sub[1]] = nodeElement{n: i, t: sub[2], pE: &map[string]nodeElement{}}
+				}
+			}
+		}
+		err = s.Err()
+		if err != nil {
+			fmt.Fprintf(os.Stderr, "%v\n", err)
+			os.Exit(1)
+		}
+		file.Close()
+	}
+	buildSysctl(&mib, "", []int{})
+
+	sort.Strings(sysCtl)
+	text := strings.Join(sysCtl, "")
+
+	fmt.Printf(srcTemplate, cmdLine(), buildTags(), text)
+}
+
+const srcTemplate = `// %s
+// Code generated by the command above; DO NOT EDIT.
+
+// +build %s
+
+package unix
+
+type mibentry struct {
+	ctlname string
+	ctloid []_C_int
+}
+
+var sysctlMib = []mibentry {
+%s
+}
+`
diff --git a/vendor/golang.org/x/sys/unix/mksysnum.go b/vendor/golang.org/x/sys/unix/mksysnum.go
new file mode 100644
index 0000000..baa6ecd
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/mksysnum.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.
+
+// +build ignore
+
+// Generate system call table for DragonFly, NetBSD,
+// FreeBSD, OpenBSD or Darwin from master list
+// (for example, /usr/src/sys/kern/syscalls.master or
+// sys/syscall.h).
+package main
+
+import (
+	"bufio"
+	"fmt"
+	"io"
+	"io/ioutil"
+	"net/http"
+	"os"
+	"regexp"
+	"strings"
+)
+
+var (
+	goos, goarch string
+)
+
+// cmdLine returns this programs's commandline arguments
+func cmdLine() string {
+	return "go run mksysnum.go " + strings.Join(os.Args[1:], " ")
+}
+
+// buildTags returns build tags
+func buildTags() string {
+	return fmt.Sprintf("%s,%s", goarch, goos)
+}
+
+func checkErr(err error) {
+	if err != nil {
+		fmt.Fprintf(os.Stderr, "%v\n", err)
+		os.Exit(1)
+	}
+}
+
+// source string and substring slice for regexp
+type re struct {
+	str string   // source string
+	sub []string // matched sub-string
+}
+
+// Match performs regular expression match
+func (r *re) Match(exp string) bool {
+	r.sub = regexp.MustCompile(exp).FindStringSubmatch(r.str)
+	if r.sub != nil {
+		return true
+	}
+	return false
+}
+
+// fetchFile fetches a text file from URL
+func fetchFile(URL string) io.Reader {
+	resp, err := http.Get(URL)
+	checkErr(err)
+	defer resp.Body.Close()
+	body, err := ioutil.ReadAll(resp.Body)
+	checkErr(err)
+	return strings.NewReader(string(body))
+}
+
+// readFile reads a text file from path
+func readFile(path string) io.Reader {
+	file, err := os.Open(os.Args[1])
+	checkErr(err)
+	return file
+}
+
+func format(name, num, proto string) string {
+	name = strings.ToUpper(name)
+	// There are multiple entries for enosys and nosys, so comment them out.
+	nm := re{str: name}
+	if nm.Match(`^SYS_E?NOSYS$`) {
+		name = fmt.Sprintf("// %s", name)
+	}
+	if name == `SYS_SYS_EXIT` {
+		name = `SYS_EXIT`
+	}
+	return fmt.Sprintf("	%s = %s;  // %s\n", name, num, proto)
+}
+
+func main() {
+	// Get the OS (using GOOS_TARGET if it exist)
+	goos = os.Getenv("GOOS_TARGET")
+	if goos == "" {
+		goos = os.Getenv("GOOS")
+	}
+	// Get the architecture (using GOARCH_TARGET if it exists)
+	goarch = os.Getenv("GOARCH_TARGET")
+	if goarch == "" {
+		goarch = os.Getenv("GOARCH")
+	}
+	// Check if GOOS and GOARCH environment variables are defined
+	if goarch == "" || goos == "" {
+		fmt.Fprintf(os.Stderr, "GOARCH or GOOS not defined in environment\n")
+		os.Exit(1)
+	}
+
+	file := strings.TrimSpace(os.Args[1])
+	var syscalls io.Reader
+	if strings.HasPrefix(file, "https://") || strings.HasPrefix(file, "http://") {
+		// Download syscalls.master file
+		syscalls = fetchFile(file)
+	} else {
+		syscalls = readFile(file)
+	}
+
+	var text, line string
+	s := bufio.NewScanner(syscalls)
+	for s.Scan() {
+		t := re{str: line}
+		if t.Match(`^(.*)\\$`) {
+			// Handle continuation
+			line = t.sub[1]
+			line += strings.TrimLeft(s.Text(), " \t")
+		} else {
+			// New line
+			line = s.Text()
+		}
+		t = re{str: line}
+		if t.Match(`\\$`) {
+			continue
+		}
+		t = re{str: line}
+
+		switch goos {
+		case "dragonfly":
+			if t.Match(`^([0-9]+)\s+STD\s+({ \S+\s+(\w+).*)$`) {
+				num, proto := t.sub[1], t.sub[2]
+				name := fmt.Sprintf("SYS_%s", t.sub[3])
+				text += format(name, num, proto)
+			}
+		case "freebsd":
+			if t.Match(`^([0-9]+)\s+\S+\s+(?:(?:NO)?STD|COMPAT10)\s+({ \S+\s+(\w+).*)$`) {
+				num, proto := t.sub[1], t.sub[2]
+				name := fmt.Sprintf("SYS_%s", t.sub[3])
+				text += format(name, num, proto)
+			}
+		case "openbsd":
+			if t.Match(`^([0-9]+)\s+STD\s+(NOLOCK\s+)?({ \S+\s+\*?(\w+).*)$`) {
+				num, proto, name := t.sub[1], t.sub[3], t.sub[4]
+				text += format(name, num, proto)
+			}
+		case "netbsd":
+			if t.Match(`^([0-9]+)\s+((STD)|(NOERR))\s+(RUMP\s+)?({\s+\S+\s*\*?\s*\|(\S+)\|(\S*)\|(\w+).*\s+})(\s+(\S+))?$`) {
+				num, proto, compat := t.sub[1], t.sub[6], t.sub[8]
+				name := t.sub[7] + "_" + t.sub[9]
+				if t.sub[11] != "" {
+					name = t.sub[7] + "_" + t.sub[11]
+				}
+				name = strings.ToUpper(name)
+				if compat == "" || compat == "13" || compat == "30" || compat == "50" {
+					text += fmt.Sprintf("	%s = %s;  // %s\n", name, num, proto)
+				}
+			}
+		case "darwin":
+			if t.Match(`^#define\s+SYS_(\w+)\s+([0-9]+)`) {
+				name, num := t.sub[1], t.sub[2]
+				name = strings.ToUpper(name)
+				text += fmt.Sprintf("	SYS_%s = %s;\n", name, num)
+			}
+		default:
+			fmt.Fprintf(os.Stderr, "unrecognized GOOS=%s\n", goos)
+			os.Exit(1)
+
+		}
+	}
+	err := s.Err()
+	checkErr(err)
+
+	fmt.Printf(template, cmdLine(), buildTags(), text)
+}
+
+const template = `// %s
+// Code generated by the command above; see README.md. DO NOT EDIT.
+
+// +build %s
+
+package unix
+
+const(
+%s)`
diff --git a/vendor/golang.org/x/sys/unix/types_aix.go b/vendor/golang.org/x/sys/unix/types_aix.go
new file mode 100644
index 0000000..40d2bee
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/types_aix.go
@@ -0,0 +1,237 @@
+// 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 ignore
+// +build aix
+
+/*
+Input to cgo -godefs.  See also mkerrors.sh and mkall.sh
+*/
+
+// +godefs map struct_in_addr [4]byte /* in_addr */
+// +godefs map struct_in6_addr [16]byte /* in6_addr */
+
+package unix
+
+/*
+#include <sys/types.h>
+#include <sys/time.h>
+#include <sys/limits.h>
+#include <sys/un.h>
+#include <utime.h>
+#include <sys/utsname.h>
+#include <sys/poll.h>
+#include <sys/resource.h>
+#include <sys/stat.h>
+#include <sys/statfs.h>
+#include <sys/termio.h>
+#include <sys/ioctl.h>
+
+#include <termios.h>
+
+#include <net/if.h>
+#include <net/if_dl.h>
+#include <netinet/in.h>
+#include <netinet/icmp6.h>
+
+
+#include <dirent.h>
+#include <fcntl.h>
+
+enum {
+	sizeofPtr = sizeof(void*),
+};
+
+union sockaddr_all {
+	struct sockaddr s1;     // this one gets used for fields
+	struct sockaddr_in s2;  // these pad it out
+	struct sockaddr_in6 s3;
+	struct sockaddr_un s4;
+	struct sockaddr_dl s5;
+};
+
+struct sockaddr_any {
+	struct sockaddr addr;
+	char pad[sizeof(union sockaddr_all) - sizeof(struct sockaddr)];
+};
+
+*/
+import "C"
+
+// Machine characteristics
+
+const (
+	SizeofPtr      = C.sizeofPtr
+	SizeofShort    = C.sizeof_short
+	SizeofInt      = C.sizeof_int
+	SizeofLong     = C.sizeof_long
+	SizeofLongLong = C.sizeof_longlong
+	PathMax        = C.PATH_MAX
+)
+
+// Basic types
+
+type (
+	_C_short     C.short
+	_C_int       C.int
+	_C_long      C.long
+	_C_long_long C.longlong
+)
+
+type off64 C.off64_t
+type off C.off_t
+type Mode_t C.mode_t
+
+// Time
+
+type Timespec C.struct_timespec
+
+type Timeval C.struct_timeval
+
+type Timeval32 C.struct_timeval32
+
+type Timex C.struct_timex
+
+type Time_t C.time_t
+
+type Tms C.struct_tms
+
+type Utimbuf C.struct_utimbuf
+
+type Timezone C.struct_timezone
+
+// Processes
+
+type Rusage C.struct_rusage
+
+type Rlimit C.struct_rlimit64
+
+type Pid_t C.pid_t
+
+type _Gid_t C.gid_t
+
+type dev_t C.dev_t
+
+// Files
+
+type Stat_t C.struct_stat
+
+type StatxTimestamp C.struct_statx_timestamp
+
+type Statx_t C.struct_statx
+
+type Dirent C.struct_dirent
+
+// Sockets
+
+type RawSockaddrInet4 C.struct_sockaddr_in
+
+type RawSockaddrInet6 C.struct_sockaddr_in6
+
+type RawSockaddrUnix C.struct_sockaddr_un
+
+type RawSockaddrDatalink C.struct_sockaddr_dl
+
+type RawSockaddr C.struct_sockaddr
+
+type RawSockaddrAny C.struct_sockaddr_any
+
+type _Socklen C.socklen_t
+
+type Cmsghdr C.struct_cmsghdr
+
+type ICMPv6Filter C.struct_icmp6_filter
+
+type Iovec C.struct_iovec
+
+type IPMreq C.struct_ip_mreq
+
+type IPv6Mreq C.struct_ipv6_mreq
+
+type IPv6MTUInfo C.struct_ip6_mtuinfo
+
+type Linger C.struct_linger
+
+type Msghdr C.struct_msghdr
+
+const (
+	SizeofSockaddrInet4    = C.sizeof_struct_sockaddr_in
+	SizeofSockaddrInet6    = C.sizeof_struct_sockaddr_in6
+	SizeofSockaddrAny      = C.sizeof_struct_sockaddr_any
+	SizeofSockaddrUnix     = C.sizeof_struct_sockaddr_un
+	SizeofSockaddrDatalink = C.sizeof_struct_sockaddr_dl
+	SizeofLinger           = C.sizeof_struct_linger
+	SizeofIPMreq           = C.sizeof_struct_ip_mreq
+	SizeofIPv6Mreq         = C.sizeof_struct_ipv6_mreq
+	SizeofIPv6MTUInfo      = C.sizeof_struct_ip6_mtuinfo
+	SizeofMsghdr           = C.sizeof_struct_msghdr
+	SizeofCmsghdr          = C.sizeof_struct_cmsghdr
+	SizeofICMPv6Filter     = C.sizeof_struct_icmp6_filter
+)
+
+// Routing and interface messages
+
+const (
+	SizeofIfMsghdr = C.sizeof_struct_if_msghdr
+)
+
+type IfMsgHdr C.struct_if_msghdr
+
+// Misc
+
+type FdSet C.fd_set
+
+type Utsname C.struct_utsname
+
+type Ustat_t C.struct_ustat
+
+type Sigset_t C.sigset_t
+
+const (
+	AT_FDCWD            = C.AT_FDCWD
+	AT_REMOVEDIR        = C.AT_REMOVEDIR
+	AT_SYMLINK_NOFOLLOW = C.AT_SYMLINK_NOFOLLOW
+)
+
+// Terminal handling
+
+type Termios C.struct_termios
+
+type Termio C.struct_termio
+
+type Winsize C.struct_winsize
+
+//poll
+
+type PollFd struct {
+	Fd      int32
+	Events  uint16
+	Revents uint16
+}
+
+const (
+	POLLERR    = C.POLLERR
+	POLLHUP    = C.POLLHUP
+	POLLIN     = C.POLLIN
+	POLLNVAL   = C.POLLNVAL
+	POLLOUT    = C.POLLOUT
+	POLLPRI    = C.POLLPRI
+	POLLRDBAND = C.POLLRDBAND
+	POLLRDNORM = C.POLLRDNORM
+	POLLWRBAND = C.POLLWRBAND
+	POLLWRNORM = C.POLLWRNORM
+)
+
+//flock_t
+
+type Flock_t C.struct_flock64
+
+// Statfs
+
+type Fsid_t C.struct_fsid_t
+type Fsid64_t C.struct_fsid64_t
+
+type Statfs_t C.struct_statfs
+
+const RNDGETENTCNT = 0x80045200
diff --git a/vendor/golang.org/x/sys/unix/types_darwin.go b/vendor/golang.org/x/sys/unix/types_darwin.go
new file mode 100644
index 0000000..155c2e6
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/types_darwin.go
@@ -0,0 +1,283 @@
+// 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 ignore
+
+/*
+Input to cgo -godefs.  See README.md
+*/
+
+// +godefs map struct_in_addr [4]byte /* in_addr */
+// +godefs map struct_in6_addr [16]byte /* in6_addr */
+
+package unix
+
+/*
+#define __DARWIN_UNIX03 0
+#define KERNEL
+#define _DARWIN_USE_64_BIT_INODE
+#include <dirent.h>
+#include <fcntl.h>
+#include <poll.h>
+#include <signal.h>
+#include <termios.h>
+#include <unistd.h>
+#include <mach/mach.h>
+#include <mach/message.h>
+#include <sys/event.h>
+#include <sys/mman.h>
+#include <sys/mount.h>
+#include <sys/param.h>
+#include <sys/ptrace.h>
+#include <sys/resource.h>
+#include <sys/select.h>
+#include <sys/signal.h>
+#include <sys/socket.h>
+#include <sys/stat.h>
+#include <sys/time.h>
+#include <sys/types.h>
+#include <sys/uio.h>
+#include <sys/un.h>
+#include <sys/utsname.h>
+#include <sys/wait.h>
+#include <net/bpf.h>
+#include <net/if.h>
+#include <net/if_dl.h>
+#include <net/if_var.h>
+#include <net/route.h>
+#include <netinet/in.h>
+#include <netinet/icmp6.h>
+#include <netinet/tcp.h>
+
+enum {
+	sizeofPtr = sizeof(void*),
+};
+
+union sockaddr_all {
+	struct sockaddr s1;	// this one gets used for fields
+	struct sockaddr_in s2;	// these pad it out
+	struct sockaddr_in6 s3;
+	struct sockaddr_un s4;
+	struct sockaddr_dl s5;
+};
+
+struct sockaddr_any {
+	struct sockaddr addr;
+	char pad[sizeof(union sockaddr_all) - sizeof(struct sockaddr)];
+};
+
+*/
+import "C"
+
+// Machine characteristics
+
+const (
+	SizeofPtr      = C.sizeofPtr
+	SizeofShort    = C.sizeof_short
+	SizeofInt      = C.sizeof_int
+	SizeofLong     = C.sizeof_long
+	SizeofLongLong = C.sizeof_longlong
+)
+
+// Basic types
+
+type (
+	_C_short     C.short
+	_C_int       C.int
+	_C_long      C.long
+	_C_long_long C.longlong
+)
+
+// Time
+
+type Timespec C.struct_timespec
+
+type Timeval C.struct_timeval
+
+type Timeval32 C.struct_timeval32
+
+// Processes
+
+type Rusage C.struct_rusage
+
+type Rlimit C.struct_rlimit
+
+type _Gid_t C.gid_t
+
+// Files
+
+type Stat_t C.struct_stat64
+
+type Statfs_t C.struct_statfs64
+
+type Flock_t C.struct_flock
+
+type Fstore_t C.struct_fstore
+
+type Radvisory_t C.struct_radvisory
+
+type Fbootstraptransfer_t C.struct_fbootstraptransfer
+
+type Log2phys_t C.struct_log2phys
+
+type Fsid C.struct_fsid
+
+type Dirent C.struct_dirent
+
+// Sockets
+
+type RawSockaddrInet4 C.struct_sockaddr_in
+
+type RawSockaddrInet6 C.struct_sockaddr_in6
+
+type RawSockaddrUnix C.struct_sockaddr_un
+
+type RawSockaddrDatalink C.struct_sockaddr_dl
+
+type RawSockaddr C.struct_sockaddr
+
+type RawSockaddrAny C.struct_sockaddr_any
+
+type _Socklen C.socklen_t
+
+type Linger C.struct_linger
+
+type Iovec C.struct_iovec
+
+type IPMreq C.struct_ip_mreq
+
+type IPv6Mreq C.struct_ipv6_mreq
+
+type Msghdr C.struct_msghdr
+
+type Cmsghdr C.struct_cmsghdr
+
+type Inet4Pktinfo C.struct_in_pktinfo
+
+type Inet6Pktinfo C.struct_in6_pktinfo
+
+type IPv6MTUInfo C.struct_ip6_mtuinfo
+
+type ICMPv6Filter C.struct_icmp6_filter
+
+const (
+	SizeofSockaddrInet4    = C.sizeof_struct_sockaddr_in
+	SizeofSockaddrInet6    = C.sizeof_struct_sockaddr_in6
+	SizeofSockaddrAny      = C.sizeof_struct_sockaddr_any
+	SizeofSockaddrUnix     = C.sizeof_struct_sockaddr_un
+	SizeofSockaddrDatalink = C.sizeof_struct_sockaddr_dl
+	SizeofLinger           = C.sizeof_struct_linger
+	SizeofIPMreq           = C.sizeof_struct_ip_mreq
+	SizeofIPv6Mreq         = C.sizeof_struct_ipv6_mreq
+	SizeofMsghdr           = C.sizeof_struct_msghdr
+	SizeofCmsghdr          = C.sizeof_struct_cmsghdr
+	SizeofInet4Pktinfo     = C.sizeof_struct_in_pktinfo
+	SizeofInet6Pktinfo     = C.sizeof_struct_in6_pktinfo
+	SizeofIPv6MTUInfo      = C.sizeof_struct_ip6_mtuinfo
+	SizeofICMPv6Filter     = C.sizeof_struct_icmp6_filter
+)
+
+// Ptrace requests
+
+const (
+	PTRACE_TRACEME = C.PT_TRACE_ME
+	PTRACE_CONT    = C.PT_CONTINUE
+	PTRACE_KILL    = C.PT_KILL
+)
+
+// Events (kqueue, kevent)
+
+type Kevent_t C.struct_kevent
+
+// Select
+
+type FdSet C.fd_set
+
+// Routing and interface messages
+
+const (
+	SizeofIfMsghdr    = C.sizeof_struct_if_msghdr
+	SizeofIfData      = C.sizeof_struct_if_data
+	SizeofIfaMsghdr   = C.sizeof_struct_ifa_msghdr
+	SizeofIfmaMsghdr  = C.sizeof_struct_ifma_msghdr
+	SizeofIfmaMsghdr2 = C.sizeof_struct_ifma_msghdr2
+	SizeofRtMsghdr    = C.sizeof_struct_rt_msghdr
+	SizeofRtMetrics   = C.sizeof_struct_rt_metrics
+)
+
+type IfMsghdr C.struct_if_msghdr
+
+type IfData C.struct_if_data
+
+type IfaMsghdr C.struct_ifa_msghdr
+
+type IfmaMsghdr C.struct_ifma_msghdr
+
+type IfmaMsghdr2 C.struct_ifma_msghdr2
+
+type RtMsghdr C.struct_rt_msghdr
+
+type RtMetrics C.struct_rt_metrics
+
+// Berkeley packet filter
+
+const (
+	SizeofBpfVersion = C.sizeof_struct_bpf_version
+	SizeofBpfStat    = C.sizeof_struct_bpf_stat
+	SizeofBpfProgram = C.sizeof_struct_bpf_program
+	SizeofBpfInsn    = C.sizeof_struct_bpf_insn
+	SizeofBpfHdr     = C.sizeof_struct_bpf_hdr
+)
+
+type BpfVersion C.struct_bpf_version
+
+type BpfStat C.struct_bpf_stat
+
+type BpfProgram C.struct_bpf_program
+
+type BpfInsn C.struct_bpf_insn
+
+type BpfHdr C.struct_bpf_hdr
+
+// Terminal handling
+
+type Termios C.struct_termios
+
+type Winsize C.struct_winsize
+
+// fchmodat-like syscalls.
+
+const (
+	AT_FDCWD            = C.AT_FDCWD
+	AT_REMOVEDIR        = C.AT_REMOVEDIR
+	AT_SYMLINK_FOLLOW   = C.AT_SYMLINK_FOLLOW
+	AT_SYMLINK_NOFOLLOW = C.AT_SYMLINK_NOFOLLOW
+)
+
+// poll
+
+type PollFd C.struct_pollfd
+
+const (
+	POLLERR    = C.POLLERR
+	POLLHUP    = C.POLLHUP
+	POLLIN     = C.POLLIN
+	POLLNVAL   = C.POLLNVAL
+	POLLOUT    = C.POLLOUT
+	POLLPRI    = C.POLLPRI
+	POLLRDBAND = C.POLLRDBAND
+	POLLRDNORM = C.POLLRDNORM
+	POLLWRBAND = C.POLLWRBAND
+	POLLWRNORM = C.POLLWRNORM
+)
+
+// uname
+
+type Utsname C.struct_utsname
+
+// Clockinfo
+
+const SizeofClockinfo = C.sizeof_struct_clockinfo
+
+type Clockinfo C.struct_clockinfo
diff --git a/vendor/golang.org/x/sys/unix/types_dragonfly.go b/vendor/golang.org/x/sys/unix/types_dragonfly.go
new file mode 100644
index 0000000..6574f6b
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/types_dragonfly.go
@@ -0,0 +1,269 @@
+// 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 ignore
+
+/*
+Input to cgo -godefs.  See README.md
+*/
+
+// +godefs map struct_in_addr [4]byte /* in_addr */
+// +godefs map struct_in6_addr [16]byte /* in6_addr */
+
+package unix
+
+/*
+#define KERNEL
+#include <dirent.h>
+#include <fcntl.h>
+#include <poll.h>
+#include <signal.h>
+#include <termios.h>
+#include <stdio.h>
+#include <unistd.h>
+#include <sys/event.h>
+#include <sys/mman.h>
+#include <sys/mount.h>
+#include <sys/param.h>
+#include <sys/ptrace.h>
+#include <sys/resource.h>
+#include <sys/select.h>
+#include <sys/signal.h>
+#include <sys/socket.h>
+#include <sys/stat.h>
+#include <sys/time.h>
+#include <sys/types.h>
+#include <sys/un.h>
+#include <sys/utsname.h>
+#include <sys/wait.h>
+#include <net/bpf.h>
+#include <net/if.h>
+#include <net/if_dl.h>
+#include <net/route.h>
+#include <netinet/in.h>
+#include <netinet/icmp6.h>
+#include <netinet/tcp.h>
+
+enum {
+	sizeofPtr = sizeof(void*),
+};
+
+union sockaddr_all {
+	struct sockaddr s1;	// this one gets used for fields
+	struct sockaddr_in s2;	// these pad it out
+	struct sockaddr_in6 s3;
+	struct sockaddr_un s4;
+	struct sockaddr_dl s5;
+};
+
+struct sockaddr_any {
+	struct sockaddr addr;
+	char pad[sizeof(union sockaddr_all) - sizeof(struct sockaddr)];
+};
+
+*/
+import "C"
+
+// Machine characteristics
+
+const (
+	SizeofPtr      = C.sizeofPtr
+	SizeofShort    = C.sizeof_short
+	SizeofInt      = C.sizeof_int
+	SizeofLong     = C.sizeof_long
+	SizeofLongLong = C.sizeof_longlong
+)
+
+// Basic types
+
+type (
+	_C_short     C.short
+	_C_int       C.int
+	_C_long      C.long
+	_C_long_long C.longlong
+)
+
+// Time
+
+type Timespec C.struct_timespec
+
+type Timeval C.struct_timeval
+
+// Processes
+
+type Rusage C.struct_rusage
+
+type Rlimit C.struct_rlimit
+
+type _Gid_t C.gid_t
+
+// Files
+
+type Stat_t C.struct_stat
+
+type Statfs_t C.struct_statfs
+
+type Flock_t C.struct_flock
+
+type Dirent C.struct_dirent
+
+type Fsid C.struct_fsid
+
+// File system limits
+
+const (
+	PathMax = C.PATH_MAX
+)
+
+// Sockets
+
+type RawSockaddrInet4 C.struct_sockaddr_in
+
+type RawSockaddrInet6 C.struct_sockaddr_in6
+
+type RawSockaddrUnix C.struct_sockaddr_un
+
+type RawSockaddrDatalink C.struct_sockaddr_dl
+
+type RawSockaddr C.struct_sockaddr
+
+type RawSockaddrAny C.struct_sockaddr_any
+
+type _Socklen C.socklen_t
+
+type Linger C.struct_linger
+
+type Iovec C.struct_iovec
+
+type IPMreq C.struct_ip_mreq
+
+type IPv6Mreq C.struct_ipv6_mreq
+
+type Msghdr C.struct_msghdr
+
+type Cmsghdr C.struct_cmsghdr
+
+type Inet6Pktinfo C.struct_in6_pktinfo
+
+type IPv6MTUInfo C.struct_ip6_mtuinfo
+
+type ICMPv6Filter C.struct_icmp6_filter
+
+const (
+	SizeofSockaddrInet4    = C.sizeof_struct_sockaddr_in
+	SizeofSockaddrInet6    = C.sizeof_struct_sockaddr_in6
+	SizeofSockaddrAny      = C.sizeof_struct_sockaddr_any
+	SizeofSockaddrUnix     = C.sizeof_struct_sockaddr_un
+	SizeofSockaddrDatalink = C.sizeof_struct_sockaddr_dl
+	SizeofLinger           = C.sizeof_struct_linger
+	SizeofIPMreq           = C.sizeof_struct_ip_mreq
+	SizeofIPv6Mreq         = C.sizeof_struct_ipv6_mreq
+	SizeofMsghdr           = C.sizeof_struct_msghdr
+	SizeofCmsghdr          = C.sizeof_struct_cmsghdr
+	SizeofInet6Pktinfo     = C.sizeof_struct_in6_pktinfo
+	SizeofIPv6MTUInfo      = C.sizeof_struct_ip6_mtuinfo
+	SizeofICMPv6Filter     = C.sizeof_struct_icmp6_filter
+)
+
+// Ptrace requests
+
+const (
+	PTRACE_TRACEME = C.PT_TRACE_ME
+	PTRACE_CONT    = C.PT_CONTINUE
+	PTRACE_KILL    = C.PT_KILL
+)
+
+// Events (kqueue, kevent)
+
+type Kevent_t C.struct_kevent
+
+// Select
+
+type FdSet C.fd_set
+
+// Routing and interface messages
+
+const (
+	SizeofIfMsghdr         = C.sizeof_struct_if_msghdr
+	SizeofIfData           = C.sizeof_struct_if_data
+	SizeofIfaMsghdr        = C.sizeof_struct_ifa_msghdr
+	SizeofIfmaMsghdr       = C.sizeof_struct_ifma_msghdr
+	SizeofIfAnnounceMsghdr = C.sizeof_struct_if_announcemsghdr
+	SizeofRtMsghdr         = C.sizeof_struct_rt_msghdr
+	SizeofRtMetrics        = C.sizeof_struct_rt_metrics
+)
+
+type IfMsghdr C.struct_if_msghdr
+
+type IfData C.struct_if_data
+
+type IfaMsghdr C.struct_ifa_msghdr
+
+type IfmaMsghdr C.struct_ifma_msghdr
+
+type IfAnnounceMsghdr C.struct_if_announcemsghdr
+
+type RtMsghdr C.struct_rt_msghdr
+
+type RtMetrics C.struct_rt_metrics
+
+// Berkeley packet filter
+
+const (
+	SizeofBpfVersion = C.sizeof_struct_bpf_version
+	SizeofBpfStat    = C.sizeof_struct_bpf_stat
+	SizeofBpfProgram = C.sizeof_struct_bpf_program
+	SizeofBpfInsn    = C.sizeof_struct_bpf_insn
+	SizeofBpfHdr     = C.sizeof_struct_bpf_hdr
+)
+
+type BpfVersion C.struct_bpf_version
+
+type BpfStat C.struct_bpf_stat
+
+type BpfProgram C.struct_bpf_program
+
+type BpfInsn C.struct_bpf_insn
+
+type BpfHdr C.struct_bpf_hdr
+
+// Terminal handling
+
+type Termios C.struct_termios
+
+type Winsize C.struct_winsize
+
+// fchmodat-like syscalls.
+
+const (
+	AT_FDCWD            = C.AT_FDCWD
+	AT_SYMLINK_NOFOLLOW = C.AT_SYMLINK_NOFOLLOW
+)
+
+// poll
+
+type PollFd C.struct_pollfd
+
+const (
+	POLLERR    = C.POLLERR
+	POLLHUP    = C.POLLHUP
+	POLLIN     = C.POLLIN
+	POLLNVAL   = C.POLLNVAL
+	POLLOUT    = C.POLLOUT
+	POLLPRI    = C.POLLPRI
+	POLLRDBAND = C.POLLRDBAND
+	POLLRDNORM = C.POLLRDNORM
+	POLLWRBAND = C.POLLWRBAND
+	POLLWRNORM = C.POLLWRNORM
+)
+
+// Uname
+
+type Utsname C.struct_utsname
+
+// Clockinfo
+
+const SizeofClockinfo = C.sizeof_struct_clockinfo
+
+type Clockinfo C.struct_clockinfo
diff --git a/vendor/golang.org/x/sys/unix/types_freebsd.go b/vendor/golang.org/x/sys/unix/types_freebsd.go
new file mode 100644
index 0000000..c6fde42
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/types_freebsd.go
@@ -0,0 +1,406 @@
+// 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 ignore
+
+/*
+Input to cgo -godefs.  See README.md
+*/
+
+// +godefs map struct_in_addr [4]byte /* in_addr */
+// +godefs map struct_in6_addr [16]byte /* in6_addr */
+
+package unix
+
+/*
+#define	_WANT_FREEBSD11_STAT	1
+#define	_WANT_FREEBSD11_STATFS	1
+#define	_WANT_FREEBSD11_DIRENT	1
+#define	_WANT_FREEBSD11_KEVENT  1
+
+#include <dirent.h>
+#include <fcntl.h>
+#include <poll.h>
+#include <signal.h>
+#include <termios.h>
+#include <stdio.h>
+#include <unistd.h>
+#include <sys/capsicum.h>
+#include <sys/event.h>
+#include <sys/mman.h>
+#include <sys/mount.h>
+#include <sys/param.h>
+#include <sys/ptrace.h>
+#include <sys/resource.h>
+#include <sys/select.h>
+#include <sys/signal.h>
+#include <sys/socket.h>
+#include <sys/stat.h>
+#include <sys/time.h>
+#include <sys/types.h>
+#include <sys/un.h>
+#include <sys/utsname.h>
+#include <sys/wait.h>
+#include <net/bpf.h>
+#include <net/if.h>
+#include <net/if_dl.h>
+#include <net/route.h>
+#include <netinet/in.h>
+#include <netinet/icmp6.h>
+#include <netinet/tcp.h>
+
+enum {
+	sizeofPtr = sizeof(void*),
+};
+
+union sockaddr_all {
+	struct sockaddr s1;	// this one gets used for fields
+	struct sockaddr_in s2;	// these pad it out
+	struct sockaddr_in6 s3;
+	struct sockaddr_un s4;
+	struct sockaddr_dl s5;
+};
+
+struct sockaddr_any {
+	struct sockaddr addr;
+	char pad[sizeof(union sockaddr_all) - sizeof(struct sockaddr)];
+};
+
+// This structure is a duplicate of if_data on FreeBSD 8-STABLE.
+// See /usr/include/net/if.h.
+struct if_data8 {
+	u_char  ifi_type;
+	u_char  ifi_physical;
+	u_char  ifi_addrlen;
+	u_char  ifi_hdrlen;
+	u_char  ifi_link_state;
+	u_char  ifi_spare_char1;
+	u_char  ifi_spare_char2;
+	u_char  ifi_datalen;
+	u_long  ifi_mtu;
+	u_long  ifi_metric;
+	u_long  ifi_baudrate;
+	u_long  ifi_ipackets;
+	u_long  ifi_ierrors;
+	u_long  ifi_opackets;
+	u_long  ifi_oerrors;
+	u_long  ifi_collisions;
+	u_long  ifi_ibytes;
+	u_long  ifi_obytes;
+	u_long  ifi_imcasts;
+	u_long  ifi_omcasts;
+	u_long  ifi_iqdrops;
+	u_long  ifi_noproto;
+	u_long  ifi_hwassist;
+// FIXME: these are now unions, so maybe need to change definitions?
+#undef ifi_epoch
+	time_t  ifi_epoch;
+#undef ifi_lastchange
+	struct  timeval ifi_lastchange;
+};
+
+// This structure is a duplicate of if_msghdr on FreeBSD 8-STABLE.
+// See /usr/include/net/if.h.
+struct if_msghdr8 {
+	u_short ifm_msglen;
+	u_char  ifm_version;
+	u_char  ifm_type;
+	int     ifm_addrs;
+	int     ifm_flags;
+	u_short ifm_index;
+	struct  if_data8 ifm_data;
+};
+*/
+import "C"
+
+// Machine characteristics
+
+const (
+	SizeofPtr      = C.sizeofPtr
+	SizeofShort    = C.sizeof_short
+	SizeofInt      = C.sizeof_int
+	SizeofLong     = C.sizeof_long
+	SizeofLongLong = C.sizeof_longlong
+)
+
+// Basic types
+
+type (
+	_C_short     C.short
+	_C_int       C.int
+	_C_long      C.long
+	_C_long_long C.longlong
+)
+
+// Time
+
+type Timespec C.struct_timespec
+
+type Timeval C.struct_timeval
+
+// Processes
+
+type Rusage C.struct_rusage
+
+type Rlimit C.struct_rlimit
+
+type _Gid_t C.gid_t
+
+// Files
+
+const (
+	_statfsVersion = C.STATFS_VERSION
+	_dirblksiz     = C.DIRBLKSIZ
+)
+
+type Stat_t C.struct_stat
+
+type stat_freebsd11_t C.struct_freebsd11_stat
+
+type Statfs_t C.struct_statfs
+
+type statfs_freebsd11_t C.struct_freebsd11_statfs
+
+type Flock_t C.struct_flock
+
+type Dirent C.struct_dirent
+
+type dirent_freebsd11 C.struct_freebsd11_dirent
+
+type Fsid C.struct_fsid
+
+// File system limits
+
+const (
+	PathMax = C.PATH_MAX
+)
+
+// Advice to Fadvise
+
+const (
+	FADV_NORMAL     = C.POSIX_FADV_NORMAL
+	FADV_RANDOM     = C.POSIX_FADV_RANDOM
+	FADV_SEQUENTIAL = C.POSIX_FADV_SEQUENTIAL
+	FADV_WILLNEED   = C.POSIX_FADV_WILLNEED
+	FADV_DONTNEED   = C.POSIX_FADV_DONTNEED
+	FADV_NOREUSE    = C.POSIX_FADV_NOREUSE
+)
+
+// Sockets
+
+type RawSockaddrInet4 C.struct_sockaddr_in
+
+type RawSockaddrInet6 C.struct_sockaddr_in6
+
+type RawSockaddrUnix C.struct_sockaddr_un
+
+type RawSockaddrDatalink C.struct_sockaddr_dl
+
+type RawSockaddr C.struct_sockaddr
+
+type RawSockaddrAny C.struct_sockaddr_any
+
+type _Socklen C.socklen_t
+
+type Linger C.struct_linger
+
+type Iovec C.struct_iovec
+
+type IPMreq C.struct_ip_mreq
+
+type IPMreqn C.struct_ip_mreqn
+
+type IPv6Mreq C.struct_ipv6_mreq
+
+type Msghdr C.struct_msghdr
+
+type Cmsghdr C.struct_cmsghdr
+
+type Inet6Pktinfo C.struct_in6_pktinfo
+
+type IPv6MTUInfo C.struct_ip6_mtuinfo
+
+type ICMPv6Filter C.struct_icmp6_filter
+
+const (
+	SizeofSockaddrInet4    = C.sizeof_struct_sockaddr_in
+	SizeofSockaddrInet6    = C.sizeof_struct_sockaddr_in6
+	SizeofSockaddrAny      = C.sizeof_struct_sockaddr_any
+	SizeofSockaddrUnix     = C.sizeof_struct_sockaddr_un
+	SizeofSockaddrDatalink = C.sizeof_struct_sockaddr_dl
+	SizeofLinger           = C.sizeof_struct_linger
+	SizeofIPMreq           = C.sizeof_struct_ip_mreq
+	SizeofIPMreqn          = C.sizeof_struct_ip_mreqn
+	SizeofIPv6Mreq         = C.sizeof_struct_ipv6_mreq
+	SizeofMsghdr           = C.sizeof_struct_msghdr
+	SizeofCmsghdr          = C.sizeof_struct_cmsghdr
+	SizeofInet6Pktinfo     = C.sizeof_struct_in6_pktinfo
+	SizeofIPv6MTUInfo      = C.sizeof_struct_ip6_mtuinfo
+	SizeofICMPv6Filter     = C.sizeof_struct_icmp6_filter
+)
+
+// Ptrace requests
+
+const (
+	PTRACE_ATTACH     = C.PT_ATTACH
+	PTRACE_CONT       = C.PT_CONTINUE
+	PTRACE_DETACH     = C.PT_DETACH
+	PTRACE_GETFPREGS  = C.PT_GETFPREGS
+	PTRACE_GETFSBASE  = C.PT_GETFSBASE
+	PTRACE_GETLWPLIST = C.PT_GETLWPLIST
+	PTRACE_GETNUMLWPS = C.PT_GETNUMLWPS
+	PTRACE_GETREGS    = C.PT_GETREGS
+	PTRACE_GETXSTATE  = C.PT_GETXSTATE
+	PTRACE_IO         = C.PT_IO
+	PTRACE_KILL       = C.PT_KILL
+	PTRACE_LWPEVENTS  = C.PT_LWP_EVENTS
+	PTRACE_LWPINFO    = C.PT_LWPINFO
+	PTRACE_SETFPREGS  = C.PT_SETFPREGS
+	PTRACE_SETREGS    = C.PT_SETREGS
+	PTRACE_SINGLESTEP = C.PT_STEP
+	PTRACE_TRACEME    = C.PT_TRACE_ME
+)
+
+const (
+	PIOD_READ_D  = C.PIOD_READ_D
+	PIOD_WRITE_D = C.PIOD_WRITE_D
+	PIOD_READ_I  = C.PIOD_READ_I
+	PIOD_WRITE_I = C.PIOD_WRITE_I
+)
+
+const (
+	PL_FLAG_BORN   = C.PL_FLAG_BORN
+	PL_FLAG_EXITED = C.PL_FLAG_EXITED
+	PL_FLAG_SI     = C.PL_FLAG_SI
+)
+
+const (
+	TRAP_BRKPT = C.TRAP_BRKPT
+	TRAP_TRACE = C.TRAP_TRACE
+)
+
+type PtraceLwpInfoStruct C.struct_ptrace_lwpinfo
+
+type __Siginfo C.struct___siginfo
+
+type Sigset_t C.sigset_t
+
+type Reg C.struct_reg
+
+type FpReg C.struct_fpreg
+
+type PtraceIoDesc C.struct_ptrace_io_desc
+
+// Events (kqueue, kevent)
+
+type Kevent_t C.struct_kevent_freebsd11
+
+// Select
+
+type FdSet C.fd_set
+
+// Routing and interface messages
+
+const (
+	sizeofIfMsghdr         = C.sizeof_struct_if_msghdr
+	SizeofIfMsghdr         = C.sizeof_struct_if_msghdr8
+	sizeofIfData           = C.sizeof_struct_if_data
+	SizeofIfData           = C.sizeof_struct_if_data8
+	SizeofIfaMsghdr        = C.sizeof_struct_ifa_msghdr
+	SizeofIfmaMsghdr       = C.sizeof_struct_ifma_msghdr
+	SizeofIfAnnounceMsghdr = C.sizeof_struct_if_announcemsghdr
+	SizeofRtMsghdr         = C.sizeof_struct_rt_msghdr
+	SizeofRtMetrics        = C.sizeof_struct_rt_metrics
+)
+
+type ifMsghdr C.struct_if_msghdr
+
+type IfMsghdr C.struct_if_msghdr8
+
+type ifData C.struct_if_data
+
+type IfData C.struct_if_data8
+
+type IfaMsghdr C.struct_ifa_msghdr
+
+type IfmaMsghdr C.struct_ifma_msghdr
+
+type IfAnnounceMsghdr C.struct_if_announcemsghdr
+
+type RtMsghdr C.struct_rt_msghdr
+
+type RtMetrics C.struct_rt_metrics
+
+// Berkeley packet filter
+
+const (
+	SizeofBpfVersion    = C.sizeof_struct_bpf_version
+	SizeofBpfStat       = C.sizeof_struct_bpf_stat
+	SizeofBpfZbuf       = C.sizeof_struct_bpf_zbuf
+	SizeofBpfProgram    = C.sizeof_struct_bpf_program
+	SizeofBpfInsn       = C.sizeof_struct_bpf_insn
+	SizeofBpfHdr        = C.sizeof_struct_bpf_hdr
+	SizeofBpfZbufHeader = C.sizeof_struct_bpf_zbuf_header
+)
+
+type BpfVersion C.struct_bpf_version
+
+type BpfStat C.struct_bpf_stat
+
+type BpfZbuf C.struct_bpf_zbuf
+
+type BpfProgram C.struct_bpf_program
+
+type BpfInsn C.struct_bpf_insn
+
+type BpfHdr C.struct_bpf_hdr
+
+type BpfZbufHeader C.struct_bpf_zbuf_header
+
+// Terminal handling
+
+type Termios C.struct_termios
+
+type Winsize C.struct_winsize
+
+// fchmodat-like syscalls.
+
+const (
+	AT_FDCWD            = C.AT_FDCWD
+	AT_REMOVEDIR        = C.AT_REMOVEDIR
+	AT_SYMLINK_FOLLOW   = C.AT_SYMLINK_FOLLOW
+	AT_SYMLINK_NOFOLLOW = C.AT_SYMLINK_NOFOLLOW
+)
+
+// poll
+
+type PollFd C.struct_pollfd
+
+const (
+	POLLERR      = C.POLLERR
+	POLLHUP      = C.POLLHUP
+	POLLIN       = C.POLLIN
+	POLLINIGNEOF = C.POLLINIGNEOF
+	POLLNVAL     = C.POLLNVAL
+	POLLOUT      = C.POLLOUT
+	POLLPRI      = C.POLLPRI
+	POLLRDBAND   = C.POLLRDBAND
+	POLLRDNORM   = C.POLLRDNORM
+	POLLWRBAND   = C.POLLWRBAND
+	POLLWRNORM   = C.POLLWRNORM
+)
+
+// Capabilities
+
+type CapRights C.struct_cap_rights
+
+// Uname
+
+type Utsname C.struct_utsname
+
+// Clockinfo
+
+const SizeofClockinfo = C.sizeof_struct_clockinfo
+
+type Clockinfo C.struct_clockinfo
diff --git a/vendor/golang.org/x/sys/unix/types_netbsd.go b/vendor/golang.org/x/sys/unix/types_netbsd.go
new file mode 100644
index 0000000..0a81aad
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/types_netbsd.go
@@ -0,0 +1,300 @@
+// 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 ignore
+
+/*
+Input to cgo -godefs.  See README.md
+*/
+
+// +godefs map struct_in_addr [4]byte /* in_addr */
+// +godefs map struct_in6_addr [16]byte /* in6_addr */
+
+package unix
+
+/*
+#define KERNEL
+#include <dirent.h>
+#include <fcntl.h>
+#include <poll.h>
+#include <signal.h>
+#include <termios.h>
+#include <stdio.h>
+#include <unistd.h>
+#include <sys/param.h>
+#include <sys/types.h>
+#include <sys/event.h>
+#include <sys/mman.h>
+#include <sys/mount.h>
+#include <sys/ptrace.h>
+#include <sys/resource.h>
+#include <sys/select.h>
+#include <sys/signal.h>
+#include <sys/socket.h>
+#include <sys/stat.h>
+#include <sys/statvfs.h>
+#include <sys/sysctl.h>
+#include <sys/time.h>
+#include <sys/uio.h>
+#include <sys/un.h>
+#include <sys/utsname.h>
+#include <sys/wait.h>
+#include <net/bpf.h>
+#include <net/if.h>
+#include <net/if_dl.h>
+#include <net/route.h>
+#include <netinet/in.h>
+#include <netinet/icmp6.h>
+#include <netinet/tcp.h>
+
+enum {
+	sizeofPtr = sizeof(void*),
+};
+
+union sockaddr_all {
+	struct sockaddr s1;	// this one gets used for fields
+	struct sockaddr_in s2;	// these pad it out
+	struct sockaddr_in6 s3;
+	struct sockaddr_un s4;
+	struct sockaddr_dl s5;
+};
+
+struct sockaddr_any {
+	struct sockaddr addr;
+	char pad[sizeof(union sockaddr_all) - sizeof(struct sockaddr)];
+};
+
+*/
+import "C"
+
+// Machine characteristics
+
+const (
+	SizeofPtr      = C.sizeofPtr
+	SizeofShort    = C.sizeof_short
+	SizeofInt      = C.sizeof_int
+	SizeofLong     = C.sizeof_long
+	SizeofLongLong = C.sizeof_longlong
+)
+
+// Basic types
+
+type (
+	_C_short     C.short
+	_C_int       C.int
+	_C_long      C.long
+	_C_long_long C.longlong
+)
+
+// Time
+
+type Timespec C.struct_timespec
+
+type Timeval C.struct_timeval
+
+// Processes
+
+type Rusage C.struct_rusage
+
+type Rlimit C.struct_rlimit
+
+type _Gid_t C.gid_t
+
+// Files
+
+type Stat_t C.struct_stat
+
+type Statfs_t C.struct_statfs
+
+type Statvfs_t C.struct_statvfs
+
+type Flock_t C.struct_flock
+
+type Dirent C.struct_dirent
+
+type Fsid C.fsid_t
+
+// File system limits
+
+const (
+	PathMax = C.PATH_MAX
+)
+
+// Fstatvfs/Statvfs flags
+
+const (
+	ST_WAIT   = C.ST_WAIT
+	ST_NOWAIT = C.ST_NOWAIT
+)
+
+// Advice to Fadvise
+
+const (
+	FADV_NORMAL     = C.POSIX_FADV_NORMAL
+	FADV_RANDOM     = C.POSIX_FADV_RANDOM
+	FADV_SEQUENTIAL = C.POSIX_FADV_SEQUENTIAL
+	FADV_WILLNEED   = C.POSIX_FADV_WILLNEED
+	FADV_DONTNEED   = C.POSIX_FADV_DONTNEED
+	FADV_NOREUSE    = C.POSIX_FADV_NOREUSE
+)
+
+// Sockets
+
+type RawSockaddrInet4 C.struct_sockaddr_in
+
+type RawSockaddrInet6 C.struct_sockaddr_in6
+
+type RawSockaddrUnix C.struct_sockaddr_un
+
+type RawSockaddrDatalink C.struct_sockaddr_dl
+
+type RawSockaddr C.struct_sockaddr
+
+type RawSockaddrAny C.struct_sockaddr_any
+
+type _Socklen C.socklen_t
+
+type Linger C.struct_linger
+
+type Iovec C.struct_iovec
+
+type IPMreq C.struct_ip_mreq
+
+type IPv6Mreq C.struct_ipv6_mreq
+
+type Msghdr C.struct_msghdr
+
+type Cmsghdr C.struct_cmsghdr
+
+type Inet6Pktinfo C.struct_in6_pktinfo
+
+type IPv6MTUInfo C.struct_ip6_mtuinfo
+
+type ICMPv6Filter C.struct_icmp6_filter
+
+const (
+	SizeofSockaddrInet4    = C.sizeof_struct_sockaddr_in
+	SizeofSockaddrInet6    = C.sizeof_struct_sockaddr_in6
+	SizeofSockaddrAny      = C.sizeof_struct_sockaddr_any
+	SizeofSockaddrUnix     = C.sizeof_struct_sockaddr_un
+	SizeofSockaddrDatalink = C.sizeof_struct_sockaddr_dl
+	SizeofLinger           = C.sizeof_struct_linger
+	SizeofIPMreq           = C.sizeof_struct_ip_mreq
+	SizeofIPv6Mreq         = C.sizeof_struct_ipv6_mreq
+	SizeofMsghdr           = C.sizeof_struct_msghdr
+	SizeofCmsghdr          = C.sizeof_struct_cmsghdr
+	SizeofInet6Pktinfo     = C.sizeof_struct_in6_pktinfo
+	SizeofIPv6MTUInfo      = C.sizeof_struct_ip6_mtuinfo
+	SizeofICMPv6Filter     = C.sizeof_struct_icmp6_filter
+)
+
+// Ptrace requests
+
+const (
+	PTRACE_TRACEME = C.PT_TRACE_ME
+	PTRACE_CONT    = C.PT_CONTINUE
+	PTRACE_KILL    = C.PT_KILL
+)
+
+// Events (kqueue, kevent)
+
+type Kevent_t C.struct_kevent
+
+// Select
+
+type FdSet C.fd_set
+
+// Routing and interface messages
+
+const (
+	SizeofIfMsghdr         = C.sizeof_struct_if_msghdr
+	SizeofIfData           = C.sizeof_struct_if_data
+	SizeofIfaMsghdr        = C.sizeof_struct_ifa_msghdr
+	SizeofIfAnnounceMsghdr = C.sizeof_struct_if_announcemsghdr
+	SizeofRtMsghdr         = C.sizeof_struct_rt_msghdr
+	SizeofRtMetrics        = C.sizeof_struct_rt_metrics
+)
+
+type IfMsghdr C.struct_if_msghdr
+
+type IfData C.struct_if_data
+
+type IfaMsghdr C.struct_ifa_msghdr
+
+type IfAnnounceMsghdr C.struct_if_announcemsghdr
+
+type RtMsghdr C.struct_rt_msghdr
+
+type RtMetrics C.struct_rt_metrics
+
+type Mclpool C.struct_mclpool
+
+// Berkeley packet filter
+
+const (
+	SizeofBpfVersion = C.sizeof_struct_bpf_version
+	SizeofBpfStat    = C.sizeof_struct_bpf_stat
+	SizeofBpfProgram = C.sizeof_struct_bpf_program
+	SizeofBpfInsn    = C.sizeof_struct_bpf_insn
+	SizeofBpfHdr     = C.sizeof_struct_bpf_hdr
+)
+
+type BpfVersion C.struct_bpf_version
+
+type BpfStat C.struct_bpf_stat
+
+type BpfProgram C.struct_bpf_program
+
+type BpfInsn C.struct_bpf_insn
+
+type BpfHdr C.struct_bpf_hdr
+
+type BpfTimeval C.struct_bpf_timeval
+
+// Terminal handling
+
+type Termios C.struct_termios
+
+type Winsize C.struct_winsize
+
+type Ptmget C.struct_ptmget
+
+// fchmodat-like syscalls.
+
+const (
+	AT_FDCWD            = C.AT_FDCWD
+	AT_SYMLINK_FOLLOW   = C.AT_SYMLINK_FOLLOW
+	AT_SYMLINK_NOFOLLOW = C.AT_SYMLINK_NOFOLLOW
+)
+
+// poll
+
+type PollFd C.struct_pollfd
+
+const (
+	POLLERR    = C.POLLERR
+	POLLHUP    = C.POLLHUP
+	POLLIN     = C.POLLIN
+	POLLNVAL   = C.POLLNVAL
+	POLLOUT    = C.POLLOUT
+	POLLPRI    = C.POLLPRI
+	POLLRDBAND = C.POLLRDBAND
+	POLLRDNORM = C.POLLRDNORM
+	POLLWRBAND = C.POLLWRBAND
+	POLLWRNORM = C.POLLWRNORM
+)
+
+// Sysctl
+
+type Sysctlnode C.struct_sysctlnode
+
+// Uname
+
+type Utsname C.struct_utsname
+
+// Clockinfo
+
+const SizeofClockinfo = C.sizeof_struct_clockinfo
+
+type Clockinfo C.struct_clockinfo
diff --git a/vendor/golang.org/x/sys/unix/types_openbsd.go b/vendor/golang.org/x/sys/unix/types_openbsd.go
new file mode 100644
index 0000000..775cb57
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/types_openbsd.go
@@ -0,0 +1,283 @@
+// 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 ignore
+
+/*
+Input to cgo -godefs.  See README.md
+*/
+
+// +godefs map struct_in_addr [4]byte /* in_addr */
+// +godefs map struct_in6_addr [16]byte /* in6_addr */
+
+package unix
+
+/*
+#define KERNEL
+#include <dirent.h>
+#include <fcntl.h>
+#include <poll.h>
+#include <signal.h>
+#include <termios.h>
+#include <stdio.h>
+#include <unistd.h>
+#include <sys/param.h>
+#include <sys/types.h>
+#include <sys/event.h>
+#include <sys/mman.h>
+#include <sys/mount.h>
+#include <sys/ptrace.h>
+#include <sys/resource.h>
+#include <sys/select.h>
+#include <sys/signal.h>
+#include <sys/socket.h>
+#include <sys/stat.h>
+#include <sys/time.h>
+#include <sys/uio.h>
+#include <sys/un.h>
+#include <sys/utsname.h>
+#include <sys/wait.h>
+#include <uvm/uvmexp.h>
+#include <net/bpf.h>
+#include <net/if.h>
+#include <net/if_dl.h>
+#include <net/route.h>
+#include <netinet/in.h>
+#include <netinet/icmp6.h>
+#include <netinet/tcp.h>
+
+enum {
+	sizeofPtr = sizeof(void*),
+};
+
+union sockaddr_all {
+	struct sockaddr s1;	// this one gets used for fields
+	struct sockaddr_in s2;	// these pad it out
+	struct sockaddr_in6 s3;
+	struct sockaddr_un s4;
+	struct sockaddr_dl s5;
+};
+
+struct sockaddr_any {
+	struct sockaddr addr;
+	char pad[sizeof(union sockaddr_all) - sizeof(struct sockaddr)];
+};
+
+*/
+import "C"
+
+// Machine characteristics
+
+const (
+	SizeofPtr      = C.sizeofPtr
+	SizeofShort    = C.sizeof_short
+	SizeofInt      = C.sizeof_int
+	SizeofLong     = C.sizeof_long
+	SizeofLongLong = C.sizeof_longlong
+)
+
+// Basic types
+
+type (
+	_C_short     C.short
+	_C_int       C.int
+	_C_long      C.long
+	_C_long_long C.longlong
+)
+
+// Time
+
+type Timespec C.struct_timespec
+
+type Timeval C.struct_timeval
+
+// Processes
+
+type Rusage C.struct_rusage
+
+type Rlimit C.struct_rlimit
+
+type _Gid_t C.gid_t
+
+// Files
+
+type Stat_t C.struct_stat
+
+type Statfs_t C.struct_statfs
+
+type Flock_t C.struct_flock
+
+type Dirent C.struct_dirent
+
+type Fsid C.fsid_t
+
+// File system limits
+
+const (
+	PathMax = C.PATH_MAX
+)
+
+// Sockets
+
+type RawSockaddrInet4 C.struct_sockaddr_in
+
+type RawSockaddrInet6 C.struct_sockaddr_in6
+
+type RawSockaddrUnix C.struct_sockaddr_un
+
+type RawSockaddrDatalink C.struct_sockaddr_dl
+
+type RawSockaddr C.struct_sockaddr
+
+type RawSockaddrAny C.struct_sockaddr_any
+
+type _Socklen C.socklen_t
+
+type Linger C.struct_linger
+
+type Iovec C.struct_iovec
+
+type IPMreq C.struct_ip_mreq
+
+type IPv6Mreq C.struct_ipv6_mreq
+
+type Msghdr C.struct_msghdr
+
+type Cmsghdr C.struct_cmsghdr
+
+type Inet6Pktinfo C.struct_in6_pktinfo
+
+type IPv6MTUInfo C.struct_ip6_mtuinfo
+
+type ICMPv6Filter C.struct_icmp6_filter
+
+const (
+	SizeofSockaddrInet4    = C.sizeof_struct_sockaddr_in
+	SizeofSockaddrInet6    = C.sizeof_struct_sockaddr_in6
+	SizeofSockaddrAny      = C.sizeof_struct_sockaddr_any
+	SizeofSockaddrUnix     = C.sizeof_struct_sockaddr_un
+	SizeofSockaddrDatalink = C.sizeof_struct_sockaddr_dl
+	SizeofLinger           = C.sizeof_struct_linger
+	SizeofIPMreq           = C.sizeof_struct_ip_mreq
+	SizeofIPv6Mreq         = C.sizeof_struct_ipv6_mreq
+	SizeofMsghdr           = C.sizeof_struct_msghdr
+	SizeofCmsghdr          = C.sizeof_struct_cmsghdr
+	SizeofInet6Pktinfo     = C.sizeof_struct_in6_pktinfo
+	SizeofIPv6MTUInfo      = C.sizeof_struct_ip6_mtuinfo
+	SizeofICMPv6Filter     = C.sizeof_struct_icmp6_filter
+)
+
+// Ptrace requests
+
+const (
+	PTRACE_TRACEME = C.PT_TRACE_ME
+	PTRACE_CONT    = C.PT_CONTINUE
+	PTRACE_KILL    = C.PT_KILL
+)
+
+// Events (kqueue, kevent)
+
+type Kevent_t C.struct_kevent
+
+// Select
+
+type FdSet C.fd_set
+
+// Routing and interface messages
+
+const (
+	SizeofIfMsghdr         = C.sizeof_struct_if_msghdr
+	SizeofIfData           = C.sizeof_struct_if_data
+	SizeofIfaMsghdr        = C.sizeof_struct_ifa_msghdr
+	SizeofIfAnnounceMsghdr = C.sizeof_struct_if_announcemsghdr
+	SizeofRtMsghdr         = C.sizeof_struct_rt_msghdr
+	SizeofRtMetrics        = C.sizeof_struct_rt_metrics
+)
+
+type IfMsghdr C.struct_if_msghdr
+
+type IfData C.struct_if_data
+
+type IfaMsghdr C.struct_ifa_msghdr
+
+type IfAnnounceMsghdr C.struct_if_announcemsghdr
+
+type RtMsghdr C.struct_rt_msghdr
+
+type RtMetrics C.struct_rt_metrics
+
+type Mclpool C.struct_mclpool
+
+// Berkeley packet filter
+
+const (
+	SizeofBpfVersion = C.sizeof_struct_bpf_version
+	SizeofBpfStat    = C.sizeof_struct_bpf_stat
+	SizeofBpfProgram = C.sizeof_struct_bpf_program
+	SizeofBpfInsn    = C.sizeof_struct_bpf_insn
+	SizeofBpfHdr     = C.sizeof_struct_bpf_hdr
+)
+
+type BpfVersion C.struct_bpf_version
+
+type BpfStat C.struct_bpf_stat
+
+type BpfProgram C.struct_bpf_program
+
+type BpfInsn C.struct_bpf_insn
+
+type BpfHdr C.struct_bpf_hdr
+
+type BpfTimeval C.struct_bpf_timeval
+
+// Terminal handling
+
+type Termios C.struct_termios
+
+type Winsize C.struct_winsize
+
+// fchmodat-like syscalls.
+
+const (
+	AT_FDCWD            = C.AT_FDCWD
+	AT_SYMLINK_FOLLOW   = C.AT_SYMLINK_FOLLOW
+	AT_SYMLINK_NOFOLLOW = C.AT_SYMLINK_NOFOLLOW
+)
+
+// poll
+
+type PollFd C.struct_pollfd
+
+const (
+	POLLERR    = C.POLLERR
+	POLLHUP    = C.POLLHUP
+	POLLIN     = C.POLLIN
+	POLLNVAL   = C.POLLNVAL
+	POLLOUT    = C.POLLOUT
+	POLLPRI    = C.POLLPRI
+	POLLRDBAND = C.POLLRDBAND
+	POLLRDNORM = C.POLLRDNORM
+	POLLWRBAND = C.POLLWRBAND
+	POLLWRNORM = C.POLLWRNORM
+)
+
+// Signal Sets
+
+type Sigset_t C.sigset_t
+
+// Uname
+
+type Utsname C.struct_utsname
+
+// Uvmexp
+
+const SizeofUvmexp = C.sizeof_struct_uvmexp
+
+type Uvmexp C.struct_uvmexp
+
+// Clockinfo
+
+const SizeofClockinfo = C.sizeof_struct_clockinfo
+
+type Clockinfo C.struct_clockinfo
diff --git a/vendor/golang.org/x/sys/unix/types_solaris.go b/vendor/golang.org/x/sys/unix/types_solaris.go
new file mode 100644
index 0000000..d713f09
--- /dev/null
+++ b/vendor/golang.org/x/sys/unix/types_solaris.go
@@ -0,0 +1,269 @@
+// 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 ignore
+
+/*
+Input to cgo -godefs.  See README.md
+*/
+
+// +godefs map struct_in_addr [4]byte /* in_addr */
+// +godefs map struct_in6_addr [16]byte /* in6_addr */
+
+package unix
+
+/*
+#define KERNEL
+// These defines ensure that builds done on newer versions of Solaris are
+// backwards-compatible with older versions of Solaris and
+// OpenSolaris-based derivatives.
+#define __USE_SUNOS_SOCKETS__          // msghdr
+#define __USE_LEGACY_PROTOTYPES__      // iovec
+#include <dirent.h>
+#include <fcntl.h>
+#include <netdb.h>
+#include <limits.h>
+#include <poll.h>
+#include <signal.h>
+#include <termios.h>
+#include <termio.h>
+#include <stdio.h>
+#include <unistd.h>
+#include <sys/mman.h>
+#include <sys/mount.h>
+#include <sys/param.h>
+#include <sys/resource.h>
+#include <sys/select.h>
+#include <sys/signal.h>
+#include <sys/socket.h>
+#include <sys/stat.h>
+#include <sys/statvfs.h>
+#include <sys/time.h>
+#include <sys/times.h>
+#include <sys/types.h>
+#include <sys/utsname.h>
+#include <sys/un.h>
+#include <sys/wait.h>
+#include <net/bpf.h>
+#include <net/if.h>
+#include <net/if_dl.h>
+#include <net/route.h>
+#include <netinet/in.h>
+#include <netinet/icmp6.h>
+#include <netinet/tcp.h>
+#include <ustat.h>
+#include <utime.h>
+
+enum {
+	sizeofPtr = sizeof(void*),
+};
+
+union sockaddr_all {
+	struct sockaddr s1;	// this one gets used for fields
+	struct sockaddr_in s2;	// these pad it out
+	struct sockaddr_in6 s3;
+	struct sockaddr_un s4;
+	struct sockaddr_dl s5;
+};
+
+struct sockaddr_any {
+	struct sockaddr addr;
+	char pad[sizeof(union sockaddr_all) - sizeof(struct sockaddr)];
+};
+
+*/
+import "C"
+
+// Machine characteristics
+
+const (
+	SizeofPtr      = C.sizeofPtr
+	SizeofShort    = C.sizeof_short
+	SizeofInt      = C.sizeof_int
+	SizeofLong     = C.sizeof_long
+	SizeofLongLong = C.sizeof_longlong
+	PathMax        = C.PATH_MAX
+	MaxHostNameLen = C.MAXHOSTNAMELEN
+)
+
+// Basic types
+
+type (
+	_C_short     C.short
+	_C_int       C.int
+	_C_long      C.long
+	_C_long_long C.longlong
+)
+
+// Time
+
+type Timespec C.struct_timespec
+
+type Timeval C.struct_timeval
+
+type Timeval32 C.struct_timeval32
+
+type Tms C.struct_tms
+
+type Utimbuf C.struct_utimbuf
+
+// Processes
+
+type Rusage C.struct_rusage
+
+type Rlimit C.struct_rlimit
+
+type _Gid_t C.gid_t
+
+// Files
+
+type Stat_t C.struct_stat
+
+type Flock_t C.struct_flock
+
+type Dirent C.struct_dirent
+
+// Filesystems
+
+type _Fsblkcnt_t C.fsblkcnt_t
+
+type Statvfs_t C.struct_statvfs
+
+// Sockets
+
+type RawSockaddrInet4 C.struct_sockaddr_in
+
+type RawSockaddrInet6 C.struct_sockaddr_in6
+
+type RawSockaddrUnix C.struct_sockaddr_un
+
+type RawSockaddrDatalink C.struct_sockaddr_dl
+
+type RawSockaddr C.struct_sockaddr
+
+type RawSockaddrAny C.struct_sockaddr_any
+
+type _Socklen C.socklen_t
+
+type Linger C.struct_linger
+
+type Iovec C.struct_iovec
+
+type IPMreq C.struct_ip_mreq
+
+type IPv6Mreq C.struct_ipv6_mreq
+
+type Msghdr C.struct_msghdr
+
+type Cmsghdr C.struct_cmsghdr
+
+type Inet4Pktinfo C.struct_in_pktinfo
+
+type Inet6Pktinfo C.struct_in6_pktinfo
+
+type IPv6MTUInfo C.struct_ip6_mtuinfo
+
+type ICMPv6Filter C.struct_icmp6_filter
+
+const (
+	SizeofSockaddrInet4    = C.sizeof_struct_sockaddr_in
+	SizeofSockaddrInet6    = C.sizeof_struct_sockaddr_in6
+	SizeofSockaddrAny      = C.sizeof_struct_sockaddr_any
+	SizeofSockaddrUnix     = C.sizeof_struct_sockaddr_un
+	SizeofSockaddrDatalink = C.sizeof_struct_sockaddr_dl
+	SizeofLinger           = C.sizeof_struct_linger
+	SizeofIPMreq           = C.sizeof_struct_ip_mreq
+	SizeofIPv6Mreq         = C.sizeof_struct_ipv6_mreq
+	SizeofMsghdr           = C.sizeof_struct_msghdr
+	SizeofCmsghdr          = C.sizeof_struct_cmsghdr
+	SizeofInet4Pktinfo     = C.sizeof_struct_in_pktinfo
+	SizeofInet6Pktinfo     = C.sizeof_struct_in6_pktinfo
+	SizeofIPv6MTUInfo      = C.sizeof_struct_ip6_mtuinfo
+	SizeofICMPv6Filter     = C.sizeof_struct_icmp6_filter
+)
+
+// Select
+
+type FdSet C.fd_set
+
+// Misc
+
+type Utsname C.struct_utsname
+
+type Ustat_t C.struct_ustat
+
+const (
+	AT_FDCWD            = C.AT_FDCWD
+	AT_SYMLINK_NOFOLLOW = C.AT_SYMLINK_NOFOLLOW
+	AT_SYMLINK_FOLLOW   = C.AT_SYMLINK_FOLLOW
+	AT_REMOVEDIR        = C.AT_REMOVEDIR
+	AT_EACCESS          = C.AT_EACCESS
+)
+
+// Routing and interface messages
+
+const (
+	SizeofIfMsghdr  = C.sizeof_struct_if_msghdr
+	SizeofIfData    = C.sizeof_struct_if_data
+	SizeofIfaMsghdr = C.sizeof_struct_ifa_msghdr
+	SizeofRtMsghdr  = C.sizeof_struct_rt_msghdr
+	SizeofRtMetrics = C.sizeof_struct_rt_metrics
+)
+
+type IfMsghdr C.struct_if_msghdr
+
+type IfData C.struct_if_data
+
+type IfaMsghdr C.struct_ifa_msghdr
+
+type RtMsghdr C.struct_rt_msghdr
+
+type RtMetrics C.struct_rt_metrics
+
+// Berkeley packet filter
+
+const (
+	SizeofBpfVersion = C.sizeof_struct_bpf_version
+	SizeofBpfStat    = C.sizeof_struct_bpf_stat
+	SizeofBpfProgram = C.sizeof_struct_bpf_program
+	SizeofBpfInsn    = C.sizeof_struct_bpf_insn
+	SizeofBpfHdr     = C.sizeof_struct_bpf_hdr
+)
+
+type BpfVersion C.struct_bpf_version
+
+type BpfStat C.struct_bpf_stat
+
+type BpfProgram C.struct_bpf_program
+
+type BpfInsn C.struct_bpf_insn
+
+type BpfTimeval C.struct_bpf_timeval
+
+type BpfHdr C.struct_bpf_hdr
+
+// Terminal handling
+
+type Termios C.struct_termios
+
+type Termio C.struct_termio
+
+type Winsize C.struct_winsize
+
+// poll
+
+type PollFd C.struct_pollfd
+
+const (
+	POLLERR    = C.POLLERR
+	POLLHUP    = C.POLLHUP
+	POLLIN     = C.POLLIN
+	POLLNVAL   = C.POLLNVAL
+	POLLOUT    = C.POLLOUT
+	POLLPRI    = C.POLLPRI
+	POLLRDBAND = C.POLLRDBAND
+	POLLRDNORM = C.POLLRDNORM
+	POLLWRBAND = C.POLLWRBAND
+	POLLWRNORM = C.POLLWRNORM
+)
diff --git a/vendor/golang.org/x/text/unicode/bidi/gen.go b/vendor/golang.org/x/text/unicode/bidi/gen.go
new file mode 100644
index 0000000..987fc16
--- /dev/null
+++ b/vendor/golang.org/x/text/unicode/bidi/gen.go
@@ -0,0 +1,133 @@
+// 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 ignore
+
+package main
+
+import (
+	"flag"
+	"log"
+
+	"golang.org/x/text/internal/gen"
+	"golang.org/x/text/internal/triegen"
+	"golang.org/x/text/internal/ucd"
+)
+
+var outputFile = flag.String("out", "tables.go", "output file")
+
+func main() {
+	gen.Init()
+	gen.Repackage("gen_trieval.go", "trieval.go", "bidi")
+	gen.Repackage("gen_ranges.go", "ranges_test.go", "bidi")
+
+	genTables()
+}
+
+// bidiClass names and codes taken from class "bc" in
+// https://www.unicode.org/Public/8.0.0/ucd/PropertyValueAliases.txt
+var bidiClass = map[string]Class{
+	"AL":  AL,  // ArabicLetter
+	"AN":  AN,  // ArabicNumber
+	"B":   B,   // ParagraphSeparator
+	"BN":  BN,  // BoundaryNeutral
+	"CS":  CS,  // CommonSeparator
+	"EN":  EN,  // EuropeanNumber
+	"ES":  ES,  // EuropeanSeparator
+	"ET":  ET,  // EuropeanTerminator
+	"L":   L,   // LeftToRight
+	"NSM": NSM, // NonspacingMark
+	"ON":  ON,  // OtherNeutral
+	"R":   R,   // RightToLeft
+	"S":   S,   // SegmentSeparator
+	"WS":  WS,  // WhiteSpace
+
+	"FSI": Control,
+	"PDF": Control,
+	"PDI": Control,
+	"LRE": Control,
+	"LRI": Control,
+	"LRO": Control,
+	"RLE": Control,
+	"RLI": Control,
+	"RLO": Control,
+}
+
+func genTables() {
+	if numClass > 0x0F {
+		log.Fatalf("Too many Class constants (%#x > 0x0F).", numClass)
+	}
+	w := gen.NewCodeWriter()
+	defer w.WriteVersionedGoFile(*outputFile, "bidi")
+
+	gen.WriteUnicodeVersion(w)
+
+	t := triegen.NewTrie("bidi")
+
+	// Build data about bracket mapping. These bits need to be or-ed with
+	// any other bits.
+	orMask := map[rune]uint64{}
+
+	xorMap := map[rune]int{}
+	xorMasks := []rune{0} // First value is no-op.
+
+	ucd.Parse(gen.OpenUCDFile("BidiBrackets.txt"), func(p *ucd.Parser) {
+		r1 := p.Rune(0)
+		r2 := p.Rune(1)
+		xor := r1 ^ r2
+		if _, ok := xorMap[xor]; !ok {
+			xorMap[xor] = len(xorMasks)
+			xorMasks = append(xorMasks, xor)
+		}
+		entry := uint64(xorMap[xor]) << xorMaskShift
+		switch p.String(2) {
+		case "o":
+			entry |= openMask
+		case "c", "n":
+		default:
+			log.Fatalf("Unknown bracket class %q.", p.String(2))
+		}
+		orMask[r1] = entry
+	})
+
+	w.WriteComment(`
+	xorMasks contains masks to be xor-ed with brackets to get the reverse
+	version.`)
+	w.WriteVar("xorMasks", xorMasks)
+
+	done := map[rune]bool{}
+
+	insert := func(r rune, c Class) {
+		if !done[r] {
+			t.Insert(r, orMask[r]|uint64(c))
+			done[r] = true
+		}
+	}
+
+	// Insert the derived BiDi properties.
+	ucd.Parse(gen.OpenUCDFile("extracted/DerivedBidiClass.txt"), func(p *ucd.Parser) {
+		r := p.Rune(0)
+		class, ok := bidiClass[p.String(1)]
+		if !ok {
+			log.Fatalf("%U: Unknown BiDi class %q", r, p.String(1))
+		}
+		insert(r, class)
+	})
+	visitDefaults(insert)
+
+	// TODO: use sparse blocks. This would reduce table size considerably
+	// from the looks of it.
+
+	sz, err := t.Gen(w)
+	if err != nil {
+		log.Fatal(err)
+	}
+	w.Size += sz
+}
+
+// dummy values to make methods in gen_common compile. The real versions
+// will be generated by this file to tables.go.
+var (
+	xorMasks []rune
+)
diff --git a/vendor/golang.org/x/text/unicode/bidi/gen_ranges.go b/vendor/golang.org/x/text/unicode/bidi/gen_ranges.go
new file mode 100644
index 0000000..02c3b50
--- /dev/null
+++ b/vendor/golang.org/x/text/unicode/bidi/gen_ranges.go
@@ -0,0 +1,57 @@
+// 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 ignore
+
+package main
+
+import (
+	"unicode"
+
+	"golang.org/x/text/internal/gen"
+	"golang.org/x/text/internal/ucd"
+	"golang.org/x/text/unicode/rangetable"
+)
+
+// These tables are hand-extracted from:
+// https://www.unicode.org/Public/8.0.0/ucd/extracted/DerivedBidiClass.txt
+func visitDefaults(fn func(r rune, c Class)) {
+	// first write default values for ranges listed above.
+	visitRunes(fn, AL, []rune{
+		0x0600, 0x07BF, // Arabic
+		0x08A0, 0x08FF, // Arabic Extended-A
+		0xFB50, 0xFDCF, // Arabic Presentation Forms
+		0xFDF0, 0xFDFF,
+		0xFE70, 0xFEFF,
+		0x0001EE00, 0x0001EEFF, // Arabic Mathematical Alpha Symbols
+	})
+	visitRunes(fn, R, []rune{
+		0x0590, 0x05FF, // Hebrew
+		0x07C0, 0x089F, // Nko et al.
+		0xFB1D, 0xFB4F,
+		0x00010800, 0x00010FFF, // Cypriot Syllabary et. al.
+		0x0001E800, 0x0001EDFF,
+		0x0001EF00, 0x0001EFFF,
+	})
+	visitRunes(fn, ET, []rune{ // European Terminator
+		0x20A0, 0x20Cf, // Currency symbols
+	})
+	rangetable.Visit(unicode.Noncharacter_Code_Point, func(r rune) {
+		fn(r, BN) // Boundary Neutral
+	})
+	ucd.Parse(gen.OpenUCDFile("DerivedCoreProperties.txt"), func(p *ucd.Parser) {
+		if p.String(1) == "Default_Ignorable_Code_Point" {
+			fn(p.Rune(0), BN) // Boundary Neutral
+		}
+	})
+}
+
+func visitRunes(fn func(r rune, c Class), c Class, runes []rune) {
+	for i := 0; i < len(runes); i += 2 {
+		lo, hi := runes[i], runes[i+1]
+		for j := lo; j <= hi; j++ {
+			fn(j, c)
+		}
+	}
+}
diff --git a/vendor/golang.org/x/text/unicode/bidi/gen_trieval.go b/vendor/golang.org/x/text/unicode/bidi/gen_trieval.go
new file mode 100644
index 0000000..9cb9942
--- /dev/null
+++ b/vendor/golang.org/x/text/unicode/bidi/gen_trieval.go
@@ -0,0 +1,64 @@
+// 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 ignore
+
+package main
+
+// Class is the Unicode BiDi class. Each rune has a single class.
+type Class uint
+
+const (
+	L       Class = iota // LeftToRight
+	R                    // RightToLeft
+	EN                   // EuropeanNumber
+	ES                   // EuropeanSeparator
+	ET                   // EuropeanTerminator
+	AN                   // ArabicNumber
+	CS                   // CommonSeparator
+	B                    // ParagraphSeparator
+	S                    // SegmentSeparator
+	WS                   // WhiteSpace
+	ON                   // OtherNeutral
+	BN                   // BoundaryNeutral
+	NSM                  // NonspacingMark
+	AL                   // ArabicLetter
+	Control              // Control LRO - PDI
+
+	numClass
+
+	LRO // LeftToRightOverride
+	RLO // RightToLeftOverride
+	LRE // LeftToRightEmbedding
+	RLE // RightToLeftEmbedding
+	PDF // PopDirectionalFormat
+	LRI // LeftToRightIsolate
+	RLI // RightToLeftIsolate
+	FSI // FirstStrongIsolate
+	PDI // PopDirectionalIsolate
+
+	unknownClass = ^Class(0)
+)
+
+var controlToClass = map[rune]Class{
+	0x202D: LRO, // LeftToRightOverride,
+	0x202E: RLO, // RightToLeftOverride,
+	0x202A: LRE, // LeftToRightEmbedding,
+	0x202B: RLE, // RightToLeftEmbedding,
+	0x202C: PDF, // PopDirectionalFormat,
+	0x2066: LRI, // LeftToRightIsolate,
+	0x2067: RLI, // RightToLeftIsolate,
+	0x2068: FSI, // FirstStrongIsolate,
+	0x2069: PDI, // PopDirectionalIsolate,
+}
+
+// A trie entry has the following bits:
+// 7..5  XOR mask for brackets
+// 4     1: Bracket open, 0: Bracket close
+// 3..0  Class type
+
+const (
+	openMask     = 0x10
+	xorMaskShift = 5
+)
diff --git a/vendor/golang.org/x/text/unicode/norm/maketables.go b/vendor/golang.org/x/text/unicode/norm/maketables.go
new file mode 100644
index 0000000..30a3aa9
--- /dev/null
+++ b/vendor/golang.org/x/text/unicode/norm/maketables.go
@@ -0,0 +1,986 @@
+// Copyright 2011 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build ignore
+
+// Normalization table generator.
+// Data read from the web.
+// See forminfo.go for a description of the trie values associated with each rune.
+
+package main
+
+import (
+	"bytes"
+	"encoding/binary"
+	"flag"
+	"fmt"
+	"io"
+	"log"
+	"sort"
+	"strconv"
+	"strings"
+
+	"golang.org/x/text/internal/gen"
+	"golang.org/x/text/internal/triegen"
+	"golang.org/x/text/internal/ucd"
+)
+
+func main() {
+	gen.Init()
+	loadUnicodeData()
+	compactCCC()
+	loadCompositionExclusions()
+	completeCharFields(FCanonical)
+	completeCharFields(FCompatibility)
+	computeNonStarterCounts()
+	verifyComputed()
+	printChars()
+	testDerived()
+	printTestdata()
+	makeTables()
+}
+
+var (
+	tablelist = flag.String("tables",
+		"all",
+		"comma-separated list of which tables to generate; "+
+			"can be 'decomp', 'recomp', 'info' and 'all'")
+	test = flag.Bool("test",
+		false,
+		"test existing tables against DerivedNormalizationProps and generate test data for regression testing")
+	verbose = flag.Bool("verbose",
+		false,
+		"write data to stdout as it is parsed")
+)
+
+const MaxChar = 0x10FFFF // anything above this shouldn't exist
+
+// Quick Check properties of runes allow us to quickly
+// determine whether a rune may occur in a normal form.
+// For a given normal form, a rune may be guaranteed to occur
+// verbatim (QC=Yes), may or may not combine with another
+// rune (QC=Maybe), or may not occur (QC=No).
+type QCResult int
+
+const (
+	QCUnknown QCResult = iota
+	QCYes
+	QCNo
+	QCMaybe
+)
+
+func (r QCResult) String() string {
+	switch r {
+	case QCYes:
+		return "Yes"
+	case QCNo:
+		return "No"
+	case QCMaybe:
+		return "Maybe"
+	}
+	return "***UNKNOWN***"
+}
+
+const (
+	FCanonical     = iota // NFC or NFD
+	FCompatibility        // NFKC or NFKD
+	FNumberOfFormTypes
+)
+
+const (
+	MComposed   = iota // NFC or NFKC
+	MDecomposed        // NFD or NFKD
+	MNumberOfModes
+)
+
+// This contains only the properties we're interested in.
+type Char struct {
+	name          string
+	codePoint     rune  // if zero, this index is not a valid code point.
+	ccc           uint8 // canonical combining class
+	origCCC       uint8
+	excludeInComp bool // from CompositionExclusions.txt
+	compatDecomp  bool // it has a compatibility expansion
+
+	nTrailingNonStarters uint8
+	nLeadingNonStarters  uint8 // must be equal to trailing if non-zero
+
+	forms [FNumberOfFormTypes]FormInfo // For FCanonical and FCompatibility
+
+	state State
+}
+
+var chars = make([]Char, MaxChar+1)
+var cccMap = make(map[uint8]uint8)
+
+func (c Char) String() string {
+	buf := new(bytes.Buffer)
+
+	fmt.Fprintf(buf, "%U [%s]:\n", c.codePoint, c.name)
+	fmt.Fprintf(buf, "  ccc: %v\n", c.ccc)
+	fmt.Fprintf(buf, "  excludeInComp: %v\n", c.excludeInComp)
+	fmt.Fprintf(buf, "  compatDecomp: %v\n", c.compatDecomp)
+	fmt.Fprintf(buf, "  state: %v\n", c.state)
+	fmt.Fprintf(buf, "  NFC:\n")
+	fmt.Fprint(buf, c.forms[FCanonical])
+	fmt.Fprintf(buf, "  NFKC:\n")
+	fmt.Fprint(buf, c.forms[FCompatibility])
+
+	return buf.String()
+}
+
+// In UnicodeData.txt, some ranges are marked like this:
+//	3400;<CJK Ideograph Extension A, First>;Lo;0;L;;;;;N;;;;;
+//	4DB5;<CJK Ideograph Extension A, Last>;Lo;0;L;;;;;N;;;;;
+// parseCharacter keeps a state variable indicating the weirdness.
+type State int
+
+const (
+	SNormal State = iota // known to be zero for the type
+	SFirst
+	SLast
+	SMissing
+)
+
+var lastChar = rune('\u0000')
+
+func (c Char) isValid() bool {
+	return c.codePoint != 0 && c.state != SMissing
+}
+
+type FormInfo struct {
+	quickCheck [MNumberOfModes]QCResult // index: MComposed or MDecomposed
+	verified   [MNumberOfModes]bool     // index: MComposed or MDecomposed
+
+	combinesForward  bool // May combine with rune on the right
+	combinesBackward bool // May combine with rune on the left
+	isOneWay         bool // Never appears in result
+	inDecomp         bool // Some decompositions result in this char.
+	decomp           Decomposition
+	expandedDecomp   Decomposition
+}
+
+func (f FormInfo) String() string {
+	buf := bytes.NewBuffer(make([]byte, 0))
+
+	fmt.Fprintf(buf, "    quickCheck[C]: %v\n", f.quickCheck[MComposed])
+	fmt.Fprintf(buf, "    quickCheck[D]: %v\n", f.quickCheck[MDecomposed])
+	fmt.Fprintf(buf, "    cmbForward: %v\n", f.combinesForward)
+	fmt.Fprintf(buf, "    cmbBackward: %v\n", f.combinesBackward)
+	fmt.Fprintf(buf, "    isOneWay: %v\n", f.isOneWay)
+	fmt.Fprintf(buf, "    inDecomp: %v\n", f.inDecomp)
+	fmt.Fprintf(buf, "    decomposition: %X\n", f.decomp)
+	fmt.Fprintf(buf, "    expandedDecomp: %X\n", f.expandedDecomp)
+
+	return buf.String()
+}
+
+type Decomposition []rune
+
+func parseDecomposition(s string, skipfirst bool) (a []rune, err error) {
+	decomp := strings.Split(s, " ")
+	if len(decomp) > 0 && skipfirst {
+		decomp = decomp[1:]
+	}
+	for _, d := range decomp {
+		point, err := strconv.ParseUint(d, 16, 64)
+		if err != nil {
+			return a, err
+		}
+		a = append(a, rune(point))
+	}
+	return a, nil
+}
+
+func loadUnicodeData() {
+	f := gen.OpenUCDFile("UnicodeData.txt")
+	defer f.Close()
+	p := ucd.New(f)
+	for p.Next() {
+		r := p.Rune(ucd.CodePoint)
+		char := &chars[r]
+
+		char.ccc = uint8(p.Uint(ucd.CanonicalCombiningClass))
+		decmap := p.String(ucd.DecompMapping)
+
+		exp, err := parseDecomposition(decmap, false)
+		isCompat := false
+		if err != nil {
+			if len(decmap) > 0 {
+				exp, err = parseDecomposition(decmap, true)
+				if err != nil {
+					log.Fatalf(`%U: bad decomp |%v|: "%s"`, r, decmap, err)
+				}
+				isCompat = true
+			}
+		}
+
+		char.name = p.String(ucd.Name)
+		char.codePoint = r
+		char.forms[FCompatibility].decomp = exp
+		if !isCompat {
+			char.forms[FCanonical].decomp = exp
+		} else {
+			char.compatDecomp = true
+		}
+		if len(decmap) > 0 {
+			char.forms[FCompatibility].decomp = exp
+		}
+	}
+	if err := p.Err(); err != nil {
+		log.Fatal(err)
+	}
+}
+
+// compactCCC converts the sparse set of CCC values to a continguous one,
+// reducing the number of bits needed from 8 to 6.
+func compactCCC() {
+	m := make(map[uint8]uint8)
+	for i := range chars {
+		c := &chars[i]
+		m[c.ccc] = 0
+	}
+	cccs := []int{}
+	for v, _ := range m {
+		cccs = append(cccs, int(v))
+	}
+	sort.Ints(cccs)
+	for i, c := range cccs {
+		cccMap[uint8(i)] = uint8(c)
+		m[uint8(c)] = uint8(i)
+	}
+	for i := range chars {
+		c := &chars[i]
+		c.origCCC = c.ccc
+		c.ccc = m[c.ccc]
+	}
+	if len(m) >= 1<<6 {
+		log.Fatalf("too many difference CCC values: %d >= 64", len(m))
+	}
+}
+
+// CompositionExclusions.txt has form:
+// 0958    # ...
+// See https://unicode.org/reports/tr44/ for full explanation
+func loadCompositionExclusions() {
+	f := gen.OpenUCDFile("CompositionExclusions.txt")
+	defer f.Close()
+	p := ucd.New(f)
+	for p.Next() {
+		c := &chars[p.Rune(0)]
+		if c.excludeInComp {
+			log.Fatalf("%U: Duplicate entry in exclusions.", c.codePoint)
+		}
+		c.excludeInComp = true
+	}
+	if e := p.Err(); e != nil {
+		log.Fatal(e)
+	}
+}
+
+// hasCompatDecomp returns true if any of the recursive
+// decompositions contains a compatibility expansion.
+// In this case, the character may not occur in NFK*.
+func hasCompatDecomp(r rune) bool {
+	c := &chars[r]
+	if c.compatDecomp {
+		return true
+	}
+	for _, d := range c.forms[FCompatibility].decomp {
+		if hasCompatDecomp(d) {
+			return true
+		}
+	}
+	return false
+}
+
+// Hangul related constants.
+const (
+	HangulBase = 0xAC00
+	HangulEnd  = 0xD7A4 // hangulBase + Jamo combinations (19 * 21 * 28)
+
+	JamoLBase = 0x1100
+	JamoLEnd  = 0x1113
+	JamoVBase = 0x1161
+	JamoVEnd  = 0x1176
+	JamoTBase = 0x11A8
+	JamoTEnd  = 0x11C3
+
+	JamoLVTCount = 19 * 21 * 28
+	JamoTCount   = 28
+)
+
+func isHangul(r rune) bool {
+	return HangulBase <= r && r < HangulEnd
+}
+
+func isHangulWithoutJamoT(r rune) bool {
+	if !isHangul(r) {
+		return false
+	}
+	r -= HangulBase
+	return r < JamoLVTCount && r%JamoTCount == 0
+}
+
+func ccc(r rune) uint8 {
+	return chars[r].ccc
+}
+
+// Insert a rune in a buffer, ordered by Canonical Combining Class.
+func insertOrdered(b Decomposition, r rune) Decomposition {
+	n := len(b)
+	b = append(b, 0)
+	cc := ccc(r)
+	if cc > 0 {
+		// Use bubble sort.
+		for ; n > 0; n-- {
+			if ccc(b[n-1]) <= cc {
+				break
+			}
+			b[n] = b[n-1]
+		}
+	}
+	b[n] = r
+	return b
+}
+
+// Recursively decompose.
+func decomposeRecursive(form int, r rune, d Decomposition) Decomposition {
+	dcomp := chars[r].forms[form].decomp
+	if len(dcomp) == 0 {
+		return insertOrdered(d, r)
+	}
+	for _, c := range dcomp {
+		d = decomposeRecursive(form, c, d)
+	}
+	return d
+}
+
+func completeCharFields(form int) {
+	// Phase 0: pre-expand decomposition.
+	for i := range chars {
+		f := &chars[i].forms[form]
+		if len(f.decomp) == 0 {
+			continue
+		}
+		exp := make(Decomposition, 0)
+		for _, c := range f.decomp {
+			exp = decomposeRecursive(form, c, exp)
+		}
+		f.expandedDecomp = exp
+	}
+
+	// Phase 1: composition exclusion, mark decomposition.
+	for i := range chars {
+		c := &chars[i]
+		f := &c.forms[form]
+
+		// Marks script-specific exclusions and version restricted.
+		f.isOneWay = c.excludeInComp
+
+		// Singletons
+		f.isOneWay = f.isOneWay || len(f.decomp) == 1
+
+		// Non-starter decompositions
+		if len(f.decomp) > 1 {
+			chk := c.ccc != 0 || chars[f.decomp[0]].ccc != 0
+			f.isOneWay = f.isOneWay || chk
+		}
+
+		// Runes that decompose into more than two runes.
+		f.isOneWay = f.isOneWay || len(f.decomp) > 2
+
+		if form == FCompatibility {
+			f.isOneWay = f.isOneWay || hasCompatDecomp(c.codePoint)
+		}
+
+		for _, r := range f.decomp {
+			chars[r].forms[form].inDecomp = true
+		}
+	}
+
+	// Phase 2: forward and backward combining.
+	for i := range chars {
+		c := &chars[i]
+		f := &c.forms[form]
+
+		if !f.isOneWay && len(f.decomp) == 2 {
+			f0 := &chars[f.decomp[0]].forms[form]
+			f1 := &chars[f.decomp[1]].forms[form]
+			if !f0.isOneWay {
+				f0.combinesForward = true
+			}
+			if !f1.isOneWay {
+				f1.combinesBackward = true
+			}
+		}
+		if isHangulWithoutJamoT(rune(i)) {
+			f.combinesForward = true
+		}
+	}
+
+	// Phase 3: quick check values.
+	for i := range chars {
+		c := &chars[i]
+		f := &c.forms[form]
+
+		switch {
+		case len(f.decomp) > 0:
+			f.quickCheck[MDecomposed] = QCNo
+		case isHangul(rune(i)):
+			f.quickCheck[MDecomposed] = QCNo
+		default:
+			f.quickCheck[MDecomposed] = QCYes
+		}
+		switch {
+		case f.isOneWay:
+			f.quickCheck[MComposed] = QCNo
+		case (i & 0xffff00) == JamoLBase:
+			f.quickCheck[MComposed] = QCYes
+			if JamoLBase <= i && i < JamoLEnd {
+				f.combinesForward = true
+			}
+			if JamoVBase <= i && i < JamoVEnd {
+				f.quickCheck[MComposed] = QCMaybe
+				f.combinesBackward = true
+				f.combinesForward = true
+			}
+			if JamoTBase <= i && i < JamoTEnd {
+				f.quickCheck[MComposed] = QCMaybe
+				f.combinesBackward = true
+			}
+		case !f.combinesBackward:
+			f.quickCheck[MComposed] = QCYes
+		default:
+			f.quickCheck[MComposed] = QCMaybe
+		}
+	}
+}
+
+func computeNonStarterCounts() {
+	// Phase 4: leading and trailing non-starter count
+	for i := range chars {
+		c := &chars[i]
+
+		runes := []rune{rune(i)}
+		// We always use FCompatibility so that the CGJ insertion points do not
+		// change for repeated normalizations with different forms.
+		if exp := c.forms[FCompatibility].expandedDecomp; len(exp) > 0 {
+			runes = exp
+		}
+		// We consider runes that combine backwards to be non-starters for the
+		// purpose of Stream-Safe Text Processing.
+		for _, r := range runes {
+			if cr := &chars[r]; cr.ccc == 0 && !cr.forms[FCompatibility].combinesBackward {
+				break
+			}
+			c.nLeadingNonStarters++
+		}
+		for i := len(runes) - 1; i >= 0; i-- {
+			if cr := &chars[runes[i]]; cr.ccc == 0 && !cr.forms[FCompatibility].combinesBackward {
+				break
+			}
+			c.nTrailingNonStarters++
+		}
+		if c.nTrailingNonStarters > 3 {
+			log.Fatalf("%U: Decomposition with more than 3 (%d) trailing modifiers (%U)", i, c.nTrailingNonStarters, runes)
+		}
+
+		if isHangul(rune(i)) {
+			c.nTrailingNonStarters = 2
+			if isHangulWithoutJamoT(rune(i)) {
+				c.nTrailingNonStarters = 1
+			}
+		}
+
+		if l, t := c.nLeadingNonStarters, c.nTrailingNonStarters; l > 0 && l != t {
+			log.Fatalf("%U: number of leading and trailing non-starters should be equal (%d vs %d)", i, l, t)
+		}
+		if t := c.nTrailingNonStarters; t > 3 {
+			log.Fatalf("%U: number of trailing non-starters is %d > 3", t)
+		}
+	}
+}
+
+func printBytes(w io.Writer, b []byte, name string) {
+	fmt.Fprintf(w, "// %s: %d bytes\n", name, len(b))
+	fmt.Fprintf(w, "var %s = [...]byte {", name)
+	for i, c := range b {
+		switch {
+		case i%64 == 0:
+			fmt.Fprintf(w, "\n// Bytes %x - %x\n", i, i+63)
+		case i%8 == 0:
+			fmt.Fprintf(w, "\n")
+		}
+		fmt.Fprintf(w, "0x%.2X, ", c)
+	}
+	fmt.Fprint(w, "\n}\n\n")
+}
+
+// See forminfo.go for format.
+func makeEntry(f *FormInfo, c *Char) uint16 {
+	e := uint16(0)
+	if r := c.codePoint; HangulBase <= r && r < HangulEnd {
+		e |= 0x40
+	}
+	if f.combinesForward {
+		e |= 0x20
+	}
+	if f.quickCheck[MDecomposed] == QCNo {
+		e |= 0x4
+	}
+	switch f.quickCheck[MComposed] {
+	case QCYes:
+	case QCNo:
+		e |= 0x10
+	case QCMaybe:
+		e |= 0x18
+	default:
+		log.Fatalf("Illegal quickcheck value %v.", f.quickCheck[MComposed])
+	}
+	e |= uint16(c.nTrailingNonStarters)
+	return e
+}
+
+// decompSet keeps track of unique decompositions, grouped by whether
+// the decomposition is followed by a trailing and/or leading CCC.
+type decompSet [7]map[string]bool
+
+const (
+	normalDecomp = iota
+	firstMulti
+	firstCCC
+	endMulti
+	firstLeadingCCC
+	firstCCCZeroExcept
+	firstStarterWithNLead
+	lastDecomp
+)
+
+var cname = []string{"firstMulti", "firstCCC", "endMulti", "firstLeadingCCC", "firstCCCZeroExcept", "firstStarterWithNLead", "lastDecomp"}
+
+func makeDecompSet() decompSet {
+	m := decompSet{}
+	for i := range m {
+		m[i] = make(map[string]bool)
+	}
+	return m
+}
+func (m *decompSet) insert(key int, s string) {
+	m[key][s] = true
+}
+
+func printCharInfoTables(w io.Writer) int {
+	mkstr := func(r rune, f *FormInfo) (int, string) {
+		d := f.expandedDecomp
+		s := string([]rune(d))
+		if max := 1 << 6; len(s) >= max {
+			const msg = "%U: too many bytes in decomposition: %d >= %d"
+			log.Fatalf(msg, r, len(s), max)
+		}
+		head := uint8(len(s))
+		if f.quickCheck[MComposed] != QCYes {
+			head |= 0x40
+		}
+		if f.combinesForward {
+			head |= 0x80
+		}
+		s = string([]byte{head}) + s
+
+		lccc := ccc(d[0])
+		tccc := ccc(d[len(d)-1])
+		cc := ccc(r)
+		if cc != 0 && lccc == 0 && tccc == 0 {
+			log.Fatalf("%U: trailing and leading ccc are 0 for non-zero ccc %d", r, cc)
+		}
+		if tccc < lccc && lccc != 0 {
+			const msg = "%U: lccc (%d) must be <= tcc (%d)"
+			log.Fatalf(msg, r, lccc, tccc)
+		}
+		index := normalDecomp
+		nTrail := chars[r].nTrailingNonStarters
+		nLead := chars[r].nLeadingNonStarters
+		if tccc > 0 || lccc > 0 || nTrail > 0 {
+			tccc <<= 2
+			tccc |= nTrail
+			s += string([]byte{tccc})
+			index = endMulti
+			for _, r := range d[1:] {
+				if ccc(r) == 0 {
+					index = firstCCC
+				}
+			}
+			if lccc > 0 || nLead > 0 {
+				s += string([]byte{lccc})
+				if index == firstCCC {
+					log.Fatalf("%U: multi-segment decomposition not supported for decompositions with leading CCC != 0", r)
+				}
+				index = firstLeadingCCC
+			}
+			if cc != lccc {
+				if cc != 0 {
+					log.Fatalf("%U: for lccc != ccc, expected ccc to be 0; was %d", r, cc)
+				}
+				index = firstCCCZeroExcept
+			}
+		} else if len(d) > 1 {
+			index = firstMulti
+		}
+		return index, s
+	}
+
+	decompSet := makeDecompSet()
+	const nLeadStr = "\x00\x01" // 0-byte length and tccc with nTrail.
+	decompSet.insert(firstStarterWithNLead, nLeadStr)
+
+	// Store the uniqued decompositions in a byte buffer,
+	// preceded by their byte length.
+	for _, c := range chars {
+		for _, f := range c.forms {
+			if len(f.expandedDecomp) == 0 {
+				continue
+			}
+			if f.combinesBackward {
+				log.Fatalf("%U: combinesBackward and decompose", c.codePoint)
+			}
+			index, s := mkstr(c.codePoint, &f)
+			decompSet.insert(index, s)
+		}
+	}
+
+	decompositions := bytes.NewBuffer(make([]byte, 0, 10000))
+	size := 0
+	positionMap := make(map[string]uint16)
+	decompositions.WriteString("\000")
+	fmt.Fprintln(w, "const (")
+	for i, m := range decompSet {
+		sa := []string{}
+		for s := range m {
+			sa = append(sa, s)
+		}
+		sort.Strings(sa)
+		for _, s := range sa {
+			p := decompositions.Len()
+			decompositions.WriteString(s)
+			positionMap[s] = uint16(p)
+		}
+		if cname[i] != "" {
+			fmt.Fprintf(w, "%s = 0x%X\n", cname[i], decompositions.Len())
+		}
+	}
+	fmt.Fprintln(w, "maxDecomp = 0x8000")
+	fmt.Fprintln(w, ")")
+	b := decompositions.Bytes()
+	printBytes(w, b, "decomps")
+	size += len(b)
+
+	varnames := []string{"nfc", "nfkc"}
+	for i := 0; i < FNumberOfFormTypes; i++ {
+		trie := triegen.NewTrie(varnames[i])
+
+		for r, c := range chars {
+			f := c.forms[i]
+			d := f.expandedDecomp
+			if len(d) != 0 {
+				_, key := mkstr(c.codePoint, &f)
+				trie.Insert(rune(r), uint64(positionMap[key]))
+				if c.ccc != ccc(d[0]) {
+					// We assume the lead ccc of a decomposition !=0 in this case.
+					if ccc(d[0]) == 0 {
+						log.Fatalf("Expected leading CCC to be non-zero; ccc is %d", c.ccc)
+					}
+				}
+			} else if c.nLeadingNonStarters > 0 && len(f.expandedDecomp) == 0 && c.ccc == 0 && !f.combinesBackward {
+				// Handle cases where it can't be detected that the nLead should be equal
+				// to nTrail.
+				trie.Insert(c.codePoint, uint64(positionMap[nLeadStr]))
+			} else if v := makeEntry(&f, &c)<<8 | uint16(c.ccc); v != 0 {
+				trie.Insert(c.codePoint, uint64(0x8000|v))
+			}
+		}
+		sz, err := trie.Gen(w, triegen.Compact(&normCompacter{name: varnames[i]}))
+		if err != nil {
+			log.Fatal(err)
+		}
+		size += sz
+	}
+	return size
+}
+
+func contains(sa []string, s string) bool {
+	for _, a := range sa {
+		if a == s {
+			return true
+		}
+	}
+	return false
+}
+
+func makeTables() {
+	w := &bytes.Buffer{}
+
+	size := 0
+	if *tablelist == "" {
+		return
+	}
+	list := strings.Split(*tablelist, ",")
+	if *tablelist == "all" {
+		list = []string{"recomp", "info"}
+	}
+
+	// Compute maximum decomposition size.
+	max := 0
+	for _, c := range chars {
+		if n := len(string(c.forms[FCompatibility].expandedDecomp)); n > max {
+			max = n
+		}
+	}
+	fmt.Fprintln(w, `import "sync"`)
+	fmt.Fprintln(w)
+
+	fmt.Fprintln(w, "const (")
+	fmt.Fprintln(w, "\t// Version is the Unicode edition from which the tables are derived.")
+	fmt.Fprintf(w, "\tVersion = %q\n", gen.UnicodeVersion())
+	fmt.Fprintln(w)
+	fmt.Fprintln(w, "\t// MaxTransformChunkSize indicates the maximum number of bytes that Transform")
+	fmt.Fprintln(w, "\t// may need to write atomically for any Form. Making a destination buffer at")
+	fmt.Fprintln(w, "\t// least this size ensures that Transform can always make progress and that")
+	fmt.Fprintln(w, "\t// the user does not need to grow the buffer on an ErrShortDst.")
+	fmt.Fprintf(w, "\tMaxTransformChunkSize = %d+maxNonStarters*4\n", len(string(0x034F))+max)
+	fmt.Fprintln(w, ")\n")
+
+	// Print the CCC remap table.
+	size += len(cccMap)
+	fmt.Fprintf(w, "var ccc = [%d]uint8{", len(cccMap))
+	for i := 0; i < len(cccMap); i++ {
+		if i%8 == 0 {
+			fmt.Fprintln(w)
+		}
+		fmt.Fprintf(w, "%3d, ", cccMap[uint8(i)])
+	}
+	fmt.Fprintln(w, "\n}\n")
+
+	if contains(list, "info") {
+		size += printCharInfoTables(w)
+	}
+
+	if contains(list, "recomp") {
+		// Note that we use 32 bit keys, instead of 64 bit.
+		// This clips the bits of three entries, but we know
+		// this won't cause a collision. The compiler will catch
+		// any changes made to UnicodeData.txt that introduces
+		// a collision.
+		// Note that the recomposition map for NFC and NFKC
+		// are identical.
+
+		// Recomposition map
+		nrentries := 0
+		for _, c := range chars {
+			f := c.forms[FCanonical]
+			if !f.isOneWay && len(f.decomp) > 0 {
+				nrentries++
+			}
+		}
+		sz := nrentries * 8
+		size += sz
+		fmt.Fprintf(w, "// recompMap: %d bytes (entries only)\n", sz)
+		fmt.Fprintln(w, "var recompMap map[uint32]rune")
+		fmt.Fprintln(w, "var recompMapOnce sync.Once\n")
+		fmt.Fprintln(w, `const recompMapPacked = "" +`)
+		var buf [8]byte
+		for i, c := range chars {
+			f := c.forms[FCanonical]
+			d := f.decomp
+			if !f.isOneWay && len(d) > 0 {
+				key := uint32(uint16(d[0]))<<16 + uint32(uint16(d[1]))
+				binary.BigEndian.PutUint32(buf[:4], key)
+				binary.BigEndian.PutUint32(buf[4:], uint32(i))
+				fmt.Fprintf(w, "\t\t%q + // 0x%.8X: 0x%.8X\n", string(buf[:]), key, uint32(i))
+			}
+		}
+		// hack so we don't have to special case the trailing plus sign
+		fmt.Fprintf(w, `	""`)
+		fmt.Fprintln(w)
+	}
+
+	fmt.Fprintf(w, "// Total size of tables: %dKB (%d bytes)\n", (size+512)/1024, size)
+	gen.WriteVersionedGoFile("tables.go", "norm", w.Bytes())
+}
+
+func printChars() {
+	if *verbose {
+		for _, c := range chars {
+			if !c.isValid() || c.state == SMissing {
+				continue
+			}
+			fmt.Println(c)
+		}
+	}
+}
+
+// verifyComputed does various consistency tests.
+func verifyComputed() {
+	for i, c := range chars {
+		for _, f := range c.forms {
+			isNo := (f.quickCheck[MDecomposed] == QCNo)
+			if (len(f.decomp) > 0) != isNo && !isHangul(rune(i)) {
+				log.Fatalf("%U: NF*D QC must be No if rune decomposes", i)
+			}
+
+			isMaybe := f.quickCheck[MComposed] == QCMaybe
+			if f.combinesBackward != isMaybe {
+				log.Fatalf("%U: NF*C QC must be Maybe if combinesBackward", i)
+			}
+			if len(f.decomp) > 0 && f.combinesForward && isMaybe {
+				log.Fatalf("%U: NF*C QC must be Yes or No if combinesForward and decomposes", i)
+			}
+
+			if len(f.expandedDecomp) != 0 {
+				continue
+			}
+			if a, b := c.nLeadingNonStarters > 0, (c.ccc > 0 || f.combinesBackward); a != b {
+				// We accept these runes to be treated differently (it only affects
+				// segment breaking in iteration, most likely on improper use), but
+				// reconsider if more characters are added.
+				// U+FF9E HALFWIDTH KATAKANA VOICED SOUND MARK;Lm;0;L;<narrow> 3099;;;;N;;;;;
+				// U+FF9F HALFWIDTH KATAKANA SEMI-VOICED SOUND MARK;Lm;0;L;<narrow> 309A;;;;N;;;;;
+				// U+3133 HANGUL LETTER KIYEOK-SIOS;Lo;0;L;<compat> 11AA;;;;N;HANGUL LETTER GIYEOG SIOS;;;;
+				// U+318E HANGUL LETTER ARAEAE;Lo;0;L;<compat> 11A1;;;;N;HANGUL LETTER ALAE AE;;;;
+				// U+FFA3 HALFWIDTH HANGUL LETTER KIYEOK-SIOS;Lo;0;L;<narrow> 3133;;;;N;HALFWIDTH HANGUL LETTER GIYEOG SIOS;;;;
+				// U+FFDC HALFWIDTH HANGUL LETTER I;Lo;0;L;<narrow> 3163;;;;N;;;;;
+				if i != 0xFF9E && i != 0xFF9F && !(0x3133 <= i && i <= 0x318E) && !(0xFFA3 <= i && i <= 0xFFDC) {
+					log.Fatalf("%U: nLead was %v; want %v", i, a, b)
+				}
+			}
+		}
+		nfc := c.forms[FCanonical]
+		nfkc := c.forms[FCompatibility]
+		if nfc.combinesBackward != nfkc.combinesBackward {
+			log.Fatalf("%U: Cannot combine combinesBackward\n", c.codePoint)
+		}
+	}
+}
+
+// Use values in DerivedNormalizationProps.txt to compare against the
+// values we computed.
+// DerivedNormalizationProps.txt has form:
+// 00C0..00C5    ; NFD_QC; N # ...
+// 0374          ; NFD_QC; N # ...
+// See https://unicode.org/reports/tr44/ for full explanation
+func testDerived() {
+	f := gen.OpenUCDFile("DerivedNormalizationProps.txt")
+	defer f.Close()
+	p := ucd.New(f)
+	for p.Next() {
+		r := p.Rune(0)
+		c := &chars[r]
+
+		var ftype, mode int
+		qt := p.String(1)
+		switch qt {
+		case "NFC_QC":
+			ftype, mode = FCanonical, MComposed
+		case "NFD_QC":
+			ftype, mode = FCanonical, MDecomposed
+		case "NFKC_QC":
+			ftype, mode = FCompatibility, MComposed
+		case "NFKD_QC":
+			ftype, mode = FCompatibility, MDecomposed
+		default:
+			continue
+		}
+		var qr QCResult
+		switch p.String(2) {
+		case "Y":
+			qr = QCYes
+		case "N":
+			qr = QCNo
+		case "M":
+			qr = QCMaybe
+		default:
+			log.Fatalf(`Unexpected quick check value "%s"`, p.String(2))
+		}
+		if got := c.forms[ftype].quickCheck[mode]; got != qr {
+			log.Printf("%U: FAILED %s (was %v need %v)\n", r, qt, got, qr)
+		}
+		c.forms[ftype].verified[mode] = true
+	}
+	if err := p.Err(); err != nil {
+		log.Fatal(err)
+	}
+	// Any unspecified value must be QCYes. Verify this.
+	for i, c := range chars {
+		for j, fd := range c.forms {
+			for k, qr := range fd.quickCheck {
+				if !fd.verified[k] && qr != QCYes {
+					m := "%U: FAIL F:%d M:%d (was %v need Yes) %s\n"
+					log.Printf(m, i, j, k, qr, c.name)
+				}
+			}
+		}
+	}
+}
+
+var testHeader = `const (
+	Yes = iota
+	No
+	Maybe
+)
+
+type formData struct {
+	qc              uint8
+	combinesForward bool
+	decomposition   string
+}
+
+type runeData struct {
+	r      rune
+	ccc    uint8
+	nLead  uint8
+	nTrail uint8
+	f      [2]formData // 0: canonical; 1: compatibility
+}
+
+func f(qc uint8, cf bool, dec string) [2]formData {
+	return [2]formData{{qc, cf, dec}, {qc, cf, dec}}
+}
+
+func g(qc, qck uint8, cf, cfk bool, d, dk string) [2]formData {
+	return [2]formData{{qc, cf, d}, {qck, cfk, dk}}
+}
+
+var testData = []runeData{
+`
+
+func printTestdata() {
+	type lastInfo struct {
+		ccc    uint8
+		nLead  uint8
+		nTrail uint8
+		f      string
+	}
+
+	last := lastInfo{}
+	w := &bytes.Buffer{}
+	fmt.Fprintf(w, testHeader)
+	for r, c := range chars {
+		f := c.forms[FCanonical]
+		qc, cf, d := f.quickCheck[MComposed], f.combinesForward, string(f.expandedDecomp)
+		f = c.forms[FCompatibility]
+		qck, cfk, dk := f.quickCheck[MComposed], f.combinesForward, string(f.expandedDecomp)
+		s := ""
+		if d == dk && qc == qck && cf == cfk {
+			s = fmt.Sprintf("f(%s, %v, %q)", qc, cf, d)
+		} else {
+			s = fmt.Sprintf("g(%s, %s, %v, %v, %q, %q)", qc, qck, cf, cfk, d, dk)
+		}
+		current := lastInfo{c.ccc, c.nLeadingNonStarters, c.nTrailingNonStarters, s}
+		if last != current {
+			fmt.Fprintf(w, "\t{0x%x, %d, %d, %d, %s},\n", r, c.origCCC, c.nLeadingNonStarters, c.nTrailingNonStarters, s)
+			last = current
+		}
+	}
+	fmt.Fprintln(w, "}")
+	gen.WriteVersionedGoFile("data_test.go", "norm", w.Bytes())
+}
diff --git a/vendor/golang.org/x/text/unicode/norm/triegen.go b/vendor/golang.org/x/text/unicode/norm/triegen.go
new file mode 100644
index 0000000..45d7119
--- /dev/null
+++ b/vendor/golang.org/x/text/unicode/norm/triegen.go
@@ -0,0 +1,117 @@
+// Copyright 2011 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build ignore
+
+// Trie table generator.
+// Used by make*tables tools to generate a go file with trie data structures
+// for mapping UTF-8 to a 16-bit value. All but the last byte in a UTF-8 byte
+// sequence are used to lookup offsets in the index table to be used for the
+// next byte. The last byte is used to index into a table with 16-bit values.
+
+package main
+
+import (
+	"fmt"
+	"io"
+)
+
+const maxSparseEntries = 16
+
+type normCompacter struct {
+	sparseBlocks [][]uint64
+	sparseOffset []uint16
+	sparseCount  int
+	name         string
+}
+
+func mostFrequentStride(a []uint64) int {
+	counts := make(map[int]int)
+	var v int
+	for _, x := range a {
+		if stride := int(x) - v; v != 0 && stride >= 0 {
+			counts[stride]++
+		}
+		v = int(x)
+	}
+	var maxs, maxc int
+	for stride, cnt := range counts {
+		if cnt > maxc || (cnt == maxc && stride < maxs) {
+			maxs, maxc = stride, cnt
+		}
+	}
+	return maxs
+}
+
+func countSparseEntries(a []uint64) int {
+	stride := mostFrequentStride(a)
+	var v, count int
+	for _, tv := range a {
+		if int(tv)-v != stride {
+			if tv != 0 {
+				count++
+			}
+		}
+		v = int(tv)
+	}
+	return count
+}
+
+func (c *normCompacter) Size(v []uint64) (sz int, ok bool) {
+	if n := countSparseEntries(v); n <= maxSparseEntries {
+		return (n+1)*4 + 2, true
+	}
+	return 0, false
+}
+
+func (c *normCompacter) Store(v []uint64) uint32 {
+	h := uint32(len(c.sparseOffset))
+	c.sparseBlocks = append(c.sparseBlocks, v)
+	c.sparseOffset = append(c.sparseOffset, uint16(c.sparseCount))
+	c.sparseCount += countSparseEntries(v) + 1
+	return h
+}
+
+func (c *normCompacter) Handler() string {
+	return c.name + "Sparse.lookup"
+}
+
+func (c *normCompacter) Print(w io.Writer) (retErr error) {
+	p := func(f string, x ...interface{}) {
+		if _, err := fmt.Fprintf(w, f, x...); retErr == nil && err != nil {
+			retErr = err
+		}
+	}
+
+	ls := len(c.sparseBlocks)
+	p("// %sSparseOffset: %d entries, %d bytes\n", c.name, ls, ls*2)
+	p("var %sSparseOffset = %#v\n\n", c.name, c.sparseOffset)
+
+	ns := c.sparseCount
+	p("// %sSparseValues: %d entries, %d bytes\n", c.name, ns, ns*4)
+	p("var %sSparseValues = [%d]valueRange {", c.name, ns)
+	for i, b := range c.sparseBlocks {
+		p("\n// Block %#x, offset %#x", i, c.sparseOffset[i])
+		var v int
+		stride := mostFrequentStride(b)
+		n := countSparseEntries(b)
+		p("\n{value:%#04x,lo:%#02x},", stride, uint8(n))
+		for i, nv := range b {
+			if int(nv)-v != stride {
+				if v != 0 {
+					p(",hi:%#02x},", 0x80+i-1)
+				}
+				if nv != 0 {
+					p("\n{value:%#04x,lo:%#02x", nv, 0x80+i)
+				}
+			}
+			v = int(nv)
+		}
+		if v != 0 {
+			p(",hi:%#02x},", 0x80+len(b)-1)
+		}
+	}
+	p("\n}\n\n")
+	return
+}
diff --git a/vendor/modules.txt b/vendor/modules.txt
index 8ddd284..40664de 100644
--- a/vendor/modules.txt
+++ b/vendor/modules.txt
@@ -3,8 +3,8 @@
 # github.com/Shopify/sarama v1.23.1
 github.com/Shopify/sarama
 # github.com/aead/cmac v0.0.0-20160719120800-7af84192f0b1
-github.com/aead/cmac
 github.com/aead/cmac/aes
+github.com/aead/cmac
 # github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878
 github.com/armon/go-metrics
 # github.com/bsm/sarama-cluster v2.1.15+incompatible
@@ -13,6 +13,8 @@
 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
+github.com/cevaris/ordered_map
 # github.com/coreos/go-systemd v0.0.0-20190620071333-e64a0ec8b42a
 github.com/coreos/go-systemd/journal
 # github.com/coreos/pkg v0.0.0-20180108230652-97fdf19511ea
@@ -28,17 +30,17 @@
 # github.com/eapache/queue v1.1.0
 github.com/eapache/queue
 # github.com/gogo/protobuf v1.3.1
-github.com/gogo/protobuf/gogoproto
 github.com/gogo/protobuf/proto
+github.com/gogo/protobuf/gogoproto
 github.com/gogo/protobuf/protoc-gen-gogo/descriptor
 # github.com/golang/protobuf v1.3.2
-github.com/golang/protobuf/proto
-github.com/golang/protobuf/protoc-gen-go/descriptor
 github.com/golang/protobuf/ptypes
+github.com/golang/protobuf/proto
 github.com/golang/protobuf/ptypes/any
-github.com/golang/protobuf/ptypes/duration
 github.com/golang/protobuf/ptypes/empty
 github.com/golang/protobuf/ptypes/timestamp
+github.com/golang/protobuf/ptypes/duration
+github.com/golang/protobuf/protoc-gen-go/descriptor
 # github.com/golang/snappy v0.0.1
 github.com/golang/snappy
 # github.com/google/gopacket v1.1.17
@@ -76,19 +78,22 @@
 github.com/opencord/voltha-lib-go/v3/pkg/adapters
 github.com/opencord/voltha-lib-go/v3/pkg/adapters/adapterif
 github.com/opencord/voltha-lib-go/v3/pkg/adapters/common
-github.com/opencord/voltha-lib-go/v3/pkg/db
+github.com/opencord/voltha-lib-go/v3/pkg/config
 github.com/opencord/voltha-lib-go/v3/pkg/db/kvstore
 github.com/opencord/voltha-lib-go/v3/pkg/kafka
 github.com/opencord/voltha-lib-go/v3/pkg/log
 github.com/opencord/voltha-lib-go/v3/pkg/probe
+github.com/opencord/voltha-lib-go/v3/pkg/version
+github.com/opencord/voltha-lib-go/v3/pkg/flows
+github.com/opencord/voltha-lib-go/v3/pkg/db
 # github.com/opencord/voltha-protos/v3 v3.3.0
-github.com/opencord/voltha-protos/v3/go/common
 github.com/opencord/voltha-protos/v3/go/inter_container
-github.com/opencord/voltha-protos/v3/go/omci
+github.com/opencord/voltha-protos/v3/go/voltha
+github.com/opencord/voltha-protos/v3/go/common
 github.com/opencord/voltha-protos/v3/go/openflow_13
 github.com/opencord/voltha-protos/v3/go/openolt
+github.com/opencord/voltha-protos/v3/go/omci
 github.com/opencord/voltha-protos/v3/go/tech_profile
-github.com/opencord/voltha-protos/v3/go/voltha
 # github.com/pierrec/lz4 v2.3.0+incompatible
 github.com/pierrec/lz4
 github.com/pierrec/lz4/internal/xxh32
@@ -99,20 +104,20 @@
 # github.com/stretchr/testify v1.5.1
 github.com/stretchr/testify/assert
 # go.etcd.io/etcd v0.0.0-20190930204107-236ac2a90522
-go.etcd.io/etcd/auth/authpb
 go.etcd.io/etcd/clientv3
+go.etcd.io/etcd/clientv3/concurrency
+go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes
+go.etcd.io/etcd/auth/authpb
 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/concurrency
 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/clientv3/balancer/connectivity
+go.etcd.io/etcd/pkg/systemd
 go.etcd.io/etcd/raft
 go.etcd.io/etcd/raft/confchange
 go.etcd.io/etcd/raft/quorum
@@ -124,49 +129,50 @@
 go.uber.org/multierr
 # go.uber.org/zap v1.10.0
 go.uber.org/zap
-go.uber.org/zap/buffer
+go.uber.org/zap/zapcore
 go.uber.org/zap/internal/bufferpool
+go.uber.org/zap/buffer
 go.uber.org/zap/internal/color
 go.uber.org/zap/internal/exit
-go.uber.org/zap/zapcore
 # golang.org/x/crypto v0.0.0-20200221231518-2aa609cf4a9d
 golang.org/x/crypto/md4
 golang.org/x/crypto/pbkdf2
 # golang.org/x/net v0.0.0-20200222125558-5a598a2470a0
-golang.org/x/net/context
-golang.org/x/net/http/httpguts
-golang.org/x/net/http2
-golang.org/x/net/http2/hpack
-golang.org/x/net/idna
-golang.org/x/net/internal/socks
-golang.org/x/net/internal/timeseries
 golang.org/x/net/proxy
 golang.org/x/net/trace
+golang.org/x/net/context
+golang.org/x/net/internal/socks
+golang.org/x/net/internal/timeseries
+golang.org/x/net/http2
+golang.org/x/net/http2/hpack
+golang.org/x/net/http/httpguts
+golang.org/x/net/idna
 # golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae
 golang.org/x/sys/unix
 # golang.org/x/text v0.3.2
 golang.org/x/text/secure/bidirule
-golang.org/x/text/transform
 golang.org/x/text/unicode/bidi
 golang.org/x/text/unicode/norm
+golang.org/x/text/transform
 # google.golang.org/genproto v0.0.0-20190927181202-20e1ac93f88c
 google.golang.org/genproto/googleapis/api/annotations
 google.golang.org/genproto/googleapis/rpc/status
 # google.golang.org/grpc v1.25.1
+google.golang.org/grpc/codes
+google.golang.org/grpc/status
 google.golang.org/grpc
+google.golang.org/grpc/internal
+google.golang.org/grpc/credentials
+google.golang.org/grpc/grpclog
+google.golang.org/grpc/keepalive
+google.golang.org/grpc/metadata
 google.golang.org/grpc/backoff
 google.golang.org/grpc/balancer
 google.golang.org/grpc/balancer/base
 google.golang.org/grpc/balancer/roundrobin
-google.golang.org/grpc/binarylog/grpc_binarylog_v1
-google.golang.org/grpc/codes
 google.golang.org/grpc/connectivity
-google.golang.org/grpc/credentials
-google.golang.org/grpc/credentials/internal
 google.golang.org/grpc/encoding
 google.golang.org/grpc/encoding/proto
-google.golang.org/grpc/grpclog
-google.golang.org/grpc/internal
 google.golang.org/grpc/internal/backoff
 google.golang.org/grpc/internal/balancerload
 google.golang.org/grpc/internal/binarylog
@@ -177,19 +183,18 @@
 google.golang.org/grpc/internal/grpcsync
 google.golang.org/grpc/internal/resolver/dns
 google.golang.org/grpc/internal/resolver/passthrough
-google.golang.org/grpc/internal/syscall
 google.golang.org/grpc/internal/transport
-google.golang.org/grpc/keepalive
-google.golang.org/grpc/metadata
 google.golang.org/grpc/naming
 google.golang.org/grpc/peer
 google.golang.org/grpc/resolver
-google.golang.org/grpc/resolver/dns
-google.golang.org/grpc/resolver/passthrough
 google.golang.org/grpc/serviceconfig
 google.golang.org/grpc/stats
-google.golang.org/grpc/status
 google.golang.org/grpc/tap
+google.golang.org/grpc/resolver/dns
+google.golang.org/grpc/resolver/passthrough
+google.golang.org/grpc/credentials/internal
+google.golang.org/grpc/binarylog/grpc_binarylog_v1
+google.golang.org/grpc/internal/syscall
 # gopkg.in/jcmturner/aescts.v1 v1.0.1
 gopkg.in/jcmturner/aescts.v1
 # gopkg.in/jcmturner/dnsutils.v1 v1.0.1
@@ -199,32 +204,32 @@
 gopkg.in/jcmturner/gokrb5.v7/client
 gopkg.in/jcmturner/gokrb5.v7/config
 gopkg.in/jcmturner/gokrb5.v7/credentials
+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/keytab
+gopkg.in/jcmturner/gokrb5.v7/messages
+gopkg.in/jcmturner/gokrb5.v7/types
 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/iana/errorcode
+gopkg.in/jcmturner/gokrb5.v7/iana/flags
+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/krberror
+gopkg.in/jcmturner/gokrb5.v7/iana/etypeID
+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/msgtype
+gopkg.in/jcmturner/gokrb5.v7/pac
+gopkg.in/jcmturner/gokrb5.v7/iana/addrtype
+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/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