Merge remote-tracking branch 'origin/fuel-777-vvk-mihgen' into deduplication_yaml

Conflicts:
	examples/example_new_provisioning.yaml
	examples/example_provision_and_deploy.yaml
	lib/astute/deployment_engine.rb
	lib/astute/deployment_engine/nailyfact.rb
This commit is contained in:
Vladmir Sharhsov(warpc) 2013-08-05 12:39:50 +04:00
commit 32a1946453
21 changed files with 928 additions and 221 deletions

View File

@ -16,7 +16,7 @@
if RUBY_VERSION < "1.9"
puts "Astute tested and works only on Ruby 1.9.3 but you use #{RUBY_VERSION}"
puts "Please run astute using ruby -rubygems bin/astute.rb"
puts "If you still want to try it on older versions of ruby, try 'ruby -rubygems bin/astute'"
end
require 'optparse'

View File

@ -32,12 +32,12 @@ node_01: &node1
name: controller-5
hostname: controller-5.domain.tld
profile: centos-x86_64
# fqdn: &fqdn controller-5.domain.tld
# id: &id 5
# uid: *id
# mac: &mac 08:00:27:E3:BC:28
# ip: &ip 10.20.0.41
# power_address: *ip
# fqdn: controller-5.domain.tld
# id: 5
# uid: 5
# mac: 08:00:27:E3:BC:28
# ip: 10.20.0.41
# power_address: 10.20.0.41
<<: *power_info
#Write size in megabytes
ks_meta:
@ -76,6 +76,6 @@ node_01: &node1
eth0:
onboot: 'yes'
peerdns: 'no'
nodes:
- <<: *node1

View File

@ -227,3 +227,14 @@ attributes:
floating_network_range:
- 240.0.12.10
- 240.0.12.11
auth_key: ''
syslog:
syslog_port: '514'
syslog_transport: udp
syslog_server: ''
compute_scheduler_driver: nova.scheduler.filter_scheduler.FilterScheduler
deployment_mode: multinode
cinder:
db_password: rveahKih
user_password: ENwyu6oa
deployment_id: 8

View File

@ -16,7 +16,10 @@ require 'astute/ruby_removed_functions'
require 'json'
require 'logger'
require 'shellwords'
require 'astute/ext/exception'
require 'astute/ext/deep_copy'
require 'astute/config'
require 'astute/logparser'
require 'astute/orchestrator'
@ -36,10 +39,11 @@ module Astute
autoload 'NodesRemover', 'astute/nodes_remover'
autoload 'Node', 'astute/node'
autoload 'NodesHash', 'astute/node'
autoload 'RedhatChecker', 'astute/redhat_checker'
LogParser.autoload :ParseDeployLogs, 'astute/logparser/deployment'
LogParser.autoload :ParseProvisionLogs, 'astute/logparser/provision'
LogParser.autoload :Patterns, 'astute/logparser/parser_patterns'
SUCCESS = 0
FAIL = 1
LOG_PATH = '/var/log/astute.log'
@ -48,7 +52,14 @@ module Astute
unless @logger
@logger = Logger.new(LOG_PATH)
@logger.formatter = proc do |severity, datetime, progname, msg|
severity_map = {'DEBUG' => 'debug', 'INFO' => 'info', 'WARN' => 'warning', 'ERROR' => 'err', 'FATAL' => 'crit'}
severity_map = {
'DEBUG' => 'debug',
'INFO' => 'info',
'WARN' => 'warning',
'ERROR' => 'err',
'FATAL' => 'crit'
}
"#{datetime.strftime("%Y-%m-%dT%H:%M:%S")} #{severity_map[severity]}: [#{Process.pid}] #{msg}\n"
end
end

View File

@ -53,14 +53,18 @@ module Astute
def self.default_config
conf = {}
conf[:PUPPET_TIMEOUT] = 60*60 # maximum time it waits for the whole deployment
conf[:PUPPET_TIMEOUT] = 60 * 60 # maximum time it waits for the whole deployment
conf[:PUPPET_DEPLOY_INTERVAL] = 2 # sleep for ## sec, then check puppet status again
conf[:PUPPET_FADE_TIMEOUT] = 60 # how long it can take for puppet to exit after dumping to last_run_summary
conf[:MC_RETRIES] = 5 # MClient tries to call mcagent before failure
conf[:MC_RETRY_INTERVAL] = 1 # MClient sleeps for ## sec between retries
conf[:PUPPET_FADE_INTERVAL] = 1 # retry every ## seconds to check puppet state if it was running
conf[:PROVISIONING_TIMEOUT] = 90 * 60 # timeout for booting target OS in provision
conf[:REBOOT_TIMEOUT] = 120 # how long it can take for node to reboot
return conf
conf[:REBOOT_TIMEOUT] = 120 # how long it can take for node to reboot
conf[:REDHAT_CHECK_CREDENTIALS_TIMEOUT] = 30 # checking redhat credentials througs mcollective
conf[:REDHAT_GET_LICENSES_POOL_TIMEOUT] = 60 # getting redhat licenses through mcollective
conf
end
end

View File

@ -39,20 +39,6 @@ module Astute
raise "Method #{method} is not implemented for #{self.class}"
end
# def attrs_singlenode(nodes, attrs)
# ctrl_management_ip = nodes[0]['network_data'].select {|nd| nd['name'] == 'management'}[0]['ip']
# ctrl_public_ip = nodes[0]['network_data'].select {|nd| nd['name'] == 'public'}[0]['ip']
# attrs['controller_node_address'] = ctrl_management_ip.split('/')[0]
# attrs['controller_node_public'] = ctrl_public_ip.split('/')[0]
# attrs
# end
# def deploy_singlenode(nodes, attrs)
# # TODO(mihgen) some real stuff is needed
# Astute.logger.info "Starting deployment of single node OpenStack"
# deploy_piece(nodes, attrs)
# end
# we mix all attrs and prepare them for Puppet
# Works for multinode deployment mode
def attrs_multinode(nodes, attrs)
@ -79,58 +65,49 @@ module Astute
# It does only support of deployment sequence. See deploy_piece implementation in subclasses.
def deploy_multinode(nodes, attrs)
ctrl_nodes = nodes.select {|n| n['role'] == 'controller'}
compute_nodes = nodes.select {|n| n['role'] == 'compute'}
other_nodes = nodes - ctrl_nodes
Astute.logger.info "Starting deployment of primary controller"
deploy_piece(ctrl_nodes, attrs)
Astute.logger.info "Starting deployment of other nodes"
deploy_piece(compute_nodes, attrs)
return
deploy_piece(other_nodes, attrs)
return
end
def attrs_ha(nodes, attrs)
# TODO(mihgen): we should report error back if there are not enough metadata passed
#ctrl_nodes = attrs['controller_nodes']
#ctrl_manag_addrs = {}
#ctrl_public_addrs = {}
#ctrl_storage_addrs = {}
#ctrl_nodes.each do |n|
# current puppet modules require `hostname -s`
# hostname = n['fqdn'].split(/\./)[0]
# ctrl_manag_addrs.merge!({hostname =>
# n['network_data'].select {|nd| nd['name'] == 'management'}[0]['ip'].split(/\//)[0]})
# ctrl_public_addrs.merge!({hostname =>
# n['network_data'].select {|nd| nd['name'] == 'public'}[0]['ip'].split(/\//)[0]})
# ctrl_storage_addrs.merge!({hostname =>
# n['network_data'].select {|nd| nd['name'] == 'storage'}[0]['ip'].split(/\//)[0]})
#end
# we use the same set of mount points for all storage nodes
attrs['mp'] = [{'point' => '1', 'weight' => '1'},{'point'=>'2','weight'=>'2'}]
mountpoints = ""
attrs['mp'].each do |mountpoint|
mountpoints << "#{mountpoint['point']} #{mountpoint['weight']}\n"
end
Astute.logger.debug("#{nodes}")
attrs['nodes'] = nodes.map do |n|
{
'fqdn' => n['fqdn'],
'name' => n['fqdn'].split(/\./)[0],
'role' => n['role'],
'mountpoints' => mountpoints,
'internal_address' => n['network_data'].select {|nd| select_ifaces(nd['name'], 'management')}[0]['ip'].split(/\//)[0],
'internal_br' => n['internal_br'],
'internal_netmask' => n['network_data'].select {|nd| select_ifaces(nd['name'], 'management')}[0]['netmask'],
'public_address' => n['network_data'].select {|nd| select_ifaces(nd['name'], 'public')}[0]['ip'].split(/\//)[0],
'public_br' => n['public_br'],
'public_netmask' => n['network_data'].select {|nd| select_ifaces(nd['name'], 'public')}[0]['netmask'],
'mountpoints' => "1 1\n2 2",
'swift_zone' => n['id'],
'storage_address' => n['network_data'].select {|nd| select_ifaces(nd['name'], 'storage')}[0]['ip'].split(/\//)[0],
'storage_netmask' => n['network_data'].select {|nd| select_ifaces(nd['name'], 'storage')}[0]['ip'].split(/\//)[0],
'storage_netmask' => n['network_data'].select {|nd| select_ifaces(nd['name'], 'storage')}[0]['netmask'],
'default_gateway' => n['default_gateway']
}
end
attrs['nodes'].first['role'] = 'primary-controller' if attrs['nodes'].select { |node| node['role'] == "primary-controller" }.empty?
#attrs['ctrl_hostnames'] = ctrl_nodes.map {|n| n['fqdn'].split(/\./)[0]}
#attrs['ctrl_public_addresses'] = ctrl_public_addrs
#attrs['ctrl_management_addresses'] = ctrl_manag_addrs
#attrs['ctrl_storage_addresses'] = ctrl_storage_addrs
#Astute.logger.debug("#{attrs}")
if attrs['nodes'].select { |node| node['role'] == 'primary-controller' }.empty?
ctrl_nodes = attrs['nodes'].select {|n| n['role'] == 'controller'}
ctrl_nodes[0]['role'] = 'primary-controller'
end
attrs
end
@ -140,18 +117,14 @@ module Astute
def deploy_ha_full(nodes, attrs)
primary_ctrl_nodes = nodes.select {|n| n['role'] == 'primary-controller'}
ctrl_nodes = nodes.select {|n| n['role'] == 'controller'}
unless primary_ctrl_nodes.any?
if ctrl_nodes.size > 1
primary_ctrl_nodes = [ctrl_nodes.shift]
end
end
compute_nodes = nodes.select {|n| n['role'] == 'compute'}
quantum_nodes = nodes.select {|n| n['role'] == 'quantum'}
storage_nodes = nodes.select {|n| n['role'] == 'storage'}
proxy_nodes = nodes.select {|n| n['role'] == 'swift-proxy'}
primary_proxy_nodes = nodes.select {|n| n['role'] == 'primary-swift-proxy'}
other_nodes = nodes - ctrl_nodes - primary_ctrl_nodes - \
primary_proxy_nodes - quantum_nodes - storage_nodes
primary_proxy_nodes - quantum_nodes - storage_nodes - proxy_nodes
Astute.logger.info "Starting deployment of primary swift proxy"
deploy_piece(primary_proxy_nodes, attrs)
@ -176,11 +149,6 @@ module Astute
def deploy_ha_compact(nodes, attrs)
primary_ctrl_nodes = nodes.select {|n| n['role'] == 'primary-controller'}
ctrl_nodes = nodes.select {|n| n['role'] == 'controller'}
unless primary_ctrl_nodes.any?
if ctrl_nodes.size > 1
primary_ctrl_nodes = [ctrl_nodes.shift]
end
end
compute_nodes = nodes.select {|n| n['role'] == 'compute'}
quantum_nodes = nodes.select {|n| n['role'] == 'quantum'}
storage_nodes = nodes.select {|n| n['role'] == 'storage'}
@ -189,18 +157,18 @@ module Astute
other_nodes = nodes - ctrl_nodes - primary_ctrl_nodes - \
primary_proxy_nodes - quantum_nodes
#FIXME: add last_controller attribute to attributes hash in order to determine
#if we are the last controller in deployment sequence and it is safe to
#upload test virtual machine image
attrs['last_controller'] = ctrl_nodes.last['name']
Astute.logger.info "Starting deployment of primary controller"
deploy_piece(primary_ctrl_nodes, attrs)
Astute.logger.info "Starting deployment of all controllers one by one"
ctrl_nodes.each {|n| deploy_piece([n], attrs)}
#Astute.logger.info "Starting deployment of 1st controller and 1st proxy"
#deploy_piece(primary_ctrl_nodes + primary_proxy_nodes, attrs)
#Astute.logger.info "Starting deployment of quantum nodes"
#deploy_piece(quantum_nodes, attrs)
Astute.logger.info "Starting deployment of other nodes"
deploy_piece(other_nodes, attrs)
return
@ -224,8 +192,9 @@ module Astute
result = true if var.include?(name)
elsif var.is_a?(String)
result = true if var == name
end
end
end
end
def nodes_status(nodes, status, data_to_merge)
{'nodes' => nodes.map { |n| {'uid' => n['uid'], 'status' => status}.merge(data_to_merge) }}
end
@ -284,4 +253,4 @@ module Astute
interfaces
end
end
end
end

View File

@ -27,21 +27,12 @@ class Astute::DeploymentEngine::NailyFact < Astute::DeploymentEngine
node_network_data = node['network_data'].nil? ? [] : node['network_data']
interfaces = node['meta']['interfaces']
network_data_puppet = calculate_networks(node_network_data, interfaces)
metadata = {
attrs_to_puppet = {
'role' => node['role'],
'uid' => node['uid'],
'network_data' => network_data_puppet.to_json
}
attrs.each do |k, v|
if v.is_a? String
metadata[k] = v
else
# And it's the problem on the puppet side now to decode json
metadata[k] = v.to_json
end
end
# Let's calculate interface settings we need for OpenStack:
node_network_data.each do |iface|
device = if iface['vlan'] && iface['vlan'] > 0
@ -51,48 +42,70 @@ class Astute::DeploymentEngine::NailyFact < Astute::DeploymentEngine
end
if iface['name'].is_a?(String)
metadata["#{iface['name']}_interface"] = device
if iface['ip']
# metadata["#{iface['name']}_address"] = iface['ip'].split('/')[0]
end
attrs_to_puppet["#{iface['name']}_interface"] = device
elsif iface['name'].is_a?(Array)
iface['name'].each do |name|
metadata["#{name}_interface"] = device
if iface['ip']
# metadata["#{name}_address"] = iface['ip'].split('/')[0]
end
attrs_to_puppet["#{name}_interface"] = device
end
end
end
# internal_address is required for HA..
#metadata['internal_address'] = node['network_data'].select{|nd| select_ifaces(nd['name'], 'management')}[0]['ip'].split('/')[0]
if attrs['novanetwork_parameters'] && \
attrs['novanetwork_parameters']['network_manager'] == 'VlanManager' && \
!attrs_to_puppet['fixed_interface']
if metadata['network_manager'] == 'VlanManager' && !metadata['fixed_interface']
metadata['fixed_interface'] = get_fixed_interface(node)
attrs_to_puppet['fixed_interface'] = get_fixed_interface(node)
end
metadata
attrs_to_puppet.merge!(deep_copy(attrs))
attrs_to_puppet.each do |k, v|
unless v.is_a?(String) || v.is_a?(Integer)
attrs_to_puppet[k] = v.to_json
end
end
attrs_to_puppet
end
def deploy_piece(nodes, attrs, retries=2, change_node_status=true)
return false unless validate_nodes(nodes)
@ctx.reporter.report nodes_status(nodes, 'deploying', {'progress' => 0})
Astute.logger.info "#{@ctx.task_id}: Getting which nodes to deploy"
Astute.logger.debug "#{@ctx.task_id}: Running get_nodes_to_deploy() function:::"
nodes_to_deploy = get_nodes_to_deploy(nodes)
if nodes_to_deploy.empty?
Astute.logger.info "#{@ctx.task_id}: Returning from deployment stage. No nodes to deploy"
return
end
Astute.logger.info "#{@ctx.task_id}: Calculation of required attributes to pass, include netw.settings"
nodes.each do |node|
@ctx.reporter.report nodes_status(nodes_to_deploy, 'deploying', {'progress' => 0})
nodes_to_deploy.each do |node|
# Use predefined facts or create new.
node['facts'] ||= create_facts(node, attrs)
Astute::Metadata.publish_facts(@ctx, node['uid'], node['facts'])
end
Astute.logger.info "#{@ctx.task_id}: All required attrs/metadata passed via facts extension. Starting deployment."
Astute::PuppetdDeployer.deploy(@ctx, nodes, retries, change_node_status)
nodes_roles = nodes.map { |n| { n['uid'] => n['role'] } }
Astute::PuppetdDeployer.deploy(@ctx, nodes_to_deploy, retries, change_node_status)
nodes_roles = nodes_to_deploy.map { |n| { n['uid'] => n['role'] } }
Astute.logger.info "#{@ctx.task_id}: Finished deployment of nodes => roles: #{nodes_roles.inspect}"
end
private
def get_nodes_to_deploy(nodes)
nodes_to_deploy = []
nodes.each do |node|
if node['status'] != 'ready'
nodes_to_deploy << node
else
Astute.logger.info "#{@ctx.task_id}: Not adding node #{node['uid']} with hostname #{node['name']} as it does not require deploying."
end
end
nodes_to_deploy
end
def get_fixed_interface(node)
return node['vlan_interface'] if node['vlan_interface']
@ -100,4 +113,4 @@ class Astute::DeploymentEngine::NailyFact < Astute::DeploymentEngine
nil
end
end
end

View File

@ -0,0 +1,18 @@
# Copyright 2013 Mirantis, 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.
def deep_copy(data)
Marshal.load(Marshal.dump(data))
end

View File

@ -0,0 +1,20 @@
# Copyright 2013 Mirantis, 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.
class Exception
def format_backtrace
backtrace.join("\n")
end
end

View File

@ -1,3 +1,5 @@
# -*- coding: utf-8 -*-
# Copyright 2013 Mirantis, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License"); you may
@ -13,7 +15,6 @@
# under the License.
# -*- coding: utf-8 -*-
require 'erb'
module Astute
@ -37,6 +38,93 @@ module Astute
end
end
class DirSizeCalculation
attr_reader :nodes
def initialize(nodes)
@nodes = nodes.map{|n| n.dup}
@nodes.each{|node| node[:path_items] = weight_reassignment(node[:path_items])}
end
def deploy_type=(*args)
# Because we mimic the DeploymentParser, we should define all auxiliary method
# even they do nothing.
end
def progress_calculate(uids_to_calc, nodes)
uids_to_calc.map do |uid|
node = @nodes.find{|n| n[:uid] == uid}
node[:path_items] ||= []
progress = 0
node[:path_items].each do |item|
size = recursive_size(item[:path])
sub_progress = 100 * size / item[:max_size]
sub_progress = 0 if sub_progress < 0
sub_progress = 100 if sub_progress > 100
progress += sub_progress * item[:weight]
end
{'uid' => uid, 'progress' => progress.to_i}
end
end
private
def recursive_size(path, opts={})
return File.size?(path).to_i if not File.directory?(path)
total_size = 0
Dir[File.join("#{path}", '**/*')].each do |f|
# Option :files_only used when you want to calculate total size of
# regular files only. The default :files_only is false, so the function will
# include inode size of each dir (4096 bytes in most cases) to total value
# as the unix util 'du' does it.
total_size += File.size?(f).to_i if File.file?(f) || ! opts[:files_only]
end
total_size
end
def weight_reassignment(items)
# The finction normalizes the weights of each item in order to make sum of
# all weights equal to one.
# It divides items as wighted and unweighted depending on the existence of
# the :weight key in the item.
# - Each unweighted item will be weighted as a one N-th part of the total number of items.
# - All weights of weighted items are summed up and then each weighted item
# gets a new weight as a multiplication of a relative weight among all
# weighted items and the ratio of the number of the weighted items to
# the total number of items.
# E.g. we have four items: one with weight 0.5, another with weight 1.5, and
# two others as unweighted. All unweighted items will get the weight 1/4.
# Weight's sum of weighted items is 2. So the first item will get the weight:
# (relative weight 0.5/2) * (weighted items ratio 2/4) = 1/8.
# Finally all items will be normalised with next weights:
# 1/8, 3/8, 1/4, and 1/4.
ret_items = items.reject do |item|
weight = item[:weight]
# Save an item if it unweighted.
next if weight.nil?
raise "Weight should be a non-negative number" unless [Fixnum, Float].include?(weight.class) && weight >= 0
# Drop an item if it weighted as zero.
item[:weight] == 0
end
return [] if ret_items.empty?
ret_items.map!{|n| n.dup}
partial_weight = 1.0 / ret_items.length
weighted_items = ret_items.select{|n| n[:weight]}
weighted_sum = 0.0
weighted_items.each{|n| weighted_sum += n[:weight]}
weighted_sum = weighted_sum * ret_items.length / weighted_items.length if weighted_items.any?
raise "Unexpectedly a summary weight of weighted items is a non-positive" if weighted_items.any? && weighted_sum <= 0
ret_items.each do |item|
weight = item[:weight]
item[:weight] = weight ? weight / weighted_sum : partial_weight
end
ret_items
end
end
class ParseNodeLogs
attr_reader :pattern_spec
@ -61,7 +149,7 @@ module Astute
begin
progress = (get_log_progress(path, node_pattern_spec)*100).to_i # Return percent of progress
rescue Exception => e
Astute.logger.warn "Some error occurred when calculate progress for node '#{uid}': #{e.message}, trace: #{e.backtrace.join("\n")}"
Astute.logger.warn "Some error occurred when calculate progress for node '#{uid}': #{e.message}, trace: #{e.format_backtrace}"
progress = 0
end

View File

@ -16,7 +16,7 @@ module Astute
class Orchestrator
def initialize(deploy_engine=nil, log_parsing=false)
@deploy_engine = deploy_engine || Astute::DeploymentEngine::NailyFact
@log_parser = log_parsing ? LogParser::ParseDeployLogs.new : LogParser::NoParsing.new
@log_parsing = log_parsing
end
def node_type(reporter, task_id, nodes, timeout=nil)
@ -37,27 +37,28 @@ module Astute
# Following line fixes issues with uids: it should always be string
nodes.map { |x| x['uid'] = x['uid'].to_s } # NOTE: perform that on environment['nodes'] initialization
proxy_reporter = ProxyReporter::DeploymentProxyReporter.new(up_reporter)
context = Context.new(task_id, proxy_reporter, @log_parser)
log_parser = @log_parsing ? LogParser::ParseDeployLogs.new : LogParser::NoParsing.new
context = Context.new(task_id, proxy_reporter, log_parser)
deploy_engine_instance = @deploy_engine.new(context)
Astute.logger.info "Using #{deploy_engine_instance.class} for deployment."
begin
@log_parser.prepare(nodes)
log_parser.prepare(nodes)
rescue Exception => e
Astute.logger.warn "Some error occurred when prepare LogParser: #{e.message}, trace: #{e.backtrace.join("\n")}"
Astute.logger.warn "Some error occurred when prepare LogParser: #{e.message}, trace: #{e.format_backtrace}"
end
deploy_engine_instance.deploy(nodes, attrs)
return SUCCESS
end
def fast_provision(reporter, engine_attrs, nodes)
raise "Nodes to provision are not provided!" if nodes.empty?
engine = create_engine(engine_attrs, reporter)
begin
reboot_events = reboot_nodes(engine, nodes)
failed_nodes = check_reboot_nodes(engine, reboot_events)
rescue RuntimeError => e
Astute.logger.error("Error occured while provisioning: #{e.inspect}")
reporter.report({
@ -69,7 +70,7 @@ module Astute
ensure
engine.sync
end
if failed_nodes.empty?
report_result({}, reporter)
return SUCCESS
@ -83,23 +84,29 @@ module Astute
raise StopIteration
end
end
def provision(reporter, task_id, nodes)
raise "Nodes to provision are not provided!" if nodes.empty?
def provision(reporter, task_id, nodes_up)
raise "Nodes to provision are not provided!" if nodes_up.empty?
# We need only those which are not ready/provisioned yet
nodes = []
nodes_up.each do |n|
nodes << n unless ['provisioned', 'ready'].include?(n['status'])
end
# Following line fixes issues with uids: it should always be string
nodes.map { |x| x['uid'] = x['uid'].to_s } # NOTE: perform that on environment['nodes'] initialization
nodes_uids = nodes.map { |n| n['uid'] }
provisionLogParser = LogParser::ParseProvisionLogs.new
provisionLogParser = @log_parsing ? LogParser::ParseProvisionLogs.new : LogParser::NoParsing.new
proxy_reporter = ProxyReporter::DeploymentProxyReporter.new(reporter)
sleep_not_greater_than(10) do # Wait while nodes going to reboot
Astute.logger.info "Starting OS provisioning for nodes: #{nodes_uids.join(',')}"
begin
provisionLogParser.prepare(nodes)
rescue => e
Astute.logger.warn "Some error occurred when prepare LogParser: #{e.message}, trace: #{e.backtrace.join("\n")}"
Astute.logger.warn "Some error occurred when prepare LogParser: #{e.message}, trace: #{e.format_backtrace}"
end
end
nodes_not_booted = nodes_uids.clone
@ -107,15 +114,15 @@ module Astute
Timeout.timeout(Astute.config.PROVISIONING_TIMEOUT) do # Timeout for booting target OS
catch :done do
while true
sleep_not_greater_than(5) do
sleep_not_greater_than(5) do
types = node_type(proxy_reporter, task_id, nodes, 2)
types.each { |t| Astute.logger.debug("Got node types: uid=#{t['uid']} type=#{t['node_type']}") }
Astute.logger.debug("Not target nodes will be rejected")
target_uids = types.reject{|n| n['node_type'] != 'target'}.map{|n| n['uid']}
nodes_not_booted -= types.map { |n| n['uid'] }
Astute.logger.debug "Not provisioned: #{nodes_not_booted.join(',')}, got target OSes: #{target_uids.join(',')}"
if nodes.length == target_uids.length
Astute.logger.info "All nodes #{target_uids.join(',')} are provisioned."
throw :done
@ -123,11 +130,11 @@ module Astute
Astute.logger.debug("Nodes list length is not equal to target nodes list length: #{nodes.length} != #{target_uids.length}")
end
report_about_progress(proxy_reporter, provisionLogParser, nodes_uids, target_uids, nodes)
report_about_progress(proxy_reporter, provisionLogParser, nodes_uids, target_uids, nodes)
end
end
end
# We are here if jumped by throw from while cycle
# We are here if jumped by throw from while cycle
end
rescue Timeout::Error
msg = "Timeout of provisioning is exceeded."
@ -147,7 +154,6 @@ module Astute
proxy_reporter.report({'nodes' => nodes_progress})
return SUCCESS
end
def remove_nodes(reporter, task_id, nodes)
NodesRemover.new(Context.new(task_id, reporter), nodes).remove
@ -172,42 +178,67 @@ module Astute
end
nodes = [{'uid' => 'master', 'facts' => facts}]
proxy_reporter = ProxyReporter::DLReleaseProxyReporter.new(up_reporter, nodes.size)
context = Context.new(task_id, proxy_reporter, @log_parser)
#FIXME: These parameters should be propagated from Nailgun. Maybe they should be saved
# in Release.json.
nodes_to_parser = [
{:uid => 'master',
:path_items => [
{:max_size => 1111280705, :path => '/var/www/nailgun/rhel', :weight => 3},
{:max_size => 195900000, :path => '/var/cache/yum/x86_64/6Server', :weight => 1},
]}
]
log_parser = @log_parsing ? LogParser::DirSizeCalculation.new(nodes_to_parser) : LogParser::NoParsing.new
context = Context.new(task_id, proxy_reporter, log_parser)
deploy_engine_instance = @deploy_engine.new(context)
Astute.logger.info "Using #{deploy_engine_instance.class} for release download."
begin
@log_parser.prepare(nodes)
rescue Exception => e
Astute.logger.warn "Some error occurred when prepare LogParser: #{e.message}, trace: #{e.backtrace.join("\n")}"
end
deploy_engine_instance.deploy(nodes, attrs)
proxy_reporter.report({'status' => 'ready', 'progress' => 100})
end
def check_redhat_credentials(reporter, task_id, credentials)
ctx = Context.new(task_id, reporter)
begin
Astute::RedhatChecker.new(ctx, credentials).check_redhat_credentials
rescue Exception => e
Astute.logger.error("Error #{e.message} traceback #{e.format_backtrace}")
raise StopIteration
end
end
def check_redhat_licenses(reporter, task_id, credentials, nodes=nil)
ctx = Context.new(task_id, reporter)
begin
Astute::RedhatChecker.new(ctx, credentials).check_redhat_licenses(nodes)
rescue Exception => e
Astute.logger.error("Error #{e.message} traceback #{e.format_backtrace}")
raise StopIteration
end
end
private
def report_result(result, reporter)
default_result = {'status' => 'ready', 'progress' => 100}
result = {} unless result.instance_of?(Hash)
status = default_result.merge(result)
reporter.report(status)
end
def sleep_not_greater_than(sleep_time, &block)
time = Time.now.to_f
block.call
time = time + sleep_time - Time.now.to_f
sleep (time) if time > 0
end
def create_engine(engine_attrs, reporter)
begin
Astute.logger.info("Trying to instantiate cobbler engine: #{engine_attrs.inspect}")
Astute::Provision::Cobbler.new(engine_attrs)
rescue
Astute.logger.error("Error occured during cobbler initializing")
reporter.report({
'status' => 'error',
'error' => 'Cobbler can not be initialized',
@ -216,7 +247,7 @@ module Astute
raise StopIteration
end
end
def reboot_nodes(engine, nodes)
reboot_events = {}
nodes.each do |node|
@ -233,7 +264,7 @@ module Astute
end
reboot_events
end
def check_reboot_nodes(engine, reboot_events)
begin
Astute.logger.debug("Waiting for reboot to be complete: nodes: #{reboot_events.keys}")
@ -261,7 +292,7 @@ module Astute
end
failed_nodes
end
def report_about_progress(reporter, provisionLogParser, nodes_uids, target_uids, nodes)
begin
nodes_progress = provisionLogParser.progress_calculate(nodes_uids, nodes)
@ -275,9 +306,9 @@ module Astute
end
reporter.report({'nodes' => nodes_progress})
rescue => e
Astute.logger.warn "Some error occurred when parse logs for nodes progress: #{e.message}, trace: #{e.backtrace.join("\n")}"
Astute.logger.warn "Some error occurred when parse logs for nodes progress: #{e.message}, trace: #{e.format_backtrace}"
end
end
end
end

View File

@ -148,7 +148,7 @@ module Astute
end
rescue Exception => e
Astute.logger.warn "Some error occurred when parse logs for nodes progress: #{e.message}, "\
"trace: #{e.backtrace.join("\n")}"
"trace: #{e.format_backtrace}"
end
end
ctx.reporter.report('nodes' => nodes_to_report) if nodes_to_report.any?

View File

@ -0,0 +1,152 @@
# -*- coding: utf-8 -*-
# Copyright 2013 Mirantis, 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.
module Astute
class RedhatCheckingError < Exception; end
class RedhatChecker
def initialize(ctx, credentials)
@ctx = ctx
@username = Shellwords.escape(credentials['redhat']['username'])
@password = Shellwords.escape(credentials['redhat']['password'])
release_name = credentials['release_name']
@network_error = 'Unable to reach host cdn.redhat.com. ' + \
'Please check your Internet connection.'
@user_does_not_have_licenses = 'Could not find any valid Red Hat ' + \
'OpenStack subscriptions. Contact your Red Hat sales representative ' + \
'to get the proper subscriptions associated with your account: '+ \
'https://access.redhat.com/site/solutions/368643. If you are still ' + \
'encountering issues, contact Mirantis Support.'
@msg_not_enough_licenses = "Your account has only %d licenses " + \
'available to deploy Red Hat OpenStack. Contact your Red Hat sales ' + \
'representative to get the proper subscriptions associated with your ' + \
'account. https://access.redhat.com/site/solutions/368643'
@check_credentials_success = "Account information for #{release_name} has been successfully modified."
@common_errors = {
/^Network error|^Remote server error/ => @network_error,
/^Invalid username or password/ => 'Invalid username or password. ' + \
'To create a login, please visit https://www.redhat.com/wapps/ugc/register.html'
}
end
# Checking redhat credentials
def check_redhat_credentials
timeout = Astute.config[:REDHAT_CHECK_CREDENTIALS_TIMEOUT]
check_credentials_cmd = "subscription-manager orgs " + \
"--username #{@username} " + \
"--password #{@password}"
shell = MClient.new(@ctx, 'execute_shell_command', ['master'])
response = {}
begin
Timeout.timeout(timeout) do
response = shell.execute(:cmd => check_credentials_cmd).first
end
rescue Timeout::Error
Astute.logger.warn("Time out error for shell command '#{check_credentials_cmd}'")
report_error(@network_error)
end
report(response.results[:data], @common_errors, @check_credentials_success)
end
# Check redhat linceses and return message, if not enough licenses
def check_redhat_licenses(nodes=nil)
response = execute_get_licenses
report_error(@network_error) unless response
licenses_count = nil
begin
licenses_pool = JSON.load(response.results[:data][:stdout])
licenses_count = licenses_pool['openstack_licenses_physical_hosts_count']
rescue JSON::ParserError
report(response.results[:data], @common_errors)
return
end
if licenses_count <= 0
report_error(@user_does_not_have_licenses)
elsif nodes && licenses_count < nodes.count
report_success(format(@msg_not_enough_licenses, licenses_count))
else
report_success
end
end
private
def report(result, errors, success_msg=nil)
stdout = result[:stdout]
stderr = result[:stderr]
exit_code = result[:exit_code]
if !get_error(result, errors) && exit_code == 0
report_success(success_msg)
else
err_msg = "Unknown error Stdout: #{stdout} Stderr: #{stderr}"
error = get_error(result, errors) || err_msg
report_error(error)
end
end
def get_error(result, errors)
errors.each_pair do |regex, msg|
return msg if regex.match(result[:stdout])
return msg if regex.match(result[:stderr])
end
nil
end
def report_success(msg=nil)
success_msg = {'status' => 'ready', 'progress' => 100}
success_msg.merge!({'msg' => msg}) if msg
@ctx.reporter.report(success_msg)
end
# Report error and raise exception
def report_error(msg)
@ctx.reporter.report({'status' => 'error', 'error' => msg, 'progress' => 100})
raise RedhatCheckingError.new(msg)
end
def execute_get_licenses
timeout = Astute.config[:REDHAT_GET_LICENSES_POOL_TIMEOUT]
get_redhat_licenses_cmd = "get_redhat_licenses " + \
"#{@username} " + \
"#{@password}"
shell = MClient.new(@ctx, 'execute_shell_command', ['master'])
begin
Timeout.timeout(timeout) do
return shell.execute(:cmd => get_redhat_licenses_cmd).first
end
rescue Timeout::Error
Astute.logger.warn("Time out error for shell command '#{get_redhat_licenses_cmd}'")
end
end
end
end

View File

@ -0,0 +1,30 @@
metadata :name => "Execute shell command",
:description => "Execute shell command",
:author => "Mirantis Inc.",
:license => "Apache License 2.0",
:version => "0.0.1",
:url => "http://mirantis.com",
:timeout => 600
action "execute", :description => "Execute shell command" do
input :cmd,
:prompt => "Shell command",
:description => "Shell command for running",
:type => :string,
:validation => '.*',
:optional => false,
:maxlength => 0
output :stdout,
:description => "Output from #{:cmd}",
:display_as => "Output"
output :stderr,
:description => "Stderr from #{:cmd}",
:display_as => "Stderr"
output :exit_code,
:description => "Exit code of #{:cmd}",
:display_as => "Exit code"
end

View File

@ -0,0 +1,35 @@
# -*- coding: utf-8 -*-
# Copyright 2013 Mirantis, 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.
module MCollective
module Agent
class Execute_shell_command < RPC::Agent
action 'execute' do
reply[:stdout], reply[:stderr], reply[:exit_code] = run_shell_command(request[:cmd])
end
private
def run_shell_command(command)
shell = Shell.new(command)
shell.runcommand
[shell.stdout, shell.stderr, shell.status.exitstatus]
end
end
end
end

View File

@ -15,6 +15,7 @@
require 'tempfile'
require 'tmpdir'
require 'fileutils'
require 'date'
require 'yaml'
require 'rspec'
@ -42,7 +43,10 @@ module SpecHelpers
stubs(:progress=)
unless timeout.nil?
expects(:timeout=).with(timeout)
else
stubs(:timeout=)
end
if discover_nodes.nil?
stubs(:discover)
else

View File

@ -273,6 +273,186 @@ describe LogParser do
calculated_nodes = deployment_parser_wrapper('multinode', nodes)
calculated_nodes.each {|node| node['statistics']['pcc'].should > 0.94}
end
end
context "Dirsize-based progress calculation" do
def create_dir_with_size(size, given_opts={})
raise "The required size should be a non-negative number" if size < 0
default_opts = {
:chunksize => 10000,
:tmpdir => Dir::tmpdir,
}
opts = default_opts.merge(given_opts)
if !opts[:chunksize].instance_of?(Fixnum) || opts[:chunksize] <= 0
raise "The 'chunksize' option should be a positive number"
end
raise "The 'tmpdir' option should be a path to a existent directory" if !opts[:tmpdir].instance_of?(String)
dir = Dir::mktmpdir(nil, opts[:tmpdir])
chunk = 'A' * opts[:chunksize]
while size >= opts[:chunksize]
Tempfile::open('prefix', dir){|file| file.write(chunk)}
size -= opts[:chunksize]
end
Tempfile::open('prefix', dir){|file| file.write('A' * size)} if size > 0
return dir
end
it "should correctly calculate size of directory" do
size = 10**6
dir = create_dir_with_size(size)
nodes = [
{:uid => '1',
:path_items => [
{:max_size => size*100/75,
:path => dir}
]
}
]
correct_progress = [
{'uid' => '1',
'progress' => 75}
]
dirsize_parser = Astute::LogParser::DirSizeCalculation.new(nodes)
dirsize_parser.progress_calculate(['1'], nil).should eql(correct_progress)
FileUtils::remove_entry_secure dir
end
it "should correctly calculate size of nested directories" do
size = 10**6
dir = create_dir_with_size(size)
create_dir_with_size(size, {:tmpdir => dir})
nodes = [
{:uid => '1',
:path_items => [
{:max_size => size*2,
:path => dir}
]
}
]
correct_progress = [
{'uid' => '1',
'progress' => 100}
]
dirsize_parser = Astute::LogParser::DirSizeCalculation.new(nodes)
dirsize_parser.progress_calculate(['1'], nil).should eql(correct_progress)
FileUtils::remove_entry_secure dir
end
it "should return zero if there is no directory" do
nodes = [
{:uid => '1',
:path_items => [
{:max_size => 10000,
:path => '/the-dir-that-should-not-exist'}
]
}
]
correct_progress = [
{'uid' => '1',
'progress' => 0}
]
dirsize_parser = Astute::LogParser::DirSizeCalculation.new(nodes)
dirsize_parser.progress_calculate(['1'], nil).should eql(correct_progress)
end
it "should return zero if no items is propagated" do
nodes = [
{:uid => '1',
:path_items => []
}
]
correct_progress = [
{'uid' => '1',
'progress' => 0}
]
dirsize_parser = Astute::LogParser::DirSizeCalculation.new(nodes)
dirsize_parser.progress_calculate(['1'], nil).should eql(correct_progress)
end
end
context "Dirsize-based weight reassignment" do
it "should correctly assign weights to unweighted items" do
nodes = [
{:uid => '1',
:path_items => [{}, {}, {}, {}]
}
]
dirsize_parser = Astute::LogParser::DirSizeCalculation.new(nodes)
dirsize_parser.nodes.first[:path_items].each{|n| n[:weight].should eql(0.25)}
end
it "should correctly recalculate weights of weighted items" do
nodes = [
{:uid => '1',
:path_items => [
{:weight => 10},
{:weight => 30},
]
}
]
dirsize_parser = Astute::LogParser::DirSizeCalculation.new(nodes)
items = dirsize_parser.nodes.first[:path_items]
items[0][:weight].should eql(0.25)
items[1][:weight].should eql(0.75)
end
it "should correctly recalculate weights of mixed items" do
nodes = [
{:uid => '1',
:path_items => [
{:weight => 10},
{:weight => 30},
{}, {}
]
}
]
dirsize_parser = Astute::LogParser::DirSizeCalculation.new(nodes)
items = dirsize_parser.nodes.first[:path_items]
items[0][:weight].should eql(0.125)
items[1][:weight].should eql(0.375)
items[2][:weight].should eql(0.25)
items[3][:weight].should eql(0.25)
end
it "should raise exception if a negative weight propagated" do
nodes = [
{:uid => '1',
:path_items => [
{:weight => -10},
]
}
]
expect{Astute::LogParser::DirSizeCalculation.new(nodes)}.to \
raise_error("Weight should be a non-negative number")
end
it "should drop items with zero weight" do
nodes = [
{:uid => '1',
:path_items => [
{:weight => 0},
{:weight => 0},
]
}
]
dirsize_parser = Astute::LogParser::DirSizeCalculation.new(nodes)
dirsize_parser.nodes.first[:path_items].length.should eql(0)
end
it "should not change initialization attribute" do
nodes = [
{:uid => '1',
:path_items => [
{:weight => 0},
{:weight => 5},
{}
]
}
]
dirsize_parser = Astute::LogParser::DirSizeCalculation.new(nodes)
dirsize_parser.nodes.should_not eql(nodes)
end
end
end

View File

@ -196,7 +196,6 @@ describe "NailyFact DeploymentEngine" do
controller_nodes = @data_ha['args']['nodes'].select{|n| n['role'] == 'controller'}
primary_nodes = [controller_nodes.shift]
compute_nodes = @data_ha['args']['nodes'].select{|n| n['role'] == 'compute'}
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, primary_nodes, 0, false).once
controller_nodes.each do |n|
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, [n], 2, true).once
end
@ -212,14 +211,6 @@ describe "NailyFact DeploymentEngine" do
@deploy_engine.deploy([ctrl], @data_ha['args']['attributes'])
end
it "singlenode deploy should not raise any exception" do
@data['args']['attributes']['deployment_mode'] = "singlenode"
@data['args']['nodes'] = [@data['args']['nodes'][0]] # We have only one node in singlenode
Astute::Metadata.expects(:publish_facts).times(@data['args']['nodes'].size)
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, @data['args']['nodes'], instance_of(Fixnum), true).once
@deploy_engine.deploy(@data['args']['nodes'], @data['args']['attributes'])
end
describe 'Vlan manager' do
it 'Should set fixed_interface value' do
node = {
@ -245,12 +236,14 @@ describe "NailyFact DeploymentEngine" do
}
}
attrs = {
'network_manager' => 'VlanManager'
'novanetwork_parameters' => {
'network_manager' => 'VlanManager'
}
}
expect = {
"role" => "controller",
"uid"=>1,
"uid" => 1,
"network_data" => {"eth0.102" =>
{
@ -272,10 +265,8 @@ describe "NailyFact DeploymentEngine" do
}.to_json,
"fixed_interface" => "eth2",
"network_manager" => "VlanManager",
"management_interface" => "eth0.102",
"internal_address" => "192.168.0.2",
'management_address' => '192.168.0.2'
"novanetwork_parameters" => '{"network_manager":"VlanManager"}',
"management_interface" => "eth0.102"
}
@deploy_engine.create_facts(node, attrs).should == expect

View File

@ -217,15 +217,14 @@ describe Astute::Orchestrator do
end
it "remove_nodes do not fail if any of nodes failed"
before(:all) do
@data = {
"engine"=>{
"url"=>"http://localhost/cobbler_api",
"username"=>"cobbler",
"url"=>"http://localhost/cobbler_api",
"username"=>"cobbler",
"password"=>"cobbler"
},
},
"task_uuid"=>"a5c44b9a-285a-4a0c-ae65-2ed6b3d250f4",
"nodes" => [
{
@ -270,19 +269,19 @@ describe Astute::Orchestrator do
]
}.freeze
end
describe '#fast_provision' do
context 'cobler cases' do
it "raise error if cobler settings empty" do
expect {@orchestrator.fast_provision(@reporter, {}, @data['nodes'])}.
to raise_error(StopIteration)
end
end
context 'node state cases' do
before(:each) do
remote = mock() do
stubs(:call)
stubs(:call).with('login', 'cobbler', 'cobbler').returns('remotetoken')
@ -292,12 +291,12 @@ describe Astute::Orchestrator do
stubs(:new).returns(remote)
end
end
it "raises error if nodes list is empty" do
expect {@orchestrator.fast_provision(@reporter, @data['engine'], {})}.
to raise_error(/Nodes to provision are not provided!/)
end
it "try to reboot nodes from list" do
Astute::Provision::Cobbler.any_instance do
expects(:power_reboot).with('controller-1')
@ -305,39 +304,39 @@ describe Astute::Orchestrator do
@orchestrator.stubs(:check_reboot_nodes).returns([])
@orchestrator.fast_provision(@reporter, @data['engine'], @data['nodes'])
end
before(:each) { Astute::Provision::Cobbler.any_instance.stubs(:power_reboot).returns(333) }
context 'node reboot success' do
before(:each) { Astute::Provision::Cobbler.any_instance.stubs(:event_status).
returns([Time.now.to_f, 'controller-1', 'complete'])}
it "does not find failed nodes" do
Astute::Provision::Cobbler.any_instance.stubs(:event_status).
returns([Time.now.to_f, 'controller-1', 'complete'])
@orchestrator.fast_provision(@reporter, @data['engine'], @data['nodes'])
end
it "report about success" do
@reporter.expects(:report).with({'status' => 'ready', 'progress' => 100}).returns(true)
@orchestrator.fast_provision(@reporter, @data['engine'], @data['nodes'])
end
it "sync engine state" do
Astute::Provision::Cobbler.any_instance do
expects(:sync).once
end
@orchestrator.fast_provision(@reporter, @data['engine'], @data['nodes'])
end
end
context 'node reboot fail' do
before(:each) { Astute::Provision::Cobbler.any_instance.stubs(:event_status).
returns([Time.now.to_f, 'controller-1', 'failed'])}
it "should sync engine state" do
Astute::Provision::Cobbler.any_instance do
expects(:sync).once
@ -347,77 +346,76 @@ describe Astute::Orchestrator do
rescue
end
end
it "raise error if failed node find" do
expect {@orchestrator.fast_provision(@reporter, @data['engine'], @data['nodes'])}.to raise_error(StopIteration)
end
end
end
end
describe '#provision' do
before(:each) do
# Disable sleeping in test env (doubles the test speed)
def @orchestrator.sleep_not_greater_than(time, &block)
block.call
end
end
it "raises error if nodes list is empty" do
expect {@orchestrator.provision(@reporter, @data['task_uuid'], {})}.
to raise_error(/Nodes to provision are not provided!/)
end
it "prepare provision log for parsing" do
Astute::LogParser::ParseProvisionLogs.any_instance do
expects(:prepare).with(@data['nodes']).once
end
@orchestrator.stubs(:report_about_progress).returns()
@orchestrator.stubs(:node_type).returns([{'uid' => '1', 'node_type' => 'target' }])
@orchestrator.provision(@reporter, @data['task_uuid'], @data['nodes'])
end
it "ignore problem with parsing provision log" do
Astute::LogParser::ParseProvisionLogs.any_instance do
stubs(:prepare).with(@data['nodes']).raises
end
@orchestrator.stubs(:report_about_progress).returns()
@orchestrator.stubs(:node_type).returns([{'uid' => '1', 'node_type' => 'target' }])
@orchestrator.provision(@reporter, @data['task_uuid'], @data['nodes'])
end
it 'provision nodes using mclient' do
@orchestrator.stubs(:report_about_progress).returns()
@orchestrator.expects(:node_type).returns([{'uid' => '1', 'node_type' => 'target' }])
@orchestrator.provision(@reporter, @data['task_uuid'], @data['nodes'])
end
it "fail if timeout of provisioning is exceeded" do
Astute::LogParser::ParseProvisionLogs.any_instance do
stubs(:prepare).returns()
end
Timeout.stubs(:timeout).raises(Timeout::Error)
msg = 'Timeout of provisioning is exceeded.'
msg = 'Timeout of provisioning is exceeded.'
error_mgs = {'status' => 'error', 'error' => msg, 'nodes' => [{ 'uid' => '1',
'status' => 'error',
'error_msg' => msg,
'progress' => 100,
'error_type' => 'provision'}]}
@reporter.expects(:report).with(error_mgs).once
@orchestrator.provision(@reporter, @data['task_uuid'], @data['nodes'])
end
end
end
end
end

View File

@ -0,0 +1,159 @@
# -*- encoding: utf-8 -*-
# Copyright 2013 Mirantis, 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.
require File.join(File.dirname(__FILE__), '../spec_helper')
describe Astute::RedhatChecker do
include SpecHelpers
let(:redhat_credentials) do
{
'release_name' => 'RELEASE_NAME',
'redhat' => {
'username' => 'user',
'password' => 'password'
}
}
end
let(:reporter) { mock('reporter') }
let(:ctx) { Astute::Context.new('task-uuuid', reporter) }
let(:redhat_checker) { described_class.new(ctx, redhat_credentials) }
let!(:rpcclient) { mock_rpcclient }
let(:success_result) { {'status' => 'ready', 'progress' => 100} }
def execute_returns(data)
result = mock_mc_result({:data => data })
rpcclient.expects(:execute).once.returns([result])
end
def should_report_once(data)
reporter.expects(:report).once.with(data)
end
def should_report_error(data)
error_data = {'status' => 'error', 'progress' => 100}.merge(data)
reporter.expects(:report).once.with(error_data)
end
shared_examples 'redhat checker' do
it 'should handle network connection errors' do
execute_returns({
:exit_code => 0,
:stdout => "Text before\nNetwork error, unable to connect to server.\nText after"})
err_msg = 'Unable to reach host cdn.redhat.com. ' + \
'Please check your Internet connection.'
should_report_error({'error' => err_msg})
expect { execute_handler }.to raise_error(Astute::RedhatCheckingError)
end
it 'should handle wrong username/password errors' do
execute_returns({
:exit_code => 0,
:stdout => "Text before\nInvalid username or password\nText after"})
err_msg = 'Invalid username or password. ' + \
'To create a login, please visit https://www.redhat.com/wapps/ugc/register.html'
should_report_error({'error' => err_msg})
expect { execute_handler }.to raise_error(Astute::RedhatCheckingError)
end
it 'should handle uniq errors' do
execute_returns({
:exit_code => 1,
:stdout => "Uniq error stdout",
:stderr => "Uniq error stderr"})
err_msg = "Unknown error Stdout: Uniq error stdout Stderr: Uniq error stderr"
should_report_error({'error' => err_msg})
expect { execute_handler }.to raise_error(Astute::RedhatCheckingError)
end
end
describe '#check_redhat_credentials' do
it_behaves_like 'redhat checker' do
def execute_handler
redhat_checker.check_redhat_credentials
end
end
it 'should be success with right credentials' do
execute_returns({:exit_code => 0, :stdout => '{"openstack_licenses_physical_hosts_count":1}'})
success_msg = "Account information for RELEASE_NAME has been successfully modified."
should_report_once(success_result.merge({'msg' => success_msg}))
redhat_checker.check_redhat_credentials
end
end
describe '#check_redhat_licenses' do
describe 'nodes parameter is nil' do
it_behaves_like 'redhat checker' do
def execute_handler
redhat_checker.check_redhat_credentials
end
end
it 'should be success if no errors' do
execute_returns({:exit_code => 0, :stdout => '{"openstack_licenses_physical_hosts_count":1}'})
should_report_once(success_result)
redhat_checker.check_redhat_licenses
end
end
describe 'nodes parameter is not nil' do
it_behaves_like 'redhat checker' do
def execute_handler
redhat_checker.check_redhat_licenses([1])
end
end
it 'should report ready if no errors' do
execute_returns({:exit_code => 0,
:stdout => '{"openstack_licenses_physical_hosts_count":1}'})
should_report_once(success_result)
nodes = [1]
redhat_checker.check_redhat_licenses(nodes)
end
it 'should report message if not enough licenses' do
execute_returns({:exit_code => 0,
:stdout => '{"openstack_licenses_physical_hosts_count":3}'})
err_msg = 'Your account has only 3 licenses available to deploy Red ' + \
'Hat OpenStack. Contact your Red Hat sales representative to ' + \
'get the proper subscriptions associated with your account. ' + \
'https://access.redhat.com/site/solutions/368643'
should_report_once({'progress' => 100, 'status' => 'ready', 'msg' => err_msg})
nodes = [1, 2, 3, 4]
redhat_checker.check_redhat_licenses(nodes)
end
end
end
end

View File

@ -60,14 +60,6 @@ describe "SimplePuppet DeploymentEngine" do
@deploy_engine.deploy(@env['nodes'], @env['attributes'])
end
it "singlenode deploy should not raise any exception" do
@env['attributes']['deployment_mode'] = "singlenode"
@env['nodes'] = [@env['nodes'][0]] # We have only one node in singlenode
Astute::Metadata.expects(:publish_facts).never
Astute::PuppetdDeployer.expects(:deploy).once # one call for one node
@deploy_engine.deploy(@env['nodes'], @env['attributes'])
end
it "ha_compact deploy should not raise any exception" do
@env['attributes']['deployment_mode'] = "ha_compact"
@env['nodes'].concat([{'uid'=>'c1', 'role'=>'controller'},
@ -79,7 +71,6 @@ describe "SimplePuppet DeploymentEngine" do
primary_ctrl_nodes = [controller_nodes.shift]
Astute::Metadata.expects(:publish_facts).never
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, primary_ctrl_nodes, 0, false).once
controller_nodes.each do |n|
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, [n], 2, true).once
end
@ -92,30 +83,32 @@ describe "SimplePuppet DeploymentEngine" do
it "ha_full deploy should not raise any exception" do
@env['attributes']['deployment_mode'] = "ha_full"
@env['nodes'].concat([{'uid'=>'c1', 'role'=>'controller'}, {'uid'=>'c2', 'role'=>'controller'},
{'uid'=>'q1', 'role'=>'quantum'}, {'uid'=>'q2', 'role'=>'quantum'},
{'uid'=>'st1', 'role'=>'storage'}, {'uid'=>'st2', 'role'=>'storage'},
{'uid'=>'sw1', 'role'=>'primary-swift-proxy'}, {'uid'=>'sw2', 'role'=>'swift-proxy'},
{'uid'=>'o1', 'role'=>'other'}])
{'uid'=>'st1', 'role'=>'storage'}, {'uid'=>'st2', 'role'=>'storage'},
{'uid'=>'sw1', 'role'=>'primary-swift-proxy'}, {'uid'=>'sw2', 'role'=>'swift-proxy'},
{'uid'=>'o1', 'role'=>'other'}])
controller_nodes = @env['nodes'].select{|n| n['role'] == 'controller'}
primary_ctrl_nodes = [controller_nodes.shift]
compute_nodes = @env['nodes'].select{|n| n['role'] == 'compute'}
quantum_nodes = @env['nodes'].select {|n| n['role'] == 'quantum'}
storage_nodes = @env['nodes'].select {|n| n['role'] == 'storage'}
proxy_nodes = @env['nodes'].select {|n| n['role'] == 'swift-proxy'}
primary_proxy_nodes = @env['nodes'].select {|n| n['role'] == 'primary-swift-proxy'}
primary_nodes = primary_ctrl_nodes + primary_proxy_nodes
other_nodes = @env['nodes'] - controller_nodes - primary_nodes - quantum_nodes
other_nodes = @env['nodes'] - controller_nodes - primary_proxy_nodes - \
primary_ctrl_nodes - proxy_nodes - storage_nodes
Astute::Metadata.expects(:publish_facts).never
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, primary_ctrl_nodes, 0, false).once
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, primary_proxy_nodes, 2, true).once
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, proxy_nodes, 2, true).once
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, storage_nodes, 2, true).once
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, primary_ctrl_nodes, 2, true).once
controller_nodes.each do |n|
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, [n], 2, true).once
end
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, primary_nodes, 2, true).once
# Astute::PuppetdDeployer.expects(:deploy).with(@ctx, primary_ctrl_nodes, 0, false).once
# Astute::PuppetdDeployer.expects(:deploy).with(@ctx, quantum_nodes, 2, true).once
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, other_nodes, 2, true).once
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, quantum_nodes, 2, true).once
Astute::PuppetdDeployer.expects(:deploy).with(@ctx, other_nodes, instance_of(Fixnum), true).once
@deploy_engine.deploy(@env['nodes'], @env['attributes'])
end
end