blob: 651ed1a6cf096ed2616d4365410595240da33b7d [file] [log] [blame]
#
# Copyright 2018 the original author or authors.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
import copy
from twisted.internet import reactor
import grpc
from google.protobuf.json_format import MessageToDict
import hashlib
from simplejson import dumps
from twisted.internet.defer import inlineCallbacks, returnValue
from voltha_protos.openflow_13_pb2 import OFPXMC_OPENFLOW_BASIC, \
ofp_flow_stats, OFPMT_OXM, Flows, FlowGroups, OFPXMT_OFB_IN_PORT, \
OFPXMT_OFB_VLAN_VID
from voltha_protos.device_pb2 import Port
import pyvoltha.common.openflow.utils as fd
from voltha_protos import openolt_pb2
from voltha_protos.inter_container_pb2 import SwitchCapability, PortCapability, \
InterAdapterMessageType, InterAdapterOmciMessage, InterAdapterTechProfileDownloadMessage
from pyvoltha.common.tech_profile.tech_profile import DEFAULT_TECH_PROFILE_TABLE_ID
# Flow categories
HSIA_FLOW = "HSIA_FLOW"
EAP_ETH_TYPE = 0x888e
LLDP_ETH_TYPE = 0x88cc
IGMP_PROTO = 2
# FIXME - see also BRDCM_DEFAULT_VLAN in broadcom_onu.py
DEFAULT_MGMT_VLAN = 4091
# Openolt Flow
UPSTREAM = "upstream"
DOWNSTREAM = "downstream"
PACKET_TAG_TYPE = "pkt_tag_type"
UNTAGGED = "untagged"
SINGLE_TAG = "single_tag"
DOUBLE_TAG = "double_tag"
# Classifier
ETH_TYPE = 'eth_type'
TPID = 'tpid'
IP_PROTO = 'ip_proto'
IN_PORT = 'in_port'
VLAN_VID = 'vlan_vid'
VLAN_PCP = 'vlan_pcp'
UDP_DST = 'udp_dst'
UDP_SRC = 'udp_src'
IPV4_DST = 'ipv4_dst'
IPV4_SRC = 'ipv4_src'
METADATA = 'metadata'
TUNNEL_ID = 'tunnel_id'
OUTPUT = 'output'
# Action
POP_VLAN = 'pop_vlan'
PUSH_VLAN = 'push_vlan'
TRAP_TO_HOST = 'trap_to_host'
class OpenOltFlowMgr(object):
def __init__(self, core_proxy, adapter_proxy, log, stub, device_id, logical_device_id,
platform, resource_mgr):
self.core_proxy = core_proxy
self.adapter_proxy = adapter_proxy
self.log = log
self.stub = stub
self.device_id = device_id
self.logical_device_id = logical_device_id
self.nni_intf_id = None
self.platform = platform
self.resource_mgr = resource_mgr
self.tech_profile = dict()
self._populate_tech_profile_per_pon_port()
self.retry_add_flow_list = []
def add_flow(self, flow):
self.log.debug('add flow', flow=flow)
classifier_info = dict()
action_info = dict()
for field in fd.get_ofb_fields(flow):
if field.type == fd.ETH_TYPE:
classifier_info[ETH_TYPE] = field.eth_type
self.log.debug('field-type-eth-type',
eth_type=classifier_info[ETH_TYPE])
elif field.type == fd.IP_PROTO:
classifier_info[IP_PROTO] = field.ip_proto
self.log.debug('field-type-ip-proto',
ip_proto=classifier_info[IP_PROTO])
elif field.type == fd.IN_PORT:
classifier_info[IN_PORT] = field.port
self.log.debug('field-type-in-port',
in_port=classifier_info[IN_PORT])
elif field.type == fd.VLAN_VID:
classifier_info[VLAN_VID] = field.vlan_vid & 0xfff
self.log.debug('field-type-vlan-vid',
vlan=classifier_info[VLAN_VID])
elif field.type == fd.VLAN_PCP:
classifier_info[VLAN_PCP] = field.vlan_pcp
self.log.debug('field-type-vlan-pcp',
pcp=classifier_info[VLAN_PCP])
elif field.type == fd.UDP_DST:
classifier_info[UDP_DST] = field.udp_dst
self.log.debug('field-type-udp-dst',
udp_dst=classifier_info[UDP_DST])
elif field.type == fd.UDP_SRC:
classifier_info[UDP_SRC] = field.udp_src
self.log.debug('field-type-udp-src',
udp_src=classifier_info[UDP_SRC])
elif field.type == fd.IPV4_DST:
classifier_info[IPV4_DST] = field.ipv4_dst
self.log.debug('field-type-ipv4-dst',
ipv4_dst=classifier_info[IPV4_DST])
elif field.type == fd.IPV4_SRC:
classifier_info[IPV4_SRC] = field.ipv4_src
self.log.debug('field-type-ipv4-src',
ipv4_dst=classifier_info[IPV4_SRC])
elif field.type == fd.METADATA:
classifier_info[METADATA] = field.table_metadata
self.log.debug('field-type-metadata',
metadata=classifier_info[METADATA])
elif field.type == fd.TUNNEL_ID:
classifier_info[TUNNEL_ID] = field.tunnel_id
self.log.debug('field-type-tunnel-id',
tunnel_id=classifier_info[TUNNEL_ID])
else:
raise NotImplementedError('field.type={}'.format(
field.type))
for action in fd.get_actions(flow):
if action.type == fd.OUTPUT:
action_info[OUTPUT] = action.output.port
self.log.debug('action-type-output',
output=action_info[OUTPUT],
in_port=classifier_info[IN_PORT])
elif action.type == fd.POP_VLAN:
action_info[POP_VLAN] = True
self.log.debug('action-type-pop-vlan',
in_port=classifier_info[IN_PORT])
elif action.type == fd.PUSH_VLAN:
action_info[PUSH_VLAN] = True
action_info[TPID] = action.push.ethertype
self.log.debug('action-type-push-vlan',
push_tpid=action_info[TPID], in_port=classifier_info[IN_PORT])
if action.push.ethertype != 0x8100:
self.log.error('unhandled-tpid',
ethertype=action.push.ethertype)
elif action.type == fd.SET_FIELD:
# action_info['action_type'] = 'set_field'
_field = action.set_field.field.ofb_field
assert (action.set_field.field.oxm_class ==
OFPXMC_OPENFLOW_BASIC)
self.log.debug('action-type-set-field',
field=_field, in_port=classifier_info[IN_PORT])
if _field.type == fd.VLAN_VID:
self.log.debug('set-field-type-vlan-vid',
vlan_vid=_field.vlan_vid & 0xfff)
action_info[VLAN_VID] = (_field.vlan_vid & 0xfff)
else:
self.log.error('unsupported-action-set-field-type',
field_type=_field.type)
else:
self.log.error('unsupported-action-type',
action_type=action.type, in_port=classifier_info[IN_PORT])
# controller bound trap flows.
if self.platform.is_controller(action_info[OUTPUT]):
# trap from pon port. figure out uni port from tunnel id and set as in port
if self.platform.intf_id_to_port_type_name(classifier_info[IN_PORT]) == Port.PON_OLT:
if fd.get_child_port_from_tunnelid(flow) is not None:
classifier_info[IN_PORT] = fd.get_child_port_from_tunnelid(flow)
self.log.debug('pon-to-controller-flow-port-in-tunnelid', new_in_port=classifier_info[IN_PORT])
else:
self.log.debug('pon-to-controller-flow-NO-PORT-in-tunnelid', in_port=classifier_info[IN_PORT],
out_port=action_info[OUTPUT])
# TODO NEW CORE: trap from nni port.
else:
self.log.warn('nni-to-controller-trap-unsupported', flow=flow)
else:
# do not operate on the private decomposer vlan. neither onu nor agg switch adds it.
# cannot do anything with this flow
if VLAN_VID in classifier_info and classifier_info[VLAN_VID] == 4000:
self.log.debug('skipping-private-vlan', in_port=classifier_info[IN_PORT],
out_port=action_info[OUTPUT])
return
# downstream nni port to pon port. figure out uni port from tunnel id and set as out port
if self.platform.intf_id_to_port_type_name(action_info[OUTPUT]) == Port.PON_OLT:
if fd.get_child_port_from_tunnelid(flow) is not None:
action_info[OUTPUT] = fd.get_child_port_from_tunnelid(flow)
self.log.debug('downstream-pon-port-flow-port-in-tunnelid', new_out_port=action_info[OUTPUT])
else:
self.log.debug('downstream-pon-port-flow-NO-PORT-in-tunnelid', in_port=classifier_info[IN_PORT],
out_port=action_info[OUTPUT])
return
# upstream pon port to nni port. figure out uni port from tunnel id and set as in port
if self.platform.intf_id_to_port_type_name(classifier_info[IN_PORT]) == Port.PON_OLT:
if fd.get_child_port_from_tunnelid(flow) is not None:
classifier_info[IN_PORT] = fd.get_child_port_from_tunnelid(flow)
self.log.debug('upstream-pon-port-flow-port-in-tunnelid', new_in_port=classifier_info[IN_PORT])
else:
self.log.debug('upstream-pon-port-flow-NO-PORT-in-tunnelid', in_port=classifier_info[IN_PORT],
out_port=action_info[OUTPUT])
return
self.log.debug('flow-ports', classifier_inport=classifier_info[IN_PORT], action_output=action_info[OUTPUT])
(port_no, intf_id, onu_id, uni_id) = self.platform.extract_access_from_flow(
classifier_info[IN_PORT], action_info[OUTPUT])
self.log.debug('extracted-flow-ports', port_no=port_no, intf_id=intf_id, onu_id=onu_id, uni_id=uni_id)
# TODO NEW CORE: this needs to be broken into onu type flows that need these ID, and NNI flows that do not
self.divide_and_add_flow(intf_id, onu_id, uni_id, port_no, classifier_info,
action_info, flow)
def _clear_flow_id_from_rm(self, flow, flow_id, flow_direction):
uni_port_no = None
child_device_id = None
if flow_direction == UPSTREAM:
for field in fd.get_ofb_fields(flow):
if field.type == fd.IN_PORT:
is_uni, child_device_id = self._is_uni_port(field.port)
if is_uni:
uni_port_no = field.port
elif flow_direction == DOWNSTREAM:
for field in fd.get_ofb_fields(flow):
if field.type == fd.METADATA:
uni_port = field.table_metadata & 0xFFFFFFFF
is_uni, child_device_id = self._is_uni_port(uni_port)
if is_uni:
uni_port_no = field.port
if uni_port_no is None:
for action in fd.get_actions(flow):
if action.type == fd.OUTPUT:
is_uni, child_device_id = \
self._is_uni_port(action.output.port)
if is_uni:
uni_port_no = action.output.port
if child_device_id:
child_device = self.adapter_agent.get_device(child_device_id)
pon_intf = child_device.proxy_address.channel_id
onu_id = child_device.proxy_address.onu_id
uni_id = self.platform.uni_id_from_port_num(uni_port_no) if uni_port_no is not None else None
flows = self.resource_mgr.get_flow_id_info(pon_intf, onu_id, uni_id, flow_id)
assert (isinstance(flows, list))
self.log.debug("retrieved-flows", flows=flows)
for idx in range(len(flows)):
if flow_direction == flows[idx]['flow_type']:
flows.pop(idx)
self.update_flow_info_to_kv_store(pon_intf, onu_id, uni_id, flow_id, flows)
if len(flows) > 0:
# There are still flows referencing the same flow_id.
# So the flow should not be freed yet.
# For ex: Case of HSIA where same flow is shared
# between DS and US.
return
self.resource_mgr.free_flow_id_for_uni(pon_intf, onu_id, uni_id, flow_id)
else:
self.log.error("invalid-info", uni_port_no=uni_port_no,
child_device_id=child_device_id)
def retry_add_flow(self, flow):
self.log.debug("retry-add-flow")
if flow.id in self.retry_add_flow_list:
self.retry_add_flow_list.remove(flow.id)
self.add_flow(flow)
def remove_flow(self, flow):
self.log.debug('trying to remove flows from logical flow :',
logical_flow=flow)
# TODO NEW CORE: Keep track of device flows locally. need new array
device_flows_to_remove = []
#device_flows = self.flows_proxy.get('/').items
device_flows = []
for f in device_flows:
if f.cookie == flow.id:
device_flows_to_remove.append(f)
for f in device_flows_to_remove:
(id, direction) = self.decode_stored_id(f.id)
flow_to_remove = openolt_pb2.Flow(flow_id=id, flow_type=direction)
try:
self.stub.FlowRemove(flow_to_remove)
except grpc.RpcError as grpc_e:
if grpc_e.code() == grpc.StatusCode.NOT_FOUND:
self.log.debug('This flow does not exist on the switch, '
'normal after an OLT reboot',
flow=flow_to_remove)
else:
raise grpc_e
# once we have successfully deleted the flow on the device
# release the flow_id on resource pool and also clear any
# data associated with the flow_id on KV store.
self._clear_flow_id_from_rm(f, id, direction)
self.log.debug('flow removed from device', flow=f,
flow_key=flow_to_remove)
def get_tp_path(self, intf_id, uni):
# FIXME Should get Table id form the flow, as of now hardcoded to
# DEFAULT_TECH_PROFILE_TABLE_ID (64)
# 'tp_path' contains the suffix part of the tech_profile_instance path.
# The prefix to the 'tp_path' should be set to \
# TechProfile.KV_STORE_TECH_PROFILE_PATH_PREFIX by the ONU adapter.
return self.tech_profile[intf_id]. \
get_tp_path(DEFAULT_TECH_PROFILE_TABLE_ID,
uni)
def delete_tech_profile_instance(self, intf_id, onu_id, uni_id):
# Remove the TP instance associated with the ONU
ofp_port_name = self._get_ofp_port_name(intf_id, onu_id, uni_id)
tp_path = self.get_tp_path(intf_id, ofp_port_name)
return self.tech_profile[intf_id].delete_tech_profile_instance(tp_path)
@inlineCallbacks
def divide_and_add_flow(self, intf_id, onu_id, uni_id, port_no, classifier,
action, flow):
self.log.debug('sorting flow', intf_id=intf_id, onu_id=onu_id, uni_id=uni_id, port_no=port_no,
classifier=classifier, action=action)
uni = self.get_uni_port_path(intf_id, onu_id, uni_id)
# TODO: if there is no onu_id then the flows pushed are NNI based and belows flow pushes need to be refactored
if (onu_id > 0):
alloc_id, gem_ports = self.create_tcont_gemport(intf_id, onu_id, uni_id,
uni, port_no, flow.table_id)
if alloc_id is None or gem_ports is None:
self.log.error("alloc-id-gem-ports-unavailable", alloc_id=alloc_id, gem_ports=gem_ports)
return
self.log.debug('Generated required alloc and gemport ids', alloc_id=alloc_id, gemports=gem_ports)
else:
alloc_id = -1
gem_ports = []
self.log.error('cannot-generate-alloc-gem-id-for-flow', intf_id=intf_id, onu_id=onu_id, uni_id=uni_id)
# TODO: if there are no usable onu, uni, alloc, or gem id, then the below flows are erroneous. this needs a refactor
# Flows can't be added specific to gemport unless p-bits are received.
# Hence adding flows for all gemports
for gemport_id in gem_ports:
if IP_PROTO in classifier:
if classifier[IP_PROTO] == 17:
self.log.debug('dhcp flow add')
self.add_dhcp_trap(intf_id, onu_id, uni_id, port_no, classifier,
action, flow, alloc_id, gemport_id)
elif classifier[IP_PROTO] == 2:
self.log.warn('igmp flow add ignored, not implemented yet')
else:
self.log.warn("Invalid-Classifier-to-handle",
classifier=classifier,
action=action)
elif ETH_TYPE in classifier:
if classifier[ETH_TYPE] == EAP_ETH_TYPE:
self.log.debug('eapol flow add')
self.add_eapol_flow(intf_id, onu_id, uni_id, port_no, flow, alloc_id,
gemport_id)
vlan_id = yield self.get_subscriber_vlan(fd.get_child_port_from_tunnelid(flow))
if vlan_id is not None:
self.log.debug('adding-supplimental-eap-flow-vlan', vlan_id=vlan_id)
self.add_eapol_flow(
intf_id, onu_id, uni_id, port_no, flow, alloc_id, gemport_id,
vlan_id=vlan_id)
parent_port_no = self.platform.intf_id_to_port_no(intf_id, Port.PON_OLT)
self.log.debug('get-child-device', intf_id=intf_id, onu_id=onu_id,
parent_port_no=parent_port_no, device_id=self.device_id)
onu_device = yield self.core_proxy.get_child_device(self.device_id,
onu_id=int(onu_id),
parent_port_no=int(parent_port_no))
tp_path = self.get_tp_path(intf_id, uni)
tech_msg = InterAdapterTechProfileDownloadMessage(uni_id=uni_id, path=tp_path)
self.log.debug('Load-tech-profile-request-to-brcm-handler',
onu_device=onu_device, tp_path=tp_path, tech_msg=tech_msg)
# Send the tech profile event to the onu adapter
yield self.adapter_proxy.send_inter_adapter_message(
msg=tech_msg,
type=InterAdapterMessageType.TECH_PROFILE_DOWNLOAD_REQUEST,
from_adapter="openolt",
to_adapter=onu_device.type,
to_device_id=onu_device.id
)
if classifier[ETH_TYPE] == LLDP_ETH_TYPE:
self.log.debug('lldp flow add')
nni_intf_id = yield self.get_nni_intf_id()
self.add_lldp_flow(flow, port_no, nni_intf_id)
elif PUSH_VLAN in action:
self.add_upstream_data_flow(intf_id, onu_id, uni_id, port_no, classifier,
action, flow, alloc_id, gemport_id)
elif POP_VLAN in action:
self.add_downstream_data_flow(intf_id, onu_id, uni_id, port_no, classifier,
action, flow, alloc_id, gemport_id)
else:
self.log.debug('Invalid-flow-type-to-handle',
classifier=classifier,
action=action, flow=flow)
def get_uni_port_path(self, intf_id, onu_id, uni_id):
value = 'pon-{}/onu-{}/uni-{}'.format(intf_id, onu_id, uni_id)
return value
def create_tcont_gemport(self, intf_id, onu_id, uni_id, uni, port_no, table_id):
alloc_id, gem_port_ids = None, None
pon_intf_onu_id = (intf_id, onu_id)
# If we already have allocated alloc_id and gem_ports earlier, render them
alloc_id = \
self.resource_mgr.get_current_alloc_ids_for_onu(pon_intf_onu_id)
gem_port_ids = \
self.resource_mgr.get_current_gemport_ids_for_onu(pon_intf_onu_id)
if alloc_id is not None and gem_port_ids is not None:
return alloc_id, gem_port_ids
try:
# FIXME: If table id is <= 63 using 64 as table id
if table_id < DEFAULT_TECH_PROFILE_TABLE_ID:
table_id = DEFAULT_TECH_PROFILE_TABLE_ID
# Check tech profile instance already exists for derived port name
tech_profile_instance = self.tech_profile[intf_id]. \
get_tech_profile_instance(table_id, uni)
self.log.debug('Get-tech-profile-instance-status', tech_profile_instance=tech_profile_instance)
if tech_profile_instance is None:
# create tech profile instance
tech_profile_instance = self.tech_profile[intf_id]. \
create_tech_profile_instance(table_id, uni,
intf_id)
if tech_profile_instance is None:
raise Exception('Tech-profile-instance-creation-failed')
else:
self.log.debug(
'Tech-profile-instance-already-exist-for-given port-name',
table_id=table_id, intf_id=intf_id, uni=uni)
# upstream scheduler
us_scheduler = self.tech_profile[intf_id].get_us_scheduler(
tech_profile_instance)
# downstream scheduler
ds_scheduler = self.tech_profile[intf_id].get_ds_scheduler(
tech_profile_instance)
# create Tcont
tconts = self.tech_profile[intf_id].get_tconts(tech_profile_instance,
us_scheduler,
ds_scheduler)
self.stub.CreateTconts(openolt_pb2.Tconts(intf_id=intf_id,
onu_id=onu_id,
uni_id=uni_id,
port_no=port_no,
tconts=tconts))
# Fetch alloc id and gemports from tech profile instance
alloc_id = tech_profile_instance.us_scheduler.alloc_id
gem_port_ids = []
for i in range(len(
tech_profile_instance.upstream_gem_port_attribute_list)):
gem_port_ids.append(
tech_profile_instance.upstream_gem_port_attribute_list[i].
gemport_id)
except BaseException as e:
self.log.exception(exception=e)
# Update the allocated alloc_id and gem_port_id for the ONU/UNI to KV store
pon_intf_onu_id = (intf_id, onu_id, uni_id)
self.resource_mgr.resource_mgrs[intf_id].update_alloc_ids_for_onu(
pon_intf_onu_id,
list([alloc_id])
)
self.resource_mgr.resource_mgrs[intf_id].update_gemport_ids_for_onu(
pon_intf_onu_id,
gem_port_ids
)
self.resource_mgr.update_gemports_ponport_to_onu_map_on_kv_store(
gem_port_ids, intf_id, onu_id, uni_id
)
return alloc_id, gem_port_ids
@inlineCallbacks
def add_upstream_data_flow(self, intf_id, onu_id, uni_id, port_no, uplink_classifier,
uplink_action, logical_flow, alloc_id,
gemport_id):
uplink_classifier[PACKET_TAG_TYPE] = SINGLE_TAG
self.add_hsia_flow(intf_id, onu_id, uni_id, port_no, uplink_classifier,
uplink_action, UPSTREAM,
logical_flow, alloc_id, gemport_id)
# Secondary EAP on the subscriber vlan
eap_logical_flow = yield self.is_eap_enabled(intf_id, onu_id, uni_id)
if eap_logical_flow is not None:
self.log.debug('adding-supplimental-eap-flow', flow=eap_logical_flow)
self.add_eapol_flow(intf_id, onu_id, uni_id, port_no, eap_logical_flow, alloc_id,
gemport_id, vlan_id=uplink_classifier[VLAN_VID])
def add_downstream_data_flow(self, intf_id, onu_id, uni_id, port_no, downlink_classifier,
downlink_action, flow, alloc_id, gemport_id):
downlink_classifier[PACKET_TAG_TYPE] = DOUBLE_TAG
# Needed ???? It should be already there
downlink_action[POP_VLAN] = True
downlink_action[VLAN_VID] = downlink_classifier[VLAN_VID]
self.add_hsia_flow(intf_id, onu_id, uni_id, port_no, downlink_classifier,
downlink_action, DOWNSTREAM,
flow, alloc_id, gemport_id)
@inlineCallbacks
def add_hsia_flow(self, intf_id, onu_id, uni_id, port_no, classifier, action,
direction, logical_flow, alloc_id, gemport_id):
self.log.debug('add hisa flow', flow=logical_flow, port_no=port_no,
intf_id=intf_id, onu_id=onu_id, uni_id=uni_id, gemport_id=gemport_id,
alloc_id=alloc_id)
flow_store_cookie = self._get_flow_store_cookie(classifier,
gemport_id)
self.log.debug('flow-store-cookie-classifer-action', flow_store_cookie=flow_store_cookie, classifier=classifier,
action=action)
# One of the OLT platform (Broadcom BAL) requires that symmetric
# flows require the same flow_id to be used across UL and DL.
# Since HSIA flow is the only symmetric flow currently, we need to
# re-use the flow_id across both direction. The 'flow_category'
# takes priority over flow_cookie to find any available HSIA_FLOW
# id for the ONU.
flow_id = self.resource_mgr.get_flow_id(intf_id, onu_id, uni_id,
flow_store_cookie,
HSIA_FLOW)
if flow_id is None:
self.log.error("hsia-flow-unavailable")
return
self.log.debug('flow-id', flow_id=flow_id)
network_intf_id = yield self.get_nni_intf_id()
flow = openolt_pb2.Flow(
access_intf_id=intf_id, onu_id=onu_id, uni_id=uni_id, flow_id=flow_id,
flow_type=direction, alloc_id=alloc_id, network_intf_id=network_intf_id,
gemport_id=gemport_id,
classifier=self.mk_classifier(classifier),
action=self.mk_action(action),
priority=logical_flow.priority,
port_no=port_no,
cookie=logical_flow.cookie)
self.log.debug('openolt-agent-flow', hsia_flow=flow)
if self.add_flow_to_device(flow, logical_flow):
self.log.debug('added-hsia-openolt-agent-flow', hsia_flow=flow, logical_flow=logical_flow)
flow_info = self._get_flow_info_as_json_blob(flow,
flow_store_cookie,
HSIA_FLOW)
self.update_flow_info_to_kv_store(flow.access_intf_id,
flow.onu_id, flow.uni_id,
flow.flow_id, flow_info)
@inlineCallbacks
def add_dhcp_trap(self, intf_id, onu_id, uni_id, port_no, classifier, action, logical_flow,
alloc_id, gemport_id):
self.log.debug('add dhcp upstream trap', classifier=classifier,
intf_id=intf_id, onu_id=onu_id, uni_id=uni_id, action=action)
action.clear()
action[TRAP_TO_HOST] = True
classifier[UDP_SRC] = 68
classifier[UDP_DST] = 67
classifier[PACKET_TAG_TYPE] = SINGLE_TAG
classifier.pop(VLAN_VID, None)
flow_store_cookie = self._get_flow_store_cookie(classifier,
gemport_id)
self.log.debug('flow-store-cookie-classifer-action', flow_store_cookie=flow_store_cookie, classifier=classifier,
action=action)
flow_id = self.resource_mgr.get_flow_id(
intf_id, onu_id, uni_id, flow_store_cookie
)
self.log.debug('flow-id', flow_id=flow_id)
network_intf_id = yield self.get_nni_intf_id()
dhcp_flow = openolt_pb2.Flow(
onu_id=onu_id, uni_id=uni_id, flow_id=flow_id, flow_type=UPSTREAM,
access_intf_id=intf_id, gemport_id=gemport_id,
alloc_id=alloc_id, network_intf_id=network_intf_id,
priority=logical_flow.priority,
classifier=self.mk_classifier(classifier),
action=self.mk_action(action),
port_no=port_no,
cookie=logical_flow.cookie)
self.log.debug('openolt-agent-flow', dhcp_flow=dhcp_flow)
if self.add_flow_to_device(dhcp_flow, logical_flow):
self.log.debug('added-dhcp-openolt-agent-flow', dhcp_flow=dhcp_flow, logical_flow=logical_flow)
flow_info = self._get_flow_info_as_json_blob(dhcp_flow, flow_store_cookie)
self.update_flow_info_to_kv_store(dhcp_flow.access_intf_id,
dhcp_flow.onu_id,
dhcp_flow.uni_id,
dhcp_flow.flow_id,
flow_info)
@inlineCallbacks
def add_eapol_flow(self, intf_id, onu_id, uni_id, port_no, logical_flow, alloc_id,
gemport_id, vlan_id=DEFAULT_MGMT_VLAN):
self.log.debug('add eapol upstream trap', flow=logical_flow, port_no=port_no,
intf_id=intf_id, onu_id=onu_id, uni_id=uni_id, gemport_id=gemport_id,
alloc_id=alloc_id, vlan_id=vlan_id)
uplink_classifier = dict()
uplink_classifier[ETH_TYPE] = EAP_ETH_TYPE
uplink_classifier[PACKET_TAG_TYPE] = SINGLE_TAG
uplink_classifier[VLAN_VID] = vlan_id
uplink_action = dict()
uplink_action[TRAP_TO_HOST] = True
flow_store_cookie = self._get_flow_store_cookie(uplink_classifier,
gemport_id)
self.log.debug('flow-store-cookie-classifier-action', flow_store_cookie=flow_store_cookie, uplink_classifier=uplink_classifier,
uplink_action=uplink_action)
# Add Upstream EAPOL Flow.
uplink_flow_id = self.resource_mgr.get_flow_id(
intf_id, onu_id, uni_id, flow_store_cookie
)
self.log.debug('flow-id', uplink_flow_id=uplink_flow_id)
network_intf_id = yield self.get_nni_intf_id()
upstream_flow = openolt_pb2.Flow(
access_intf_id=intf_id, onu_id=onu_id, uni_id=uni_id, flow_id=uplink_flow_id,
flow_type=UPSTREAM, alloc_id=alloc_id, network_intf_id=network_intf_id,
gemport_id=gemport_id,
classifier=self.mk_classifier(uplink_classifier),
action=self.mk_action(uplink_action),
priority=logical_flow.priority,
port_no=port_no,
cookie=logical_flow.cookie)
self.log.debug('openolt-agent-flow', upstream_flow=upstream_flow)
logical_flow = copy.deepcopy(logical_flow)
logical_flow.match.oxm_fields.extend(fd.mk_oxm_fields([fd.vlan_vid(
vlan_id | 0x1000)]))
logical_flow.match.type = OFPMT_OXM
if self.add_flow_to_device(upstream_flow, logical_flow):
self.log.debug('added-eapol-openolt-agent-flow', upstream_flow=upstream_flow, logical_flow=logical_flow)
flow_info = self._get_flow_info_as_json_blob(upstream_flow,
flow_store_cookie)
self.update_flow_info_to_kv_store(upstream_flow.access_intf_id,
upstream_flow.onu_id,
upstream_flow.uni_id,
upstream_flow.flow_id,
flow_info)
if vlan_id == DEFAULT_MGMT_VLAN:
# Add Downstream EAPOL Flow, Only for first EAP flow (BAL
# requirement)
# On one of the platforms (Broadcom BAL), when same DL classifier
# vlan was used across multiple ONUs, eapol flow re-adds after
# flow delete (cases of onu reboot/disable) fails.
# In order to generate unique vlan, a combination of intf_id
# onu_id and uni_id is used.
# uni_id defaults to 0, so add 1 to it.
special_vlan_downstream_flow = 4090 - intf_id * onu_id * (uni_id+1)
# Assert that we do not generate invalid vlans under no condition
assert (special_vlan_downstream_flow >= 2), 'invalid-vlan-generated'
self.log.warn('generating-special-downstream-vlan-for-bal', special_vlan_downstream_flow=special_vlan_downstream_flow)
downlink_classifier = dict()
downlink_classifier[PACKET_TAG_TYPE] = SINGLE_TAG
downlink_classifier[VLAN_VID] = special_vlan_downstream_flow
downlink_action = dict()
downlink_action[PUSH_VLAN] = True
downlink_action[VLAN_VID] = vlan_id
flow_store_cookie = self._get_flow_store_cookie(downlink_classifier,
gemport_id)
self.log.debug('flow-store-cookie-classifier-action', flow_store_cookie=flow_store_cookie, downlink_classifier=downlink_classifier,
downlink_action=downlink_action)
downlink_flow_id = self.resource_mgr.get_flow_id(
intf_id, onu_id, uni_id, flow_store_cookie
)
self.log.debug('flow-id', downlink_flow_id=downlink_flow_id)
downstream_flow = openolt_pb2.Flow(
access_intf_id=intf_id, onu_id=onu_id, uni_id=uni_id, flow_id=downlink_flow_id,
flow_type=DOWNSTREAM, alloc_id=alloc_id, network_intf_id=network_intf_id,
gemport_id=gemport_id,
classifier=self.mk_classifier(downlink_classifier),
action=self.mk_action(downlink_action),
priority=logical_flow.priority,
port_no=port_no,
cookie=logical_flow.cookie)
self.log.debug('openolt-agent-flow', downstream_flow=downstream_flow)
try:
downstream_logical_flow = ofp_flow_stats(
id=logical_flow.id, cookie=logical_flow.cookie,
table_id=logical_flow.table_id, priority=logical_flow.priority,
flags=logical_flow.flags)
downstream_logical_flow.match.oxm_fields.extend(fd.mk_oxm_fields([
fd.in_port(fd.get_out_port(logical_flow)),
fd.vlan_vid(special_vlan_downstream_flow | 0x1000)]))
downstream_logical_flow.match.type = OFPMT_OXM
downstream_logical_flow.instructions.extend(
fd.mk_instructions_from_actions([fd.output(
self.platform.mk_uni_port_num(intf_id, onu_id, uni_id))]))
except Exception as e:
self.log.exception("unexpected-error-building-downstream-logical-flow", intf_id=intf_id, onu_id=onu_id,
uni_id=uni_id, e=e, downstream_flow=downstream_flow)
if self.add_flow_to_device(downstream_flow, downstream_logical_flow):
self.log.debug('added-eapol-openolt-agent-flow', downstream_flow=downstream_flow,
downstream_logical_flow=downstream_logical_flow)
flow_info = self._get_flow_info_as_json_blob(downstream_flow,
flow_store_cookie)
self.update_flow_info_to_kv_store(downstream_flow.access_intf_id,
downstream_flow.onu_id,
downstream_flow.uni_id,
downstream_flow.flow_id,
flow_info)
def repush_all_different_flows(self):
# Check if the device is supposed to have flows, if so add them
# Recover static flows after a reboot
# TODO NEW CORE: Keep track of device flows locally. need new array
#logical_flows = self.logical_flows_proxy.get('/').items
#devices_flows = self.flows_proxy.get('/').items
logical_flows = []
devices_flows = []
logical_flows_ids_provisioned = [f.cookie for f in devices_flows]
for logical_flow in logical_flows:
try:
if logical_flow.id not in logical_flows_ids_provisioned:
self.add_flow(logical_flow)
except Exception as e:
self.log.exception('Problem reading this flow', e=e)
def reset_flows(self):
# TODO NEW CORE: Keep track of device flows locally. need new array. here clear them out
#self.flows_proxy.update('/', Flows())
pass
""" Add a downstream LLDP trap flow on the NNI interface
"""
def add_lldp_flow(self, logical_flow, port_no, network_intf_id=0):
self.log.debug('add lldp trap flow', flow=logical_flow, port_no=port_no,
network_intf_id=network_intf_id)
classifier = dict()
classifier[ETH_TYPE] = LLDP_ETH_TYPE
classifier[PACKET_TAG_TYPE] = UNTAGGED
action = dict()
action[TRAP_TO_HOST] = True
# LLDP flow is installed to trap LLDP packets on the NNI port.
# We manage flow_id resource pool on per PON port basis.
# Since this situation is tricky, as a hack, we pass the NNI port
# index (network_intf_id) as PON port Index for the flow_id resource
# pool. Also, there is no ONU Id available for trapping LLDP packets
# on NNI port, use onu_id as -1 (invalid)
# ****************** CAVEAT *******************
# This logic works if the NNI Port Id falls within the same valid
# range of PON Port Ids. If this doesn't work for some OLT Vendor
# we need to have a re-look at this.
# *********************************************
onu_id = -1
uni_id = -1
flow_store_cookie = self._get_flow_store_cookie(classifier)
flow_id = self.resource_mgr.get_flow_id(network_intf_id, onu_id, uni_id,
flow_store_cookie)
self.log.debug('flow-store-cookie-classifier-action', flow_store_cookie=flow_store_cookie, classifier=classifier,
action=action)
downstream_flow = openolt_pb2.Flow(
access_intf_id=-1, # access_intf_id not required
onu_id=onu_id, # onu_id not required
uni_id=uni_id, # uni_id not used
flow_id=flow_id,
flow_type=DOWNSTREAM,
network_intf_id=network_intf_id,
gemport_id=-1, # gemport_id not required
classifier=self.mk_classifier(classifier),
action=self.mk_action(action),
priority=logical_flow.priority,
port_no=port_no,
cookie=logical_flow.cookie)
self.log.debug('openolt-agent-flow', downstream_flow=downstream_flow)
if self.add_flow_to_device(downstream_flow, logical_flow):
self.log.debug('added-lldp-openolt-agent-flow', downstream_flow=downstream_flow)
self.update_flow_info_to_kv_store(network_intf_id, onu_id, uni_id,
flow_id, downstream_flow)
def mk_classifier(self, classifier_info):
classifier = openolt_pb2.Classifier()
if ETH_TYPE in classifier_info:
classifier.eth_type = classifier_info[ETH_TYPE]
if IP_PROTO in classifier_info:
classifier.ip_proto = classifier_info[IP_PROTO]
if VLAN_VID in classifier_info:
classifier.o_vid = classifier_info[VLAN_VID]
if METADATA in classifier_info:
classifier.i_vid = classifier_info[METADATA]
if VLAN_PCP in classifier_info:
classifier.o_pbits = classifier_info[VLAN_PCP]
if UDP_SRC in classifier_info:
classifier.src_port = classifier_info[UDP_SRC]
if UDP_DST in classifier_info:
classifier.dst_port = classifier_info[UDP_DST]
if IPV4_DST in classifier_info:
classifier.dst_ip = classifier_info[IPV4_DST]
if IPV4_SRC in classifier_info:
classifier.src_ip = classifier_info[IPV4_SRC]
if PACKET_TAG_TYPE in classifier_info:
if classifier_info[PACKET_TAG_TYPE] == SINGLE_TAG:
classifier.pkt_tag_type = SINGLE_TAG
elif classifier_info[PACKET_TAG_TYPE] == DOUBLE_TAG:
classifier.pkt_tag_type = DOUBLE_TAG
elif classifier_info[PACKET_TAG_TYPE] == UNTAGGED:
classifier.pkt_tag_type = UNTAGGED
else:
classifier.pkt_tag_type = 'none'
return classifier
def mk_action(self, action_info):
action = openolt_pb2.Action()
if POP_VLAN in action_info:
action.o_vid = action_info[VLAN_VID]
action.cmd.remove_outer_tag = True
elif PUSH_VLAN in action_info:
action.o_vid = action_info[VLAN_VID]
action.cmd.add_outer_tag = True
elif TRAP_TO_HOST in action_info:
action.cmd.trap_to_host = True
else:
self.log.info('Invalid-action-field', action_info=action_info)
return
return action
@inlineCallbacks
def is_eap_enabled(self, intf_id, onu_id, uni_id):
self.log.debug('looking from eap enabled for port', intf_id=intf_id, onu_id=onu_id, uni_id=uni_id)
# TODO NEW CORE: This is really spendy, likely perfomance implications
device = yield self.core_proxy.get_device(self.device_id)
flows = device.flows.items
for flow in flows:
eap_flow = False
eap_intf_id = None
eap_onu_id = None
eap_uni_id = None
for field in fd.get_ofb_fields(flow):
if field.type == fd.ETH_TYPE:
if field.eth_type == EAP_ETH_TYPE:
eap_flow = True
if field.type == fd.TUNNEL_ID:
port = fd.get_child_port_from_tunnelid(flow)
eap_intf_id = self.platform.intf_id_from_uni_port_num(port)
eap_onu_id = self.platform.onu_id_from_port_num(port)
eap_uni_id = self.platform.uni_id_from_port_num(port)
if eap_flow:
self.log.debug('eap flow detected', onu_id=onu_id, uni_id=uni_id,
intf_id=intf_id, eap_intf_id=eap_intf_id,
eap_onu_id=eap_onu_id,
eap_uni_id=eap_uni_id)
if eap_flow and intf_id == eap_intf_id and onu_id == eap_onu_id and uni_id == eap_uni_id:
returnValue(flow)
returnValue(None)
@inlineCallbacks
def get_subscriber_vlan(self, port):
self.log.debug('looking from subscriber flow for port', port=port)
# TODO NEW CORE: This is really spendy, likely perfomance implications
device = yield self.core_proxy.get_device(self.device_id)
flows = device.flows.items
for flow in flows:
in_port = fd.get_child_port_from_tunnelid(flow)
out_port = fd.get_out_port(flow)
self.log.debug('looping-flows', in_port=in_port, out_port=out_port)
if self.platform.is_controller(out_port):
self.log.debug('skipping-controller-flow', in_port=in_port, out_port=out_port)
continue
if in_port == port and out_port is not None and \
self.platform.intf_id_to_port_type_name(out_port) \
== Port.ETHERNET_NNI:
fields = fd.get_ofb_fields(flow)
self.log.debug('subscriber flow found', fields=fields)
for field in fields:
if field.type == OFPXMT_OFB_VLAN_VID:
self.log.debug('subscriber vlan found',
vlan_id=field.vlan_vid)
returnValue(field.vlan_vid & 0x0fff)
self.log.debug('No subscriber flow found', port=port)
returnValue(None)
def add_flow_to_device(self, flow, logical_flow):
self.log.debug('pushing flow to device', flow=flow)
try:
self.stub.FlowAdd(flow)
except grpc.RpcError as grpc_e:
if grpc_e.code() == grpc.StatusCode.ALREADY_EXISTS:
self.log.warn('flow already exists', e=grpc_e, flow=flow)
else:
self.log.error('failed to add flow',
logical_flow=logical_flow, flow=flow,
grpc_error=grpc_e)
return False
except Exception as f:
self.log.exception("unexpected-openolt-agent-error", flow=flow, logical_flow=logical_flow, f=f)
else:
self.register_flow(logical_flow, flow)
return True
def update_flow_info_to_kv_store(self, intf_id, onu_id, uni_id, flow_id, flow):
self.log.debug("update-flow-info", intf_id=intf_id, onu_id=onu_id, uni_id=uni_id, flow_id=flow_id, flow=flow)
self.resource_mgr.update_flow_id_info_for_uni(intf_id, onu_id, uni_id,
flow_id, flow)
def register_flow(self, logical_flow, device_flow):
self.log.debug('registering flow in device',
logical_flow=logical_flow, device_flow=device_flow)
stored_flow = copy.deepcopy(logical_flow)
stored_flow.id = self.generate_stored_id(device_flow.flow_id,
device_flow.flow_type)
self.log.debug('generated device flow id', id=stored_flow.id,
flow_id=device_flow.flow_id,
direction=device_flow.flow_type)
stored_flow.cookie = logical_flow.id
# TODO NEW CORE: Keep track of device flows locally. need new array
#flows = self.flows_proxy.get('/')
#flows.items.extend([stored_flow])
#self.flows_proxy.update('/', flows)
def clear_flows_and_scheduler_for_logical_port(self, child_device, logical_port):
ofp_port_name = logical_port.ofp_port.name
port_no = logical_port.ofp_port.port_no
pon_port = child_device.proxy_address.channel_id
onu_id = child_device.proxy_address.onu_id
uni_id = self.platform.uni_id_from_port_num(logical_port)
# TODO: The DEFAULT_TECH_PROFILE_ID is assumed. Right way to do,
# is probably to maintain a list of Tech-profile table IDs associated
# with the UNI logical_port. This way, when the logical port is deleted,
# all the associated tech-profile configuration with the UNI logical_port
# can be cleared.
tech_profile_instance = self.tech_profile[pon_port]. \
get_tech_profile_instance(
DEFAULT_TECH_PROFILE_TABLE_ID,
ofp_port_name)
flow_ids = self.resource_mgr.get_current_flow_ids_for_uni(pon_port, onu_id, uni_id)
self.log.debug("outstanding-flows-to-be-cleared", flow_ids=flow_ids)
for flow_id in flow_ids:
flow_infos = self.resource_mgr.get_flow_id_info(pon_port, onu_id, uni_id, flow_id)
for flow_info in flow_infos:
direction = flow_info['flow_type']
flow_to_remove = openolt_pb2.Flow(flow_id=flow_id,
flow_type=direction)
try:
self.stub.FlowRemove(flow_to_remove)
except grpc.RpcError as grpc_e:
if grpc_e.code() == grpc.StatusCode.NOT_FOUND:
self.log.debug('This flow does not exist on the switch, '
'normal after an OLT reboot',
flow=flow_to_remove)
else:
raise grpc_e
self.resource_mgr.free_flow_id_for_uni(pon_port, onu_id, uni_id, flow_id)
try:
tconts = self.tech_profile[pon_port].get_tconts(tech_profile_instance)
self.stub.RemoveTconts(openolt_pb2.Tconts(intf_id=pon_port,
onu_id=onu_id,
uni_id=uni_id,
port_no=port_no,
tconts=tconts))
except grpc.RpcError as grpc_e:
self.log.error('error-removing-tcont-scheduler-queues',
err=grpc_e)
def generate_stored_id(self, flow_id, direction):
if direction == UPSTREAM:
self.log.debug('upstream flow, shifting id')
return 0x1 << 15 | flow_id
elif direction == DOWNSTREAM:
self.log.debug('downstream flow, not shifting id')
return flow_id
else:
self.log.warn('Unrecognized direction', direction=direction)
return flow_id
def decode_stored_id(self, id):
if id >> 15 == 0x1:
return id & 0x7fff, UPSTREAM
else:
return id, DOWNSTREAM
def _populate_tech_profile_per_pon_port(self):
for arange in self.resource_mgr.device_info.ranges:
for intf_id in arange.intf_ids:
self.tech_profile[intf_id] = \
self.resource_mgr.resource_mgrs[intf_id].tech_profile
# Make sure we have as many tech_profiles as there are pon ports on
# the device
assert len(self.tech_profile) == self.resource_mgr.device_info.pon_ports
def _get_flow_info_as_json_blob(self, flow, flow_store_cookie,
flow_category=None):
json_blob = MessageToDict(message=flow,
preserving_proto_field_name=True)
self.log.debug("flow-info", json_blob=json_blob)
json_blob['flow_store_cookie'] = flow_store_cookie
if flow_category is not None:
json_blob['flow_category'] = flow_category
flow_info = self.resource_mgr.get_flow_id_info(flow.access_intf_id,
flow.onu_id, flow.uni_id, flow.flow_id)
if flow_info is None:
flow_info = list()
flow_info.append(json_blob)
else:
assert (isinstance(flow_info, list))
flow_info.append(json_blob)
return flow_info
@staticmethod
def _get_flow_store_cookie(classifier, gem_port=None):
assert isinstance(classifier, dict)
# We need unique flows per gem_port
if gem_port is not None:
to_hash = dumps(classifier, sort_keys=True) + str(gem_port)
else:
to_hash = dumps(classifier, sort_keys=True)
return hashlib.md5(to_hash).hexdigest()[:12]
@inlineCallbacks
def get_nni_intf_id(self):
if self.nni_intf_id is not None:
returnValue(self.nni_intf_id)
port_list = yield self.core_proxy.get_ports(self.device_id, Port.ETHERNET_NNI)
self.log.debug("nni-ports-list", port_list=port_list)
# TODO: Hardcoded only first NNI
port = port_list.items[0]
self.log.debug("nni-port", port=port)
self.nni_intf_id = self.platform.intf_id_from_nni_port_num(port.port_no)
self.log.debug("nni-intf-d ", port=port.port_no, nni_intf_id=self.nni_intf_id)
returnValue(self.nni_intf_id)