Metadata agent support in networking-ovn

Initial implementation

Changes in the mechanism driver:

    * Automatically create a port with device_owner=network:dhcp to
      serve metadata and eventually DHCP.
    * Auto-allocate an IP for the metadata port on subnet creation.
    * Push static route for 169.254.169.254 in DHCP options.
    * Wait until metadata service is provisioned in the chassis where
      a port resides before sending the event to Nova.

Agent:

    * Implementation of [0].

Devstack plugin:

    * Starts networking-ovn-agent with the proper config files.
    * Disables config drive in nova.conf for tempest against master
      branch.
    * Enables TEMPEST_RUN_VALIDATION for tempest against master branch.

In order to test this patch out we need to make these changes to
nova.conf:

    * [DEFAULT] section:
      force_config_drive = False
    * [neutron] section:
      service_metadata_proxy = True

This patch depends on [1] and [2] (already merged into OVS master).
NOTE: Metadata tests are only enabled for the non-voting tempest job
      which runs against OVS master branch. The release job runs against
      OVS 2.7 which doesn't include [1][2] so those tests are disabled
      until OVS 2.8 is released.

[0]
https://docs.openstack.org/developer/networking-ovn/design/metadata_api.html
[1] https://patchwork.ozlabs.org/patch/767369/
[2] https://patchwork.ozlabs.org/patch/771297/

Change-Id: Ife2fd18f2f88050429e70c7e557fa41d2d54b034
changes/40/471140/36
Daniel Alvarez 6 years ago
parent 9b2d20d7aa
commit ad1fea3e7b

@ -18,19 +18,25 @@
# http://git.openstack.org/cgit/openstack-infra/project-config/tree/jenkins/jobs/networking-ovn.yaml
#
export OVERRIDE_ENABLED_SERVICES=key,n-api,n-cpu,n-cond,n-sch,n-crt,n-cauth,n-obj,placement-api,g-api,g-reg,c-sch,c-api,c-vol,rabbit,mysql,dstat,ovn-northd,ovn-controller,q-svc
OVS_BRANCH=$1
OVERRIDE_ENABLED_SERVICES=key,n-api,n-cpu,n-cond,n-sch,n-crt,n-cauth,n-obj,n-api-meta,placement-api,g-api,g-reg,c-sch,c-api,c-vol,rabbit,mysql,dstat,ovn-northd,ovn-controller,q-svc
# FIXME(dalvarez): Remove this once OVS 2.8 is released. Metadata support depends on it.
if [[ "${OVS_BRANCH}" != "latest-release" ]] ; then
OVERRIDE_ENABLED_SERVICES=${OVERRIDE_ENABLED_SERVICES},networking-ovn-metadata-agent
fi
export OVERRIDE_ENABLED_SERVICES
if [ -z "${RALLY_SCENARIO}" ] ; then
# Only include tempest if this is not a rally job.
export OVERRIDE_ENABLED_SERVICES=${OVERRIDE_ENABLED_SERVICES},tempest
# NOTE(dongj): Set ssh validation to False to skip validation tests until OVN
# metadata agent is supported, because VM obtains password from metadata agent
# for these validation tests.
export DEVSTACK_LOCAL_CONFIG+=$'\n'"TEMPEST_RUN_VALIDATION=False"
# FIXME(dalvarez): Remove this once OVS 2.8 is released. Metadata support depends on it.
if [[ "${OVS_BRANCH}" == "latest-release" ]] ; then
export DEVSTACK_LOCAL_CONFIG+=$'\n'"TEMPEST_RUN_VALIDATION=False"
fi
fi
export DEVSTACK_LOCAL_CONFIG+=$'\n'"Q_USE_PROVIDERNET_FOR_PUBLIC=True"
export DEVSTACK_LOCAL_CONFIG+=$'\n'"PHYSICAL_NETWORK=public"
OVS_BRANCH=$1
if [[ "${OVS_BRANCH}" == "latest-release" ]] ; then
export DEVSTACK_LOCAL_CONFIG+=$'\n'"OVN_BRANCH=branch-2.7"
elif [[ "${OVS_BRANCH}" == "master" ]] ; then

@ -97,11 +97,16 @@ OVN_L3_SCHEDULER=${OVN_L3_SCHEDULER:-leastloaded}
# Neutron directory
NEUTRON_DIR=$DEST/neutron
OVN_META_CONF=$NEUTRON_CONF_DIR/metadata_agent.ini
# Set variables for building OVS from source
OVS_REPO=$OVN_REPO
OVS_REPO_NAME=$OVN_REPO_NAME
OVS_BRANCH=$OVN_BRANCH
NETWORKING_OVN_BIN_DIR=$(get_python_exec_prefix)
NETWORKING_OVN_METADATA_BINARY="networking-ovn-metadata-agent"
# Utility Functions
# -----------------
@ -163,6 +168,20 @@ function configure_ovn {
echo $OVN_UUID > ovn-uuid
fi
fi
# Metadata
if is_service_enabled networking-ovn-metadata-agent; then
sudo install -d -o $STACK_USER $NEUTRON_CONF_DIR
(cd $NETWORKING_OVN_DIR && exec ./tools/generate_config_file_samples.sh)
cp $NETWORKING_OVN_DIR/etc/metadata_agent.ini.sample $OVN_META_CONF
iniset $OVN_META_CONF DEFAULT debug $ENABLE_DEBUG_LOG_LEVEL
iniset $OVN_META_CONF DEFAULT nova_metadata_ip $HOST_IP
iniset $OVN_META_CONF DEFAULT metadata_workers $API_WORKERS
iniset $OVN_META_CONF ovs ovsdb_connection unix:/usr/local/var/run/openvswitch/db.sock
fi
}
function configure_ovn_plugin {
@ -196,7 +215,11 @@ function configure_ovn_plugin {
# NOTE(rtheis): OVN currently lacks support for metadata so enabling
# config drive is required to provide metadata to instances.
if is_service_enabled n-cpu ; then
iniset $NOVA_CONF DEFAULT force_config_drive True
if is_service_enabled networking-ovn-metadata-agent ; then
iniset $NOVA_CONF neutron service_metadata_proxy True
else
iniset $NOVA_CONF DEFAULT force_config_drive True
fi
fi
}
@ -436,6 +459,10 @@ function start_ovn {
echo "done."
sudo ovs-appctl -t ovn-northd vlog/set "PATTERN:CONSOLE:%D{%Y-%m-%dT%H:%M:%S.###Z}|%05N|%c%T|%p|%m"
fi
if is_service_enabled networking-ovn-metadata-agent; then
run_process networking-ovn-metadata-agent "$NETWORKING_OVN_BIN_DIR/$NETWORKING_OVN_METADATA_BINARY --config-file $NEUTRON_CONF --config-file $Q_PLUGIN_CONF_FILE --config-file $OVN_META_CONF"
fi
}
# stop_ovn() - Stop running processes (non-screen)
@ -453,6 +480,10 @@ function stop_ovn {
/usr/local/share/openvswitch/scripts/ovn-ctl stop_northd
fi
sudo killall ovsdb-server
if is_service_enabled networking-ovn-metadata-agent; then
sudo pkill -9 -f haproxy || :
stop_process networking-ovn-metadata-agent
fi
}
# stop_ovs_dp() - Stop OVS datapath

@ -25,6 +25,7 @@ ADMIN_PASSWORD=password
enable_plugin networking-ovn https://git.openstack.org/openstack/networking-ovn
enable_service ovn-northd
enable_service ovn-controller
enable_service networking-ovn-metadata-agent
# Use Neutron instead of nova-network
disable_service n-net

@ -0,0 +1,178 @@
[DEFAULT]
#
# From networking_ovn.metadata.agent
#
# Location for Metadata Proxy UNIX domain socket. (string value)
#metadata_proxy_socket = $state_path/metadata_proxy
# User (uid or name) running metadata proxy after its initialization (if empty:
# agent effective user). (string value)
#metadata_proxy_user =
# Group (gid or name) running metadata proxy after its initialization (if
# empty: agent effective group). (string value)
#metadata_proxy_group =
# Name of Open vSwitch bridge to use (string value)
#ovs_integration_bridge = br-int
# Certificate Authority public key (CA cert) file for ssl (string value)
#auth_ca_cert = <None>
# IP address or DNS name of Nova metadata server. (unknown value)
# Deprecated group/name - [DEFAULT]/nova_metadata_ip
#nova_metadata_host = 127.0.0.1
# TCP Port used by Nova metadata server. (port value)
# Minimum value: 0
# Maximum value: 65535
#nova_metadata_port = 8775
# When proxying metadata requests, Neutron signs the Instance-ID header with a
# shared secret to prevent spoofing. You may select any string for a secret,
# but it must match here and in the configuration used by the Nova Metadata
# Server. NOTE: Nova uses the same config key, but in [neutron] section.
# (string value)
#metadata_proxy_shared_secret =
# Protocol to access nova metadata, http or https (string value)
# Allowed values: http, https
#nova_metadata_protocol = http
# Allow to perform insecure SSL (https) requests to nova metadata (boolean
# value)
#nova_metadata_insecure = false
# Client certificate for nova metadata api server. (string value)
#nova_client_cert =
# Private key of client certificate. (string value)
#nova_client_priv_key =
# Metadata Proxy UNIX domain socket mode, 4 values allowed: 'deduce': deduce
# mode from metadata_proxy_user/group values, 'user': set metadata proxy socket
# mode to 0o644, to use when metadata_proxy_user is agent effective user or
# root, 'group': set metadata proxy socket mode to 0o664, to use when
# metadata_proxy_group is agent effective group or root, 'all': set metadata
# proxy socket mode to 0o666, to use otherwise. (string value)
# Allowed values: deduce, user, group, all
#metadata_proxy_socket_mode = deduce
# Number of separate worker processes for metadata server (defaults to half of
# the number of CPUs) (integer value)
#metadata_workers = 1
# Number of backlog requests to configure the metadata server socket with
# (integer value)
#metadata_backlog = 4096
#
# From oslo.log
#
# If set to true, the logging level will be set to DEBUG instead of the default
# INFO level. (boolean value)
# Note: This option can be changed without restarting.
#debug = false
# The name of a logging configuration file. This file is appended to any
# existing logging configuration files. For details about logging configuration
# files, see the Python logging module documentation. Note that when logging
# configuration files are used then all logging configuration is set in the
# configuration file and other logging configuration options are ignored (for
# example, logging_context_format_string). (string value)
# Note: This option can be changed without restarting.
# Deprecated group/name - [DEFAULT]/log_config
#log_config_append = <None>
# Defines the format string for %%(asctime)s in log records. Default:
# %(default)s . This option is ignored if log_config_append is set. (string
# value)
#log_date_format = %Y-%m-%d %H:%M:%S
# (Optional) Name of log file to send logging output to. If no default is set,
# logging will go to stderr as defined by use_stderr. This option is ignored if
# log_config_append is set. (string value)
# Deprecated group/name - [DEFAULT]/logfile
#log_file = <None>
# (Optional) The base directory used for relative log_file paths. This option
# is ignored if log_config_append is set. (string value)
# Deprecated group/name - [DEFAULT]/logdir
#log_dir = <None>
# Uses logging handler designed to watch file system. When log file is moved or
# removed this handler will open a new log file with specified path
# instantaneously. It makes sense only if log_file option is specified and
# Linux platform is used. This option is ignored if log_config_append is set.
# (boolean value)
#watch_log_file = false
# Use syslog for logging. Existing syslog format is DEPRECATED and will be
# changed later to honor RFC5424. This option is ignored if log_config_append
# is set. (boolean value)
#use_syslog = false
# Enable journald for logging. If running in a systemd environment you may wish
# to enable journal support. Doing so will use the journal native protocol
# which includes structured metadata in addition to log messages.This option is
# ignored if log_config_append is set. (boolean value)
#use_journal = false
# Syslog facility to receive log lines. This option is ignored if
# log_config_append is set. (string value)
#syslog_log_facility = LOG_USER
# Log output to standard error. This option is ignored if log_config_append is
# set. (boolean value)
#use_stderr = false
# Format string to use for log messages with context. (string value)
#logging_context_format_string = %(asctime)s.%(msecs)03d %(process)d %(levelname)s %(name)s [%(request_id)s %(user_identity)s] %(instance)s%(message)s
# Format string to use for log messages when context is undefined. (string
# value)
#logging_default_format_string = %(asctime)s.%(msecs)03d %(process)d %(levelname)s %(name)s [-] %(instance)s%(message)s
# Additional data to append to log message when logging level for the message
# is DEBUG. (string value)
#logging_debug_format_suffix = %(funcName)s %(pathname)s:%(lineno)d
# Prefix each line of exception output with this format. (string value)
#logging_exception_prefix = %(asctime)s.%(msecs)03d %(process)d ERROR %(name)s %(instance)s
# Defines the format string for %(user_identity)s that is used in
# logging_context_format_string. (string value)
#logging_user_identity_format = %(user)s %(tenant)s %(domain)s %(user_domain)s %(project_domain)s
# List of package logging levels in logger=LEVEL pairs. This option is ignored
# if log_config_append is set. (list value)
#default_log_levels = amqp=WARN,amqplib=WARN,boto=WARN,qpid=WARN,sqlalchemy=WARN,suds=INFO,oslo.messaging=INFO,oslo_messaging=INFO,iso8601=WARN,requests.packages.urllib3.connectionpool=WARN,urllib3.connectionpool=WARN,websocket=WARN,requests.packages.urllib3.util.retry=WARN,urllib3.util.retry=WARN,keystonemiddleware=WARN,routes.middleware=WARN,stevedore=WARN,taskflow=WARN,keystoneauth=WARN,oslo.cache=INFO,dogpile.core.dogpile=INFO
# Enables or disables publication of error events. (boolean value)
#publish_errors = false
# The format for an instance that is passed with the log message. (string
# value)
#instance_format = "[instance: %(uuid)s] "
# The format for an instance UUID that is passed with the log message. (string
# value)
#instance_uuid_format = "[instance: %(uuid)s] "
# Interval, number of seconds, of log rate limiting. (integer value)
#rate_limit_interval = 0
# Maximum number of logged messages per rate_limit_interval. (integer value)
#rate_limit_burst = 0
# Log level name used by rate limiting: CRITICAL, ERROR, INFO, WARNING, DEBUG
# or empty string. Logs with level greater or equal to rate_limit_except_level
# are not filtered. An empty string means that all levels are filtered. (string
# value)
#rate_limit_except_level = CRITICAL
# Enables or disables fatal status of deprecations. (boolean value)
#fatal_deprecations = false

@ -0,0 +1,6 @@
[DEFAULT]
output_file = etc/metadata_agent.ini.sample
wrap_width = 79
namespace = networking_ovn.metadata.agent
namespace = oslo.log

@ -0,0 +1,374 @@
# Copyright 2017 Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
import collections
import re
from neutron.agent.linux import external_process
from neutron.agent.linux import ip_lib
from neutron.common import utils
from neutron_lib import constants as n_const
from oslo_concurrency import lockutils
from oslo_log import log
from ovsdbapp.backend.ovs_idl import vlog
import six
from networking_ovn.agent.metadata import driver as metadata_driver
from networking_ovn.agent.metadata import ovsdb
from networking_ovn.agent.metadata import server as metadata_server
from networking_ovn.common import config
from networking_ovn.common import constants as ovn_const
from networking_ovn.ovsdb import row_event
LOG = log.getLogger(__name__)
_SYNC_STATE_LOCK = lockutils.ReaderWriterLock()
NS_PREFIX = 'ovnmeta-'
METADATA_DEFAULT_PREFIX = 16
METADATA_DEFAULT_IP = '169.254.169.254'
METADATA_DEFAULT_CIDR = '%s/%d' % (METADATA_DEFAULT_IP,
METADATA_DEFAULT_PREFIX)
METADATA_PORT = 80
MAC_PATTERN = re.compile(r'([0-9A-F]{2}[:-]){5}([0-9A-F]{2})', re.I)
MetadataPortInfo = collections.namedtuple('MetadataPortInfo', ['mac',
'ip_addresses'])
def _sync_lock(f):
"""Decorator to block all operations for a global sync call."""
@six.wraps(f)
def wrapped(*args, **kwargs):
with _SYNC_STATE_LOCK.write_lock():
return f(*args, **kwargs)
return wrapped
def _wait_if_syncing(f):
"""Decorator to wait if any sync operations are in progress."""
@six.wraps(f)
def wrapped(*args, **kwargs):
with _SYNC_STATE_LOCK.read_lock():
return f(*args, **kwargs)
return wrapped
class PortBindingChassisEvent(row_event.RowEvent):
def __init__(self, metadata_agent):
self.agent = metadata_agent
table = 'Port_Binding'
events = (self.ROW_UPDATE)
super(PortBindingChassisEvent, self).__init__(
events, table, None)
self.event_name = 'PortBindingChassisEvent'
@_wait_if_syncing
def run(self, event, row, old):
# Check if the port has been bound/unbound to our chassis and update
# the metadata namespace accordingly.
# Type must be empty to make sure it's a VIF.
if row.type != "":
return
new_chassis = getattr(row, 'chassis', [])
old_chassis = getattr(old, 'chassis', [])
if new_chassis and new_chassis[0].name == self.agent.chassis:
LOG.info("Port %s in datapath %s bound to our chassis",
row.logical_port, str(row.datapath.uuid))
self.agent.update_datapath(str(row.datapath.uuid))
elif old_chassis and old_chassis[0].name == self.agent.chassis:
LOG.info("Port %s in datapath %s unbound from our chassis",
row.logical_port, str(row.datapath.uuid))
self.agent.update_datapath(str(row.datapath.uuid))
class ChassisCreateEvent(row_event.RowEvent):
"""Row create event - Chassis name == our_chassis.
On connection, we get a dump of all chassis so if we catch a creation
of our own chassis it has to be a reconnection. In this case, we need
to do a full sync to make sure that we capture all changes while the
connection to OVSDB was down.
"""
def __init__(self, metadata_agent):
self.agent = metadata_agent
self.first_time = True
table = 'Chassis'
events = (self.ROW_CREATE)
super(ChassisCreateEvent, self).__init__(
events, table, (('name', '=', self.agent.chassis),))
self.event_name = 'ChassisCreateEvent'
def run(self, event, row, old):
if self.first_time:
self.first_time = False
else:
LOG.info("Connection to OVSDB established, doing a full sync")
self.agent.sync()
class MetadataAgent(object):
def __init__(self, conf):
self.conf = conf
vlog.use_python_logger(max_level=config.get_ovn_ovsdb_log_level())
self._process_monitor = external_process.ProcessMonitor(
config=self.conf,
resource_type='metadata')
def start(self):
# Launch the server that will act as a proxy between the VM's and Nova.
proxy = metadata_server.UnixDomainMetadataProxy(self.conf)
proxy.run()
# Open the connection to OVS database
self.ovs_idl = ovsdb.MetadataAgentOvsIdl().start()
self.chassis = self._get_own_chassis_name()
# Open the connection to OVN SB database.
self.sb_idl = ovsdb.MetadataAgentOvnSbIdl(
[PortBindingChassisEvent(self), ChassisCreateEvent(self)]).start()
# Do the initial sync.
self.sync()
proxy.wait()
def _get_own_chassis_name(self):
"""Return the external_ids:system-id value of the Open_vSwitch table.
As long as ovn-controller is running on this node, the key is
guaranteed to exist and will include the chassis name.
"""
ext_ids = self.ovs_idl.db_get(
'Open_vSwitch', '.', 'external_ids').execute()
return ext_ids['system-id']
@_sync_lock
def sync(self):
"""Agent sync.
This function will make sure that all networks with ports in our
chassis are serving metadata. Also, it will tear down those namespaces
which were serving metadata but are no longer needed.
"""
metadata_namespaces = self.ensure_all_networks_provisioned()
system_namespaces = ip_lib.IPWrapper().get_namespaces()
unused_namespaces = [ns for ns in system_namespaces if
ns.startswith(NS_PREFIX) and
ns not in metadata_namespaces]
for ns in unused_namespaces:
self.teardown_datapath(self._get_datapath_name(ns))
@staticmethod
def _get_veth_name(datapath):
return ['{}{}{}'.format(n_const.TAP_DEVICE_PREFIX,
datapath[:10], i) for i in [0, 1]]
@staticmethod
def _get_datapath_name(namespace):
return namespace[len(NS_PREFIX):]
@staticmethod
def _get_namespace_name(datapath):
return NS_PREFIX + datapath
def teardown_datapath(self, datapath):
"""Unprovision this datapath to stop serving metadata.
This function will shutdown metadata proxy if it's running and delete
the VETH pair, the OVS port and the namespace.
"""
self.update_chassis_metadata_networks(datapath, remove=True)
namespace = self._get_namespace_name(datapath)
ip = ip_lib.IPWrapper(namespace)
# If the namespace doesn't exist, return
if not ip.netns.exists(namespace):
return
LOG.info("Cleaning up %s namespace which is not needed anymore",
namespace)
metadata_driver.MetadataDriver.destroy_monitored_metadata_proxy(
self._process_monitor, datapath, self.conf, namespace)
veth_name = self._get_veth_name(datapath)
if ip_lib.device_exists(veth_name[0]):
ip_lib.IPWrapper().del_veth(veth_name[0])
self.ovs_idl.del_port('br-int', veth_name[0]).execute()
ip.garbage_collect_namespace()
def update_datapath(self, datapath):
"""Update the metadata service for this datapath.
This function will:
* Provision the namespace if it wasn't already in place.
* Update the namespace if it was already serving metadata (for example,
after binding/unbinding the first/last port of a subnet in our
chassis).
* Tear down the namespace if there are no more ports in our chassis
for this datapath.
"""
ports = self.sb_idl.get_ports_on_chassis(self.chassis)
datapath_ports = [p for p in ports if p.type == '' and
str(p.datapath.uuid) == datapath]
if datapath_ports:
self.provision_datapath(datapath)
else:
self.teardown_datapath(datapath)
def provision_datapath(self, datapath):
"""Provision the datapath so that it can serve metadata.
This function will create the namespace and VETH pair if needed
and assign the IP addresses to the interface corresponding to the
metadata port of the network. It will also remove existing IP
addresses that are no longer needed.
:return: The metadata namespace name of this datapath
"""
LOG.debug("Provisioning datapath %s", datapath)
port = self.sb_idl.get_metadata_port_network(datapath)
# If there's no metadata port or it doesn't have a MAC or IP
# addresses, then tear the namespace down if needed. This might happen
# when there are no subnets yet created so metadata port doesn't have
# an IP address.
if not (port and port.mac and
port.external_ids.get(ovn_const.OVN_CIDRS_EXT_ID_KEY, None)):
LOG.debug("There is no metadata port for datapath %s or it has no "
"MAC or IP addresses configured, tearing the namespace "
"down if needed", datapath)
self.teardown_datapath(datapath)
return
# First entry of the mac field must be the MAC address.
match = MAC_PATTERN.match(port.mac[0].split(' ')[0])
# If it is not, we can't provision the namespace. Tear it down if
# needed and log the error.
if not match:
LOG.error("Metadata port for datapath %s doesn't have a MAC "
"address, tearing the namespace down if needed",
datapath)
self.teardown_datapath(datapath)
return
mac = match.group()
ip_addresses = set(
port.external_ids[ovn_const.OVN_CIDRS_EXT_ID_KEY].split(' '))
ip_addresses.add(METADATA_DEFAULT_CIDR)
metadata_port = MetadataPortInfo(mac, ip_addresses)
# Create the VETH pair if it's not created. Also the add_veth function
# will create the namespace for us.
namespace = self._get_namespace_name(datapath)
veth_name = self._get_veth_name(datapath)
ip1 = ip_lib.IPDevice(veth_name[0])
if ip_lib.device_exists(veth_name[1], namespace):
ip2 = ip_lib.IPDevice(veth_name[1], namespace)
else:
LOG.debug("Creating VETH %s in %s namespace", veth_name[1],
namespace)
# Might happen that the end in the root namespace exists even
# though the other end doesn't. Make sure we delete it first if
# that's the case.
if ip1.exists():
ip1.link.delete()
ip1, ip2 = ip_lib.IPWrapper().add_veth(
veth_name[0], veth_name[1], namespace)
# Make sure both ends of the VETH are up
ip1.link.set_up()
ip2.link.set_up()
# Configure the MAC address.
ip2.link.set_address(metadata_port.mac)
dev_info = ip2.addr.list()
# Configure the IP addresses on the VETH pair and remove those
# that we no longer need.
current_cidrs = {dev['cidr'] for dev in dev_info}
for ipaddr in current_cidrs - metadata_port.ip_addresses:
ip2.addr.delete(ipaddr)
for ipaddr in metadata_port.ip_addresses - current_cidrs:
# NOTE(dalvarez): metadata only works on IPv4. We're doing this
# extra check here because it could be that the metadata port has
# an IPv6 address if there's an IPv6 subnet with SLAAC in its
# network. Neutron IPAM will autoallocate an IPv6 address for every
# port in the network.
if utils.get_ip_version(ipaddr) == 4:
ip2.addr.add(ipaddr)
# Configure the OVS port and add external_ids:iface-id so that it
# can be tracked by OVN.
self.ovs_idl.add_port('br-int', veth_name[0]).execute()
self.ovs_idl.db_set(
'Interface', veth_name[0],
('external_ids', {'iface-id': port.logical_port})).execute()
# Spawn metadata proxy if it's not already running.
metadata_driver.MetadataDriver.spawn_monitored_metadata_proxy(
self._process_monitor, namespace, METADATA_PORT,
self.conf, network_id=datapath)
self.update_chassis_metadata_networks(datapath)
return namespace
def ensure_all_networks_provisioned(self):
"""Ensure that all datapaths are provisioned.
This function will make sure that all datapaths with ports bound to
our chassis have its namespace, VETH pair and OVS port created and
metadata proxy is up and running.
:return: A list with the namespaces that are currently serving
metadata
"""
# Retrieve all ports in our Chassis with type == ''
ports = self.sb_idl.get_ports_on_chassis(self.chassis)
datapaths = {str(p.datapath.uuid) for p in ports if p.type == ''}
namespaces = []
# Make sure that all those datapaths are serving metadata
for datapath in datapaths:
netns = self.provision_datapath(datapath)
if netns:
namespaces.append(netns)
return namespaces
def update_chassis_metadata_networks(self, datapath, remove=False):
"""Update metadata networks hosted in this chassis.
Add or remove a datapath from the list of current datapaths that
we're currently serving metadata.
"""
current_dps = self.sb_idl.get_chassis_metadata_networks(self.chassis)
updated = False
if remove:
if datapath in current_dps:
current_dps.remove(datapath)
updated = True
else:
if datapath not in current_dps:
current_dps.append(datapath)
updated = True
if updated:
with self.sb_idl.create_transaction(check_error=True) as txn:
txn.add(self.sb_idl.set_chassis_metadata_networks(
self.chassis, current_dps))

@ -0,0 +1,217 @@
# Copyright 2017 OpenStack Foundation.
# All Rights Reserved.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
import errno
import grp
import os
import pwd
from oslo_config import cfg
from oslo_log import log as logging
from neutron._i18n import _
from neutron.agent.linux import external_process
from neutron.common import exceptions
LOG = logging.getLogger(__name__)
METADATA_SERVICE_NAME = 'metadata-proxy'
PROXY_CONFIG_DIR = "ns-metadata-proxy"
_HAPROXY_CONFIG_TEMPLATE = """
global
log /dev/log local0 %(log_level)s
user %(user)s
group %(group)s
maxconn 1024
pidfile %(pidfile)s
daemon
defaults
log global
mode http
option httplog
option dontlognull
option http-server-close
option forwardfor
retries 3
timeout http-request 30s
timeout connect 30s
timeout client 32s
timeout server 32s
timeout http-keep-alive 30s
listen listener
bind 0.0.0.0:%(port)s
server metadata %(unix_socket_path)s
http-request add-header X-OVN-%(res_type)s-ID %(res_id)s
"""
class InvalidUserOrGroupException(Exception):
pass
class HaproxyConfigurator(object):
def __init__(self, network_id, router_id, unix_socket_path, port, user,
group, state_path, pid_file):
self.network_id = network_id
self.router_id = router_id
if network_id is None and router_id is None:
raise exceptions.NetworkIdOrRouterIdRequiredError()
self.port = port
self.user = user
self.group = group
self.state_path = state_path
self.unix_socket_path = unix_socket_path
self.pidfile = pid_file
self.log_level = (
'debug' if logging.is_debug_enabled(cfg.CONF) else 'info')
def create_config_file(self):
"""Create the config file for haproxy."""
# Need to convert uid/gid into username/group
try:
username = pwd.getpwuid(int(self.user)).pw_name
except (ValueError, KeyError):
try:
username = pwd.getpwnam(self.user).pw_name
except KeyError:
raise InvalidUserOrGroupException(
_("Invalid user/uid: '%s'") % self.user)
try:
groupname = grp.getgrgid(int(self.group)).gr_name
except (ValueError, KeyError):
try:
groupname = grp.getgrnam(self.group).gr_name
except KeyError:
raise InvalidUserOrGroupException(
_("Invalid group/gid: '%s'") % self.group)
cfg_info = {
'port': self.port,
'unix_socket_path': self.unix_socket_path,
'user': username,
'group': groupname,
'pidfile': self.pidfile,
'log_level': self.log_level
}
if self.network_id:
cfg_info['res_type'] = 'Network'
cfg_info['res_id'] = self.network_id
else:
cfg_info['res_type'] = 'Router'
cfg_info['res_id'] = self.router_id
haproxy_cfg = _HAPROXY_CONFIG_TEMPLATE % cfg_info
LOG.debug("haproxy_cfg = %s", haproxy_cfg)
cfg_dir = self.get_config_path(self.state_path)
# uuid has to be included somewhere in the command line so that it can
# be tracked by process_monitor.
self.cfg_path = os.path.join(cfg_dir, "%s.conf" % cfg_info['res_id'])
if not os.path.exists(cfg_dir):
os.makedirs(cfg_dir)
with open(self.cfg_path, "w") as cfg_file:
cfg_file.write(haproxy_cfg)
@staticmethod
def get_config_path(state_path):
return os.path.join(state_path or cfg.CONF.state_path,
PROXY_CONFIG_DIR)
@staticmethod
def cleanup_config_file(uuid, state_path):
"""Delete config file created when metadata proxy was spawned."""
# Delete config file if it exists
cfg_path = os.path.join(
HaproxyConfigurator.get_config_path(state_path),
"%s.conf" % uuid)
try:
os.unlink(cfg_path)
except OSError as ex:
# It can happen that this function is called but metadata proxy
# was never spawned so its config file won't exist
if ex.errno != errno.ENOENT:
raise
class MetadataDriver(object):
monitors = {}
@classmethod
def _get_metadata_proxy_user_group(cls, conf):
user = conf.metadata_proxy_user or str(os.geteuid())
group = conf.metadata_proxy_group or str(os.getegid())
return user, group
@classmethod
def _get_metadata_proxy_callback(cls, port, conf, network_id=None,
router_id=None):
def callback(pid_file):
metadata_proxy_socket = conf.metadata_proxy_socket
user, group = (
cls._get_metadata_proxy_user_group(conf))
haproxy = HaproxyConfigurator(network_id,
router_id,
metadata_proxy_socket,
port,
user,
group,
conf.state_path,
pid_file)
haproxy.create_config_file()
proxy_cmd = ['haproxy',
'-f', haproxy.cfg_path]
return proxy_cmd
return callback
@classmethod
def spawn_monitored_metadata_proxy(cls, monitor, ns_name, port, conf,
network_id=None, router_id=None):
uuid = network_id or router_id
callback = cls._get_metadata_proxy_callback(
port, conf, network_id=network_id, router_id=router_id)
pm = cls._get_metadata_proxy_process_manager(uuid, conf,
ns_name=ns_name,
callback=callback)
pm.enable()
monitor.register(uuid, METADATA_SERVICE_NAME, pm)
cls.monitors[router_id] = pm
@classmethod
def destroy_monitored_metadata_proxy(cls, monitor, uuid, conf, ns_name):
monitor.unregister(uuid, METADATA_SERVICE_NAME)
pm = cls._get_metadata_proxy_process_manager(uuid, conf,
ns_name=ns_name)
pm.disable()
# Delete metadata proxy config file
HaproxyConfigurator.cleanup_config_file(uuid, cfg.CONF.state_path)
cls.monitors.pop(uuid, None)
@classmethod
def _get_metadata_proxy_process_manager(cls, router_id, conf, ns_name=None,
callback=None):
return external_process.ProcessManager(
conf=conf,
uuid=router_id,
namespace=ns_name,
default_cmd_callback=callback)

@ -0,0 +1,59 @@
# Copyright 2017 Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
from ovs.db import idl
from ovsdbapp.backend.ovs_idl import connection
from ovsdbapp.backend.ovs_idl import idlutils
from ovsdbapp.schema.open_vswitch import impl_idl as idl_ovs
from networking_ovn.common import config
from networking_ovn.ovsdb import impl_idl_ovn as idl_ovn
from networking_ovn.ovsdb import ovsdb_monitor
class MetadataAgentOvnSbIdl(ovsdb_monitor.OvnIdl):
SCHEMA = 'OVN_Southbound'
def __init__(self, events=None):
connection_string = config.get_ovn_sb_connection()
helper = idlutils.get_schema_helper(connection_string, self.SCHEMA)
tables = ('Chassis', 'Port_Binding', 'Datapath_Binding')
for table in tables:
helper.register_table(table)
super(MetadataAgentOvnSbIdl, self).__init__(
None, connection_string, helper)
if events:
self.notify_handler.watch_events(events)
def start(self):
ovsdb_monitor._check_and_set_ssl_files(self.SCHEMA)
conn = connection.Connection(
self, timeout=config.get_ovn_ovsdb_timeout())
return idl_ovn.OvsdbSbOvnIdl(conn)
class MetadataAgentOvsIdl(object):
def start(self):
connection_string = config.cfg.CONF.ovs.ovsdb_connection
helper = idlutils.get_schema_helper(connection_string,
'Open_vSwitch')
tables = ('Open_vSwitch', 'Bridge', 'Port', 'Interface')
for table in tables:
helper.register_table(table)
ovs_idl = idl.Idl(connection_string, helper)
conn = connection.Connection(
ovs_idl, timeout=config.cfg.CONF.ovs.ovsdb_connection_timeout)
return idl_ovs.OvsdbIdl(conn)

@ -0,0 +1,190 @@
# Copyright 2017 Red Hat, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
import hashlib
import hmac
import httplib2
from neutron._i18n import _
from neutron.agent.linux import utils as agent_utils
from neutron.conf.agent.metadata import config
from neutron_lib.callbacks import events
from neutron_lib.callbacks import registry
from neutron_lib.callbacks import resources
from oslo_config import cfg
from oslo_log import log as logging
from oslo_utils import encodeutils
import six
import six.moves.urllib.parse as urlparse
import webob
from networking_ovn.agent.metadata import ovsdb
from networking_ovn.common import constants as ovn_const
LOG = logging.getLogger(__name__)
MODE_MAP = {
config.USER_MODE: 0o644,
config.GROUP_MODE: 0o664,
config.ALL_MODE: 0o666,
}
class MetadataProxyHandler(object):
def __init__(self, conf):
self.conf = conf
self.subscribe()
def subscribe(self):
registry.subscribe(self.post_fork_initialize,
resources.PROCESS,
events.AFTER_INIT)
def post_fork_initialize(self, resource, event, trigger, **kwargs):
# We need to open a connection to OVN SouthBound database for
# each worker so that we can process the metadata requests.
self.sb_idl = ovsdb.MetadataAgentOvnSbIdl().start()
@webob.dec.wsgify(RequestClass=webob.Request)
def __call__(self, req):
try:
LOG.debug("Request: %s", req)
instance_id, project_id = self._get_instance_and_project_id(req)
if instance_id:
return self._proxy_request(instance_id, project_id, req)
else:
return webob.exc.HTTPNotFound()
except Exception:
LOG.exception("Unexpected error.")
msg = _('An unknown error has occurred. '
'Please try your request again.')
explanation = six.text_type(msg)
return webob.exc.HTTPInternalServerError(explanation=explanation)
def _get_instance_and_project_id(self, req):
remote_address = req.headers.get('X-Forwarded-For')
network_id = req.headers.get('X-OVN-Network-ID')
ports = self.sb_idl.get_network_port_bindings_by_ip(network_id,
remote_address)
if len(ports) == 1:
external_ids = ports[0].external_ids
return (external_ids[ovn_const.OVN_DEVID_EXT_ID_KEY],
external_ids[ovn_const.OVN_PROJID_EXT_ID_KEY])
return None, None
def _proxy_request(self, instance_id, tenant_id, req):
headers = {
'X-Forwarded-For': req.headers.get('X-Forwarded-For'),
'X-Instance-ID': str(instance_id),
'X-Tenant-ID': str(tenant_id),
'X-Instance-ID-Signature': self._sign_instance_id(instance_id)
}
nova_host_port = '%s:%s' % (self.conf.nova_metadata_host,
self.conf.nova_metadata_port)
LOG.debug('Request to Nova at %s', nova_host_port)
LOG.debug(headers)
url = urlparse.urlunsplit((
self.conf.nova_metadata_protocol,
nova_host_port,
req.path_info,
req.query_string,
''))
h = httplib2.Http(
ca_certs=self.conf.auth_ca_cert,
disable_ssl_certificate_validation=self.conf.nova_metadata_insecure
)
if self.conf.nova_client_cert and self.conf.nova_client_priv_key:
h.add_certificate(self.conf.nova_client_priv_key,
self.conf.nova_client_cert,
nova_host_port)
resp, content = h.request(url, method=req.method, headers=headers,
body=req.body)
if resp.status == 200:
req.response.content_type = resp['content-type']
req.response.body = content
LOG.debug(str(resp))
return req.response
elif resp.status == 403:
LOG.warning(
'The remote metadata server responded with Forbidden. This '
'response usually occurs when shared secrets do not match.'
)
return webob.exc.HTTPForbidden()
elif resp.status == 400:
return webob.exc.HTTPBadRequest()
elif resp.status == 404:
return webob.exc.HTTPNotFound()
elif resp.status == 409:
return webob.exc.HTTPConflict()
elif resp.status == 500:
msg = _(
'Remote metadata server experienced an internal server error.'
)
LOG.debug(msg)
explanation = six.text_type(msg)
return webob.exc.HTTPInternalServerError(explanation=explanation)
else:
raise Exception(_('Unexpected response code: %s') % resp.status)
def _sign_instance_id(self, instance_id):
secret = self.conf.metadata_proxy_shared_secret
secret = encodeutils.to_utf8(secret)
instance_id = encodeutils.to_utf8(instance_id)
return hmac.new(secret, instance_id, hashlib.sha256).hexdigest()
class UnixDomainMetadataProxy(object):
def __init__(self, conf):
self.conf = conf
agent_utils.ensure_directory_exists_without_file(
cfg.CONF.metadata_proxy_socket)
def _get_socket_mode(self):
mode = self.conf.metadata_proxy_socket_mode
if mode == config.DEDUCE_MODE:
user = self.conf.metadata_proxy_user
if (not user or user == '0' or user == 'root'
or agent_utils.is_effective_user(user)):
# user is agent effective user or root => USER_MODE
mode = config.USER_MODE
else:
group = self.conf.metadata_proxy_group
if not group or agent_utils.is_effective_group(group):
# group is agent effective group => GROUP_MODE
mode = config.GROUP_MODE
else:
# otherwise => ALL_MODE
mode = config.ALL_MODE
return MODE_MAP[mode]
def run(self):
self.server = agent_utils.UnixDomainWSGIServer(
'networking-ovn-metadata-agent')
self.server.start(MetadataProxyHandler(self.conf),
self.conf.metadata_proxy_socket,
workers=self.conf.metadata_workers,
backlog=self.conf.metadata_backlog,
mode=self._get_socket_mode())
def wait(self):
self.server.wait()

@ -0,0 +1,37 @@
# Copyright 2017 OpenStack 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.
import sys
from neutron.common import config
from neutron.common import utils
from oslo_config import cfg
from oslo_log import log as logging
from networking_ovn.agent.metadata import agent
from networking_ovn.conf.agent.metadata import config as meta
LOG = logging.getLogger(__name__)
def main():
meta.register_meta_conf_opts(meta.SHARED_OPTS)
meta.register_meta_conf_opts(meta.UNIX_DOMAIN_METADATA_PROXY_OPTS)
meta.register_meta_conf_opts(meta.METADATA_PROXY_HANDLER_OPTS)
meta.register_meta_conf_opts(meta.OVS_OPTS, group='ovs')
config.init(sys.argv[1:])
config.setup_logging()
utils.log_opt_values(LOG)
agt = agent.MetadataAgent(cfg.CONF)
agt.start()

@ -0,0 +1,15 @@
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
from neutron.common import eventlet_utils
eventlet_utils.monkey_patch()

@ -0,0 +1,17 @@
# Licensed under the Apache License, Version 2.0 (the "License"); you may
# not use this file except in compliance with the License. You may obtain
# a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
# WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
# License for the specific language governing permissions and limitations
# under the License.
from networking_ovn.agent import metadata_agent
def main():
metadata_agent.main()

@ -127,7 +127,10 @@ ovn_opts = [
cfg.StrOpt("ovsdb_log_level",
default="INFO",
choices=list(VLOG_LEVELS.keys()),
help=_("The log level used for OVSDB"))
help=_("The log level used for OVSDB")),
cfg.BoolOpt('ovn_metadata_enabled',
default=True,
help=_('Whether to use metadata service.'))
]
cfg.CONF.register_opts(ovn_opts, group='ovn')
@ -201,3 +204,7 @@ def get_ovn_dhcp_default_lease_time():
def get_ovn_ovsdb_log_level():
return VLOG_LEVELS[cfg.CONF.ovn.ovsdb_log_level]
def is_ovn_metadata_enabled():
return cfg.CONF.ovn.ovn_metadata_enabled

@ -11,6 +11,7 @@
# under the License.
from neutron_lib.api.definitions import portbindings
from neutron_lib import constants as const
import six
OVN_ML2_MECH_DRIVER_NAME = 'ovn'
@ -21,6 +22,9 @@ OVN_SG_NAME_EXT_ID_KEY = 'neutron:security_group_name'
OVN_PHYSNET_EXT_ID_KEY = 'neutron:provnet-physical-network'
OVN_NETTYPE_EXT_ID_KEY = 'neutron:provnet-network-type'
OVN_SEGID_EXT_ID_KEY = 'neutron:provnet-segmentation-id'
OVN_PROJID_EXT_ID_KEY = 'neutron:project_id'
OVN_DEVID_EXT_ID_KEY = 'neutron:device_id'
OVN_CIDRS_EXT_ID_KEY = 'neutron:cidrs'
OVN_PORT_BINDING_PROFILE = portbindings.PROFILE
OVN_PORT_BINDING_PROFILE_PARAMS = [{'parent_name': six.string_types,
'tag': six.integer_types},
@ -31,6 +35,8 @@ OVN_GATEWAY_CHASSIS_KEY = 'redirect-chassis'
OVN_PROVNET_PORT_NAME_PREFIX = 'provnet-'
OVN_NEUTRON_OWNER_TO_PORT_TYPE = {const.DEVICE_OWNER_DHCP: 'localport'}
# OVN ACLs have priorities. The highest priority ACL that matches is the one
# that takes effect. Our choice of priority numbers is arbitrary, but it
# leaves room above and below the ACLs we create. We only need two priorities.

@ -17,6 +17,7 @@ import collections
import copy
import netaddr
from neutron.plugins.common import utils as p_utils
from neutron_lib.api.definitions import l3
from neutron_lib.api.definitions import port_security as psec
from neutron_lib import constants as const
@ -28,6 +29,7 @@ from oslo_config import cfg
from oslo_log import log
from oslo_utils import excutils
from networking_ovn.agent.metadata import agent as metadata_agent
from networking_ovn.common import acl as ovn_acl
from networking_ovn.common import config
from networking_ovn.common import constants as ovn_const
@ -43,7 +45,8 @@ OvnPortInfo = collections.namedtuple('OvnPortInfo', ['type', 'options',
'port_security',
'parent_name', 'tag',
'dhcpv4_options',
'dhcpv6_options'])
'dhcpv6_options',
'cidrs'])
class OVNClient(object):
@ -167,6 +170,7 @@ class OVNClient(object):
qos_options = self._qos_driver.get_qos_options(port)
vtep_physical_switch = binding_prof.get('vtep-physical-switch')
cidrs = ''
if vtep_physical_switch:
vtep_logical_switch = binding_prof.get('vtep-logical-switch')
port_type = 'vtep'
@ -183,18 +187,27 @@ class OVNClient(object):
addresses = port['mac_address']
for ip in port.get('fixed_ips', []):
addresses += ' ' + ip['ip_address']
subnet = self._plugin.get_subnet(n_context.get_admin_context(),
ip['subnet_id'])
cidrs += ' {}/{}'.format(ip['ip_address'],
subnet['cidr'].split('/')[1])
port_security = self._get_allowed_addresses_from_port(port)
port_type = ''
port_type = ovn_const.OVN_NEUTRON_OWNER_TO_PORT_TYPE.get(
port['device_owner'], '')
dhcpv4_options = self._get_port_dhcp_options(port, const.IP_VERSION_4)
dhcpv6_options = self._get_port_dhcp_options(port, const.IP_VERSION_6)
return OvnPortInfo(port_type, options, [addresses], port_security,
parent_name, tag, dhcpv4_options, dhcpv6_options)
parent_name, tag, dhcpv4_options, dhcpv6_options,
cidrs.strip())
def create_port(self, port):
port_info = self._get_port_options(port)
external_ids = {ovn_const.OVN_PORT_NAME_EXT_ID_KEY: port['name']}
external_ids = {ovn_const.OVN_PORT_NAME_EXT_ID_KEY: port['name'],
ovn_const.OVN_DEVID_EXT_ID_KEY: port['device_id'],
ovn_const.OVN_PROJID_EXT_ID_KEY: port['project_id'],
ovn_const.OVN_CIDRS_EXT_ID_KEY: port_info.cidrs}
lswitch_name = utils.ovn_name(port['network_id'])
admin_context = n_context.get_admin_context()
sg_cache = {}
@ -262,8 +275,10 @@ class OVNClient(object):
def update_port(self, port, original_port, qos_options=None):
port_info = self._get_port_options(port, qos_options)
external_ids = {
ovn_const.OVN_PORT_NAME_EXT_ID_KEY: port['name']}
external_ids = {ovn_const.OVN_PORT_NAME_EXT_ID_KEY: port['name'],
ovn_const.OVN_DEVID_EXT_ID_KEY: port['device_id'],
ovn_const.OVN_PROJID_EXT_ID_KEY: port['project_id'],
ovn_const.OVN_CIDRS_EXT_ID_KEY: port_info.cidrs}
admin_context = n_context.get_admin_context()
sg_cache = {}
subnet_cache = {}
@ -765,6 +780,15 @@ class OVNClient(object):
if physnet is not None:
tag = int(segid) if segid else None
self._create_provnet_port(txn, network, physnet, tag)
if config.is_ovn_metadata_enabled():
# Create a neutron port for DHCP/metadata services
port = {'port':
{'network_id': network['id'],
'tenant_id': '',
'device_owner': const.DEVICE_OWNER_DHCP}}
p_utils.create_port(self._plugin, n_context.get_admin_context(),
port)
return network
def delete_network(self, network_id):
@ -780,18 +804,21 @@ class OVNClient(object):
check_error=True)
self._qos_driver.update_network(network, original_network)
def _add_subnet_dhcp_options(self, subnet, network, ovn_dhcp_options=None):
def _add_subnet_dhcp_options(self, subnet, network, ovn_dhcp_options=None,
metadata_port_ip=None):
if utils.is_dhcp_options_ignored(subnet):
return
if not ovn_dhcp_options:
ovn_dhcp_options = self._get_ovn_dhcp_options(subnet, network)
ovn_dhcp_options = self._get_ovn_dhcp_options(
subnet, network, metadata_port_ip=metadata_port_ip)
with self._nb_idl.transaction(check_error=True) as txn:
txn.add(self._nb_idl.add_dhcp_options(
subnet['id'], **ovn_dhcp_options))
def _get_ovn_dhcp_options(self, subnet, network, server_mac=None):
def _get_ovn_dhcp_options(self, subnet, network, server_mac=None,
metadata_port_ip=None):
external_ids = {'subnet_id': subnet['id']}
dhcp_options = {'cidr': subnet['cidr'], 'options': {},
'external_ids': external_ids}
@ -799,14 +826,16 @@ class OVNClient(object):
if subnet['enable_dhcp']:
if subnet['ip_version'] == const.IP_VERSION_4:
dhcp_options['options'] = self._get_ovn_dhcpv4_opts(
subnet, network, server_mac=server_mac)
subnet, network, server_mac=server_mac,
metadata_port_ip=metadata_port_ip)
else:
dhcp_options['options'] = self._get_ovn_dhcpv6_opts(
subnet, server_id=server_mac)
return dhcp_options
def _get_ovn_dhcpv4_opts(self, subnet, network, server_mac=None):
def _get_ovn_dhcpv4_opts(self, subnet, network, server_mac=None,
metadata_port_ip=None):
if not subnet['gateway_ip']:
return {}
@ -832,6 +861,10 @@ class OVNClient(object):
# If subnet hostroutes are defined, add them in the
# 'classless_static_route' dhcp option
classless_static_routes = "{"
if metadata_port_ip:
classless_static_routes += ("%s/32,%s, ") % (
metadata_agent.METADATA_DEFAULT_IP, metadata_port_ip)
for route in subnet['host_routes']:
classless_static_routes += ("%s,%s, ") % (
route['destination'], route['nexthop'])
@ -872,7 +905,8 @@ class OVNClient(object):
for dhcp_option in dhcp_options:
txn.add(self._nb_idl.delete_dhcp_options(dhcp_option['uuid']))
def _enable_subnet_dhcp_options(self, subnet, network):
def _enable_subnet_dhcp_options(self, subnet, network,
metadata_port_ip=None):
if utils.is_dhcp_options_ignored(subnet):
return
@ -882,7 +916,8 @@ class OVNClient(object):
ports = [p for p in all_ports if not p['device_owner'].startswith(
const.DEVICE_OWNER_PREFIXES)]
subnet_dhcp_options = self._get_ovn_dhcp_options(subnet, network)
subnet_dhcp_options = self._get_ovn_dhcp_options(
subnet, network, metadata_port_ip=metadata_port_ip)
subnet_dhcp_cmd = self._nb_idl.add_dhcp_options(subnet['id'],
**subnet_dhcp_options)
with self._nb_idl.transaction(check_error=True) as txn:
@ -913,7 +948,8 @@ class OVNClient(object):
lport_name=port['id'],
**columns))
def _update_subnet_dhcp_options(self, subnet, network):
def _update_subnet_dhcp_options(self, subnet, network,
metadata_port_ip=None):
if utils.is_dhcp_options_ignored(subnet):
return
original_options = self._nb_idl.get_subnet_dhcp_options(subnet['id'])
@ -923,7 +959,8 @@ class OVNClient(object):
mac = original_options['options'].get('server_id')
else:
mac = original_options['options'].get('server_mac')
new_options = self._get_ovn_dhcp_options(subnet, network, mac)
new_options = self._get_ovn_dhcp_options(