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, ®istry)
+ 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