Granular deployment (basic version)

Support new way to deploy tasks. Instead of direct
puppet run using one big universal manifests use
group of tasks with pre and post checks.

Support old deployment process for backward
compatibility.

New feature:
- add tasks processing;
- add tests.

Refactoring:
* move pre deployment actions to separate class;
* tests refactoring;
* unify tasks and puppet methods.

Related to blueprint granular-deployment-based-on-tasks

Change-Id: I1db257875c164ada60600bf976cb828a41585248
This commit is contained in:
Vladimir Sharshov 2014-09-02 16:35:07 +04:00
parent c3e7c7a185
commit a9013b4633
27 changed files with 1584 additions and 713 deletions

View File

@ -33,8 +33,7 @@ end
reporter = ConsoleReporter.new
deploy_engine = Astute::DeploymentEngine::NailyFact
orchestrator = Astute::Orchestrator.new(deploy_engine, log_parsing=false)
orchestrator = Astute::Orchestrator.new(log_parsing=false)
# Add systems to cobbler, reboot and start installation process.
orchestrator.provision(reporter, environment['engine'], environment['nodes'])

View File

@ -15,6 +15,7 @@
require 'astute/ruby_removed_functions'
require 'json'
require 'yaml'
require 'logger'
require 'shellwords'
require 'active_support/all'
@ -30,12 +31,15 @@ require 'astute/deployment_engine'
require 'astute/network'
require 'astute/puppetd'
require 'astute/deployment_engine/nailyfact'
require 'astute/deployment_engine/tasklib'
require 'astute/cobbler'
require 'astute/cobbler_manager'
require 'astute/image_provision'
require 'astute/dump'
require 'astute/deploy_actions'
['/astute/pre_deploy_actions/*.rb',
['/astute/pre_deployment_actions/*.rb',
'/astute/pre_deploy_actions/*.rb',
'/astute/pre_node_actions/*.rb',
'/astute/post_deploy_actions/*.rb',
'/astute/post_deployment_actions/*.rb',
@ -43,11 +47,13 @@ require 'astute/deploy_actions'
].each do |path|
Dir[File.dirname(__FILE__) + path].each{ |f| require f }
end
require 'astute/ssh'
require 'astute/ssh_actions/ssh_erase_nodes'
require 'astute/ssh_actions/ssh_hard_reboot'
require 'astute/ssh_actions/ssh_reboot_not_provisioning'
# Server
require 'astute/server/worker'
require 'astute/server/server'

View File

@ -66,6 +66,8 @@ module Astute
conf[:DUMP_TIMEOUT] = 3600 # maximum time it waits for the dump (meaningles to be larger
# than the specified in timeout of execute_shell_command mcagent
conf[:PUPPET_SSH_KEYS_DIR] = '/var/lib/astute' # folder where ssh keys will be saved. Warning!
# Do not change this at least your clear know what you do!
conf[:PUPPET_SSH_KEYS] = ['neutron', 'nova', 'ceph', 'mysql'] # name of ssh keys what will be generated
#and uploaded to all nodes before deploy
conf[:MAX_NODES_PER_CALL] = 50 # how many nodes to deploy in one puppet call

View File

@ -29,7 +29,9 @@ module Astute
class PreDeployActions < DeployActions
def initialize(deployment_info, context)
super
@actions = []
@actions = [
UploadFacts.new
]
end
end
@ -63,7 +65,25 @@ module Astute
end
end
class PreDeploymentActions < DeployActions
def initialize(deployment_info, context)
super
@actions = [
GenerateSshKeys.new,
UploadSshKeys.new,
UpdateRepoSources.new,
SyncPuppetStuff.new,
SyncTasks.new,
EnablePuppetDeploy.new,
SyncTime.new
]
end
end
class PostDeploymentActions < DeployActions
def initialize(deployment_info, context)
super
@actions = [
@ -72,6 +92,7 @@ module Astute
RestartRadosgw.new,
UpdateClusterHostsInfo.new
]
end
end

View File

@ -12,13 +12,6 @@
# License for the specific language governing permissions and limitations
# under the License.
require 'fileutils'
require 'popen4'
require 'uri'
KEY_DIR = "/var/lib/astute"
SYNC_RETRIES = 10
module Astute
class DeploymentEngine
@ -36,26 +29,10 @@ module Astute
Astute.logger.info "Deployment mode #{@ctx.deploy_log_parser.deploy_type}"
begin
# Generate ssh keys to future uploading to all cluster nodes
generate_ssh_keys(deployment_info.first['deployment_id'])
# Prevent to prepare too many nodes at once
deployment_info.uniq { |n| n['uid'] }.each_slice(Astute.config[:MAX_NODES_PER_CALL]) do |part|
# Upload ssh keys from master node to all cluster nodes.
# Will be used by puppet after to connect nodes between themselves.
upload_ssh_keys(part.map{ |n| n['uid'] }, part.first['deployment_id'])
# Update packages source list
update_repo_sources(part) if part.first['repo_metadata']
# Sync puppet manifests and modules to every node (emulate puppet master)
sync_puppet_manifests(part)
# Unlock puppet (can be lock if puppet was killed by user)
enable_puppet_deploy(part.map{ |n| n['uid'] })
# Sync time
sync_time(part.map{ |n| n['uid'] })
# Pre deploy hooks
PreDeploymentActions.new(part, @ctx).process
end
rescue => e
Astute.logger.error("Unexpected error #{e.message} traceback #{e.format_backtrace}")
@ -140,206 +117,6 @@ module Astute
nodes_array.find { |n| node['uid'] == n['uid'] }
end
# Sync puppet manifests and modules to every node
def sync_puppet_manifests(deployment_info)
master_ip = deployment_info.first['master_ip']
modules_source = deployment_info.first['puppet_modules_source'] || "rsync://#{master_ip}:/puppet/modules/"
manifests_source = deployment_info.first['puppet_manifests_source'] || "rsync://#{master_ip}:/puppet/manifests/"
# Paths to Puppet modules and manifests at the master node set by Nailgun
# Check fuel source code /deployment/puppet/nailgun/manifests/puppetsync.pp
schemas = [modules_source, manifests_source].map do |url|
begin
URI.parse(url).scheme
rescue URI::InvalidURIError => e
raise DeploymentEngineError, e.message
end
end
if schemas.select{ |x| x != schemas.first }.present?
raise DeploymentEngineError, "Scheme for puppet_modules_source '#{schemas.first}' and" \
" puppet_manifests_source '#{schemas.last}' not equivalent!"
end
sync_mclient = MClient.new(@ctx, "puppetsync", deployment_info.map{ |n| n['uid'] }.uniq)
case schemas.first
when 'rsync'
begin
sync_mclient.rsync(:modules_source => modules_source,
:manifests_source => manifests_source
)
rescue MClientError => e
sync_retries ||= 0
sync_retries += 1
if sync_retries < SYNC_RETRIES
Astute.logger.warn("Rsync problem. Try to repeat: #{sync_retries} attempt")
retry
end
raise e
end
else
raise DeploymentEngineError, "Unknown scheme '#{schemas.first}' in #{modules_source}"
end
end
def generate_ssh_keys(deployment_id, overwrite=false)
raise "Deployment_id is missing" unless deployment_id
Astute.config.PUPPET_SSH_KEYS.each do |key_name|
dir_path = File.join(KEY_DIR, deployment_id.to_s, key_name)
key_path = File.join(dir_path, key_name)
FileUtils.mkdir_p dir_path
raise DeploymentEngineError, "Could not create directory #{dir_path}" unless File.directory?(dir_path)
next if File.exist?(key_path) && !overwrite
# Generate 2 keys(<name> and <name>.pub) and save it to <KEY_DIR>/<name>/
File.delete key_path if File.exist? key_path
cmd = "ssh-keygen -b 2048 -t rsa -N '' -f #{key_path} 2>&1"
status, stdout, _ = run_system_command cmd
error_msg = "Could not generate ssh key! Command: #{cmd}, output: #{stdout}, exit code: #{status}"
raise DeploymentEngineError, error_msg if status != 0
end
end
def upload_ssh_keys(node_uids, deployment_id, overwrite=false)
Astute.config.PUPPET_SSH_KEYS.each do |key_name|
upload_mclient = MClient.new(@ctx, "uploadfile", node_uids)
[key_name, key_name + ".pub"].each do |ssh_key|
source_path = File.join(KEY_DIR, deployment_id.to_s, key_name, ssh_key)
destination_path = File.join(KEY_DIR, key_name, ssh_key)
content = File.read(source_path)
upload_mclient.upload(:path => destination_path,
:content => content,
:user_owner => 'root',
:group_owner => 'root',
:permissions => '0600',
:dir_permissions => '0700',
:overwrite => true,
:parents => true
)
end
end
end
def update_repo_sources(deployment_info)
content = generate_repo_source(deployment_info)
upload_repo_source(deployment_info, content)
regenerate_metadata(deployment_info)
end
def generate_repo_source(deployment_info)
ubuntu_source = -> (name, url) { "deb #{url}" }
centos_source = -> (name, url) do
["[#{name.downcase}]", "name=#{name}", "baseurl=#{url}", "gpgcheck=0"].join("\n")
end
formatter = case target_os(deployment_info)
when 'centos' then centos_source
when 'ubuntu' then ubuntu_source
end
content = []
deployment_info.first['repo_metadata'].each do |name, url|
content << formatter.call(name,url)
end
content.join("\n")
end
def upload_repo_source(deployment_info, content)
upload_mclient = MClient.new(@ctx, "uploadfile", deployment_info.map{ |n| n['uid'] }.uniq)
destination_path = case target_os(deployment_info)
when 'centos' then '/etc/yum.repos.d/nailgun.repo'
when 'ubuntu' then '/etc/apt/sources.list'
end
upload_mclient.upload(:path => destination_path,
:content => content,
:user_owner => 'root',
:group_owner => 'root',
:permissions => '0644',
:dir_permissions => '0755',
:overwrite => true,
:parents => true
)
end
def regenerate_metadata(deployment_info)
cmd = case target_os(deployment_info)
when 'centos' then "yum clean all"
when 'ubuntu' then "apt-get clean; apt-get update"
end
succeeded = false
nodes_uids = deployment_info.map{ |n| n['uid'] }.uniq
Astute.config.MC_RETRIES.times.each do
succeeded = run_shell_command_remotely(nodes_uids, cmd)
return if succeeded
sleep Astute.config.MC_RETRY_INTERVAL
end
if !succeeded
raise DeploymentEngineError, "Run command: '#{cmd}' in nodes: #{nodes_uids} fail." \
" Check debug output for more information"
end
end
def target_os(deployment_info)
os = deployment_info.first['cobbler']['profile']
case os
when 'centos-x86_64' then 'centos'
when 'ubuntu_1204_x86_64' then 'ubuntu'
else
raise DeploymentEngineError, "Unknown system #{os}"
end
end
def sync_time(nodes_uids)
cmd = "ntpdate -u $(egrep '^server' /etc/ntp.conf | sed '/^#/d' | awk '{print $2}')"
succeeded = false
Astute.config.MC_RETRIES.times.each do
succeeded = run_shell_command_remotely(nodes_uids, cmd)
return if succeeded
sleep Astute.config.MC_RETRY_INTERVAL
end
if !succeeded
Astute.logger.warn "Run command: '#{cmd}' in nodes: #{nodes_uids} fail. " \
"Check debug output for more information. You can try "\
"to fix it problem manually."
end
end
def run_system_command(cmd)
pid, _, stdout, stderr = Open4::popen4 cmd
_, status = Process::waitpid2 pid
return status.exitstatus, stdout, stderr
end
def run_shell_command_remotely(node_uids, cmd)
shell = MClient.new(@ctx,
'execute_shell_command',
node_uids,
check_result=true,
timeout=60,
retries=1)
#TODO: return result for all nodes not only for first
response = shell.execute(:cmd => cmd).first
Astute.logger.debug("#{@ctx.task_id}: cmd: #{cmd}
stdout: #{response[:data][:stdout]}
stderr: #{response[:data][:stderr]}
exit code: #{response[:data][:exit_code]}")
response.fetch(:data, {})[:exit_code] == 0
end
def enable_puppet_deploy(node_uids)
puppetd = MClient.new(@ctx, "puppetd", node_uids)
puppetd.enable
end
def nodes_status(nodes, status, data_to_merge)
{
'nodes' => nodes.map do |n|

View File

@ -12,8 +12,6 @@
# License for the specific language governing permissions and limitations
# under the License.
require 'yaml'
class Astute::DeploymentEngine::NailyFact < Astute::DeploymentEngine
def deploy_piece(nodes, retries=2)
@ -27,28 +25,11 @@ class Astute::DeploymentEngine::NailyFact < Astute::DeploymentEngine
Astute.logger.warn "Some error occurred when prepare LogParser: #{e.message}, trace: #{e.format_backtrace}"
end
nodes.each { |node| upload_facts(node) }
Astute.logger.info "#{@ctx.task_id}: Required attrs/metadata passed via facts extension. Starting deployment."
Astute.logger.info "#{@ctx.task_id}: Starting deployment"
Astute::PuppetdDeployer.deploy(@ctx, nodes, retries)
nodes_roles = nodes.map { |n| {n['uid'] => n['role']} }
Astute.logger.info "#{@ctx.task_id}: Finished deployment of nodes => roles: #{nodes_roles.inspect}"
end
private
def upload_facts(node)
Astute.logger.info "#{@ctx.task_id}: storing metadata for node uid=#{node['uid']}"
Astute.logger.debug "#{@ctx.task_id}: stores metadata: #{node.to_yaml}"
# This is synchronious RPC call, so we are sure that data were sent and processed remotely
upload_mclient = Astute::MClient.new(@ctx, "uploadfile", [node['uid']])
upload_mclient.upload(:path => '/etc/astute.yaml',
:content => node.to_yaml,
:overwrite => true,
:parents => true,
:permissions => '0600'
)
end
end

View File

@ -0,0 +1,226 @@
# Copyright 2014 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 Astute::DeploymentEngine::Tasklib < Astute::DeploymentEngine
TASKLIB_STATUS = {
0 => :ended_successfully,
1 => :running,
2 => :valid_but_failed,
3 => :unexpected_error,
4 => :not_found_such_task
}
def deploy_piece(nodes, retries=2)
return false unless validate_nodes(nodes)
tasklib_deploy(nodes, retries)
nodes_roles = nodes.map { |n| {n['uid'] => n['role']} }
Astute.logger.info "#{@ctx.task_id}: Finished deployment of nodes => roles: #{nodes_roles.inspect}"
end
private
def pre_tasklib_deploy
@time_before = Time.now.to_i
@ctx.reporter.report(nodes_status(@nodes, 'deploying', {'progress' => 0}))
begin
@ctx.deploy_log_parser.prepare(@nodes)
rescue => e
Astute.logger.warn "Some error occurred when prepare LogParser: #{e.message}, trace: #{e.format_backtrace}"
end
end
def tasklib_deploy(nodes, retries=2)
@nodes = nodes
@nodes_roles = nodes.inject({}) { |h, n| h.merge({n['uid'] => n['role']}) }
@task_manager = TaskManager.new(nodes)
@debug = nodes.first['debug']
Timeout::timeout(Astute.config.PUPPET_TIMEOUT) do
pre_tasklib_deploy
deploy_nodes
post_tasklib_deploy
end
end
def post_tasklib_deploy
time_spent = (Time.now.to_i - @time_before) / 60.to_f
Astute.logger.info "#{@ctx.task_id}: Spent #{time_spent.round(1)} minutes on tasklib run " \
"for following nodes(uids): #{@nodes.map {|n| n['uid']}.join(',')}"
end
def tasklib_mclient(node_ids)
shell = Astute::MClient.new(
@ctx,
'execute_shell_command',
Array(node_ids),
check_result=true,
timeout=timeout,
retries=1
)
shell.on_respond_timeout do |uids|
nodes = uids.map do |uid|
{ 'uid' => uid,
'status' => 'error',
'error_type' => 'deploy',
'role' => @nodes_roles[uid],
'task' => @task_manager.current_task(uid)
}
end
@ctx.report_and_update_status('nodes' => nodes)
@task_manager.delete(uid)
end
shell
end
def run_task(node_id, task)
Astute.logger.info "#{@ctx.task_id}: run task '#{task}' on node #{node_id}"
debug_option = @debug ? "--debug" : ""
cmd = "taskcmd #{debug_option} daemon #{task}"
tasklib_mclient(node_id).execute(:cmd => cmd).first
end
def check_status(node_id, task)
cmd = "taskcmd status #{task}"
response = tasklib_mclient(node_id).execute(:cmd => cmd).first
status = response[:data][:exit_code].to_i
if TASKLIB_STATUS.keys.include? status
TASKLIB_STATUS[status]
else
raise "Internal error. Unknown status '#{status}'"
end
end
def deploy_nodes
@task_manager.node_uids.each { |n| task = @task_manager.next_task(n) and run_task(n, task) }
while @task_manager.task_in_queue?
nodes_to_report = []
@task_manager.node_uids.each do |node_id|
if task = @task_manager.current_task(node_id)
case status = check_status(node_id, task)
when :ended_successfully
Astute.logger.info "Task '#{task}' on node uid=#{node_id} ended successfully"
new_task = @task_manager.next_task(node_id)
if new_task
run_task(node_id, new_task)
else
Astute.logger.info "No more tasks provided for node #{node_id}. All node " \
"tasks completed successfully"
nodes_to_report << {
"uid" => node_id,
'status' => 'ready',
'role' => @nodes_roles[node_id],
"progress" => 100,
'task' => task
}
end
when :running
progress_report = process_running_node(node_id, task)
nodes_to_report << progress_report if progress_report
when :valid_but_failed
Astute.logger.error "Task '#{task}' on node #{node_id} valid, but failed"
nodes_to_report << process_fail_node(node_id, task)
when :unexpected_error
Astute.logger.error "Task '#{task}' on node #{node_id} finished with an unexpected error"
nodes_to_report << process_fail_node(node_id, task)
when :not_found_such_task
Astute.logger.error "Task '#{task}' on node #{node_id} not found"
nodes_to_report << process_fail_node(node_id, task)
else
raise "Internal error. Known status '#{status}', but handler not provided"
end
else
Astute.logger.debug "No more tasks provided for node #{node_id}"
end
end
@ctx.report_and_update_status('nodes' => nodes_to_report) if nodes_to_report.present?
break unless @task_manager.task_in_queue?
sleep Astute.config.PUPPET_DEPLOY_INTERVAL
end
end
def process_fail_node(node_id, task)
Astute.logger.error "No more tasks will be executed on the node #{node_id}"
@task_manager.delete_node(node_id)
{
'uid' => node_id,
'status' => 'error',
'error_type' => 'deploy',
'role' => @nodes_roles[node_id],
'task' => task
}
end
def process_running_node(node_id, task)
begin
# Pass nodes because logs calculation needs IP address of node, not just uid
nodes_progress = @ctx.deploy_log_parser.progress_calculate(Array(node_id), @nodes)
if nodes_progress.present?
nodes_progress.map! { |x| x.merge!(
'status' => 'deploying',
'role' => @nodes_roles[x['uid']],
'task' => task
) }
end
rescue => e
Astute.logger.warn "Some error occurred when parse logs for nodes progress: #{e.message}, "\
"trace: #{e.format_backtrace}"
nil
end
nodes_progress.first
end
end # class
class TaskManager
def initialize(nodes)
@tasks = nodes.inject({}) { |h, n| h.merge({n['uid'] => n['tasks'].map{ |t| t['name'] }.each}) }
@current_task = {}
Astute.logger.info "The following tasks will be performed on nodes: " \
"#{@tasks.map {|k, v| {k => v.to_a}}.to_yaml}"
end
def current_task(node_id)
@current_task[node_id]
end
def next_task(node_id)
@current_task[node_id] = @tasks[node_id].next
rescue StopIteration
@current_task[node_id] = nil
delete_node(node_id)
end
def delete_node(node_id)
@tasks[node_id] = nil
end
def task_in_queue?
@tasks.select{ |_k,v| v }.present?
end
def node_uids
@tasks.select{ |_k,v| v }.keys
end
end

View File

@ -15,8 +15,7 @@
module Astute
class Orchestrator
def initialize(deploy_engine=nil, log_parsing=false)
@deploy_engine = deploy_engine || Astute::DeploymentEngine::NailyFact
def initialize(log_parsing=false)
@log_parsing = log_parsing
end
@ -33,18 +32,21 @@ module Astute
end
def deploy(up_reporter, task_id, deployment_info)
proxy_reporter = ProxyReporter::DeploymentProxyReporter.new(up_reporter, deployment_info)
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."
deploy_cluster(
up_reporter,
task_id,
deployment_info,
Astute::DeploymentEngine::NailyFact
)
end
deploy_engine_instance.deploy(deployment_info)
# Post deployment hooks
PostDeploymentActions.new(deployment_info, context).process
context.status
def task_deployment(up_reporter, task_id, deployment_info)
deploy_cluster(
up_reporter,
task_id,
deployment_info,
Astute::DeploymentEngine::Tasklib
)
end
def provision(reporter, task_id, engine_attrs, nodes)
@ -236,6 +238,21 @@ module Astute
private
def deploy_cluster(up_reporter, task_id, deployment_info, deploy_engine)
proxy_reporter = ProxyReporter::DeploymentProxyReporter.new(up_reporter, deployment_info)
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."
deploy_engine_instance.deploy(deployment_info)
# Post deployment hooks
PostDeploymentActions.new(deployment_info, context).process
context.status
end
def report_result(result, reporter)
default_result = {'status' => 'ready', 'progress' => 100}

View File

@ -0,0 +1,41 @@
# Copyright 2014 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 UploadFacts < PreDeployAction
# Generate ssh keys to future uploading to all cluster nodes
def process(deployment_info, context)
deployment_info.each{ |node| upload_facts(context, node) }
Astute.logger.info "#{context.task_id}: Required attrs/metadata passed via facts extension"
end #process
private
def upload_facts(context, node)
Astute.logger.info "#{context.task_id}: storing metadata for node uid=#{node['uid']}"
Astute.logger.debug "#{context.task_id}: stores metadata: #{node.to_yaml}"
# This is synchronious RPC call, so we are sure that data were sent and processed remotely
upload_mclient = Astute::MClient.new(context, "uploadfile", [node['uid']])
upload_mclient.upload(:path => '/etc/astute.yaml',
:content => node.to_yaml,
:overwrite => true,
:parents => true,
:permissions => '0600'
)
end
end #class
end

View File

@ -0,0 +1,25 @@
# Copyright 2014 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 EnablePuppetDeploy < PreDeploymentAction
# Unlock puppet (can be lock if puppet was killed by user)
def process(deployment_info, context)
nodes_uids = deployment_info.map{ |n| n['uid'] }
puppetd = MClient.new(context, "puppetd", nodes_uids)
puppetd.enable
end #process
end #class
end

View File

@ -0,0 +1,56 @@
# Copyright 2014 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 'popen4'
require 'fileutils'
module Astute
class GenerateSshKeys < PreDeploymentAction
# Generate ssh keys to future uploading to all cluster nodes
def process(deployment_info, context)
overwrite = false
deployment_id = deployment_info.first['deployment_id']
raise "Deployment_id is missing" unless deployment_id
Astute.config.PUPPET_SSH_KEYS.each do |key_name|
dir_path = File.join(Astute.config.PUPPET_SSH_KEYS_DIR, deployment_id.to_s, key_name)
key_path = File.join(dir_path, key_name)
FileUtils.mkdir_p dir_path
raise DeploymentEngineError, "Could not create directory #{dir_path}" unless File.directory?(dir_path)
next if File.exist?(key_path) && !overwrite
# Generate 2 keys(<name> and <name>.pub) and save it to <KEY_DIR>/<name>/
File.delete key_path if File.exist? key_path
cmd = "ssh-keygen -b 2048 -t rsa -N '' -f #{key_path} 2>&1"
status, stdout, _stderr = run_system_command cmd
error_msg = "Could not generate ssh key! Command: #{cmd}, output: #{stdout}, exit code: #{status}"
raise DeploymentEngineError, error_msg if status != 0
end
end #process
private
def run_system_command(cmd)
pid, _, stdout, stderr = Open4::popen4 cmd
_, status = Process::waitpid2 pid
return status.exitstatus, stdout, stderr
end
end #class
end

View File

@ -0,0 +1,62 @@
# Copyright 2014 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 'uri'
SYNC_RETRIES = 10
module Astute
class SyncPuppetStuff < PreDeploymentAction
# Sync puppet manifests and modules to every node
def process(deployment_info, context)
master_ip = deployment_info.first['master_ip']
modules_source = deployment_info.first['puppet_modules_source'] || "rsync://#{master_ip}:/puppet/modules/"
manifests_source = deployment_info.first['puppet_manifests_source'] || "rsync://#{master_ip}:/puppet/manifests/"
# Paths to Puppet modules and manifests at the master node set by Nailgun
# Check fuel source code /deployment/puppet/nailgun/manifests/puppetsync.pp
schemas = [modules_source, manifests_source].map do |url|
begin
URI.parse(url).scheme
rescue URI::InvalidURIError => e
raise DeploymentEngineError, e.message
end
end
if schemas.select{ |x| x != schemas.first }.present?
raise DeploymentEngineError, "Scheme for puppet_modules_source '#{schemas.first}' and" \
" puppet_manifests_source '#{schemas.last}' not equivalent!"
end
sync_mclient = MClient.new(context, "puppetsync", deployment_info.map{ |n| n['uid'] }.uniq)
case schemas.first
when 'rsync'
begin
sync_mclient.rsync(:modules_source => modules_source,
:manifests_source => manifests_source
)
rescue MClientError => e
sync_retries ||= 0
sync_retries += 1
if sync_retries < SYNC_RETRIES
Astute.logger.warn("Rsync problem. Try to repeat: #{sync_retries} attempt")
retry
end
raise e
end
else
raise DeploymentEngineError, "Unknown scheme '#{schemas.first}' in #{modules_source}"
end
end #process
end #class
end

View File

@ -0,0 +1,44 @@
# Copyright 2014 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.
SYNC_RETRIES = 10
module Astute
class SyncTasks < PreDeploymentAction
# Sync puppet manifests and modules to every node
def process(deployment_info, context)
return unless deployment_info.first['tasks_source']
# URI to Tasklib tasks at the master node set by Nailgun
master_ip = deployment_info.first['master_ip']
tasks_source = deployment_info.first['tasks_source'] || "rsync://#{master_ip}:/puppet/tasks/"
source = tasks_source.chomp('/').concat('/')
path = '/etc/puppet/tasks/'
rsync_options = '-c -r --delete'
rsync_cmd = "mkdir -p #{path} && rsync #{rsync_options} #{source} #{path}"
sync_retries = 0
while sync_retries < SYNC_RETRIES
sync_retries += 1
response = run_shell_command(context, deployment_info.map{ |n| n['uid'] }.uniq, rsync_cmd, 300)
break if response[:data][:exit_code] == 0
Astute.logger.warn("Rsync problem. Try to repeat: #{sync_retries} attempt")
end
end #process
end #class
end

View File

@ -0,0 +1,45 @@
# Copyright 2014 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 SyncTime < PreDeploymentAction
# Sync time
def process(deployment_info, context)
nodes_uids = deployment_info.map{ |n| n['uid'] }
cmd = "ntpdate -u $(egrep '^server' /etc/ntp.conf | sed '/^#/d' | awk '{print $2}')"
succeeded = false
Astute.config.MC_RETRIES.times.each do
succeeded = run_shell_command_remotely(context, nodes_uids, cmd)
return if succeeded
sleep Astute.config.MC_RETRY_INTERVAL
end
if !succeeded
Astute.logger.warn "Run command: '#{cmd}' in nodes: #{nodes_uids} fail. " \
"Check debug output for more information. You can try "\
"to fix it problem manually."
end
end #process
private
def run_shell_command_remotely(context, nodes_uids, cmd)
response = run_shell_command(context, nodes_uids, cmd)
response.fetch(:data, {})[:exit_code] == 0
end
end #class
end

View File

@ -0,0 +1,99 @@
# Copyright 2014 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 UpdateRepoSources < PreDeploymentAction
# Update packages source list
def process(deployment_info, context)
return unless deployment_info.first['repo_metadata']
content = generate_repo_source(deployment_info)
upload_repo_source(context, deployment_info, content)
regenerate_metadata(context, deployment_info)
end #process
private
def generate_repo_source(deployment_info)
ubuntu_source = -> (name, url) { "deb #{url}" }
centos_source = -> (name, url) do
["[#{name.downcase}]", "name=#{name}", "baseurl=#{url}", "gpgcheck=0"].join("\n")
end
formatter = case target_os(deployment_info)
when 'centos' then centos_source
when 'ubuntu' then ubuntu_source
end
content = []
deployment_info.first['repo_metadata'].each do |name, url|
content << formatter.call(name,url)
end
content.join("\n")
end
def upload_repo_source(context, deployment_info, content)
upload_mclient = MClient.new(context, "uploadfile", deployment_info.map{ |n| n['uid'] }.uniq)
destination_path = case target_os(deployment_info)
when 'centos' then '/etc/yum.repos.d/nailgun.repo'
when 'ubuntu' then '/etc/apt/sources.list'
end
upload_mclient.upload(:path => destination_path,
:content => content,
:user_owner => 'root',
:group_owner => 'root',
:permissions => '0644',
:dir_permissions => '0755',
:overwrite => true,
:parents => true
)
end
def regenerate_metadata(context, deployment_info)
cmd = case target_os(deployment_info)
when 'centos' then "yum clean all"
when 'ubuntu' then "apt-get clean; apt-get update"
end
succeeded = false
nodes_uids = deployment_info.map{ |n| n['uid'] }.uniq
Astute.config.MC_RETRIES.times.each do
succeeded = run_shell_command_remotely(context, nodes_uids, cmd)
return if succeeded
sleep Astute.config.MC_RETRY_INTERVAL
end
if !succeeded
raise DeploymentEngineError, "Run command: '#{cmd}' in nodes: #{nodes_uids} fail." \
" Check debug output for more information"
end
end
def target_os(deployment_info)
os = deployment_info.first['cobbler']['profile']
case os
when 'centos-x86_64' then 'centos'
when 'ubuntu_1204_x86_64' then 'ubuntu'
else
raise DeploymentEngineError, "Unknown system #{os}"
end
end
def run_shell_command_remotely(context, nodes_uids, cmd)
response = run_shell_command(context, nodes_uids, cmd)
response.fetch(:data, {})[:exit_code] == 0
end
end #class
end

View File

@ -0,0 +1,50 @@
# Copyright 2014 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 UploadSshKeys < PreDeploymentAction
# Upload ssh keys from master node to all cluster nodes
def process(deployment_info, context)
deployment_id = deployment_info.first['deployment_id'].to_s
node_uids = deployment_info.map{ |n| n['uid'] }
Astute.config.PUPPET_SSH_KEYS.each do |key_name|
upload_mclient = MClient.new(context, "uploadfile", node_uids)
[key_name, key_name + ".pub"].each do |ssh_key|
source_path = File.join(
Astute.config.PUPPET_SSH_KEYS_DIR,
deployment_id,
key_name,
ssh_key)
destination_path = File.join(
Astute.config.PUPPET_SSH_KEYS_DIR,
key_name,
ssh_key)
content = File.read(source_path)
upload_mclient.upload(:path => destination_path,
:content => content,
:user_owner => 'root',
:group_owner => 'root',
:permissions => '0600',
:dir_permissions => '0700',
:overwrite => true,
:parents => true
)
end
end
end #process
end #class
end

View File

@ -19,7 +19,7 @@ module Astute
class Dispatcher
def initialize(producer)
@orchestrator = Astute::Orchestrator.new(nil, log_parsing=true)
@orchestrator = Astute::Orchestrator.new(log_parsing=true)
@producer = producer
@provisionLogParser = Astute::LogParser::ParseProvisionLogs.new
end
@ -64,6 +64,20 @@ module Astute
end
end
def task_deployment(data)
Astute.logger.info("'task_deployment' method called with data: #{data.inspect}")
reporter = Astute::Server::Reporter.new(@producer, data['respond_to'], data['args']['task_uuid'])
begin
@orchestrator.task_deployment(reporter, data['args']['task_uuid'], data['args']['deployment_info'])
reporter.report('status' => 'ready', 'progress' => 100)
rescue Timeout::Error
msg = "Timeout of deployment is exceeded."
Astute.logger.error msg
reporter.report('status' => 'error', 'error' => msg)
end
end
def verify_networks(data)
data.fetch('subtasks', []).each do |subtask|
if self.respond_to?(subtask['method'])

View File

@ -35,6 +35,48 @@ describe Astute::PostDeploymentActions do
end
end
describe Astute::PreDeploymentActions do
include SpecHelpers
let(:deploy_data) {[]}
let(:ctx) { mock }
let(:pre_deployment_actions) { Astute::PreDeploymentActions.new(deploy_data, ctx) }
it 'should run post hooks' do
Astute::EnablePuppetDeploy.any_instance.expects(:process)
.with(deploy_data, ctx)
Astute::GenerateSshKeys.any_instance.expects(:process)
.with(deploy_data, ctx)
Astute::SyncPuppetStuff.any_instance.expects(:process)
.with(deploy_data, ctx)
Astute::SyncTime.any_instance.expects(:process)
.with(deploy_data, ctx)
Astute::UpdateRepoSources.any_instance.expects(:process)
.with(deploy_data, ctx)
Astute::UploadSshKeys.any_instance.expects(:process)
.with(deploy_data, ctx)
Astute::SyncTasks.any_instance.expects(:process)
.with(deploy_data, ctx)
pre_deployment_actions.process
end
end
describe Astute::PreDeployActions do
include SpecHelpers
let(:deploy_data) {[]}
let(:ctx) { mock }
let(:pre_deploy_actions) { Astute::PreDeployActions.new(deploy_data, ctx) }
it 'should run pre hooks' do
Astute::UploadFacts.any_instance.expects(:process)
.with(deploy_data, ctx)
pre_deploy_actions.process
end
end
describe Astute::PreNodeActions do
include SpecHelpers

View File

@ -44,50 +44,40 @@ describe Astute::DeploymentEngine do
Astute::PreDeployActions.any_instance.stubs(:process).returns(nil)
Astute::PostDeployActions.any_instance.stubs(:process).returns(nil)
Astute::PreNodeActions.any_instance.stubs(:process).returns(nil)
deployer.stubs(:generate_ssh_keys)
deployer.stubs(:upload_ssh_keys)
deployer.stubs(:sync_puppet_manifests)
deployer.stubs(:enable_puppet_deploy)
deployer.stubs(:update_repo_sources)
deployer.stubs(:deploy_piece)
deployer.stubs(:sync_time)
Astute::PreDeploymentActions.any_instance.stubs(:process).returns(nil)
end
it 'should generate and upload ssh keys' do
nodes = [{'uid' => 1, 'deployment_id' => 1}, {'uid' => 2}, {'uid' => 1}]
context 'hooks' do
deployer.expects(:generate_ssh_keys).with(nodes.first['deployment_id'])
deployer.expects(:upload_ssh_keys).with([1,2], nodes.first['deployment_id']).returns()
deployer.expects(:sync_puppet_manifests).with([{'uid' => 1, 'deployment_id' => 1}, {'uid' => 2}])
let(:nodes) {
[{'uid' => 1, 'priority' => 10}, {'uid' => 2, 'priority' => 0}, {'uid' => 1, 'priority' => 15}]
}
deployer.deploy(nodes)
end
before(:each) { deployer.stubs(:deploy_piece) }
it 'should setup packages repositories' do
nodes = [
{'uid' => 1,
'deployment_id' => 1,
'repo_metadata' => {
'Nailgun' => 'http://10.20.0.2:8080/centos/fuelweb/x86_64/'
}
},
{'uid' => 2},
{'uid' => 1}
]
uniq_nodes = nodes[0..-2]
it 'should run pre deployment hooks run once for all cluster' do
Astute::PreDeploymentActions.any_instance.expects(:process).once
deployer.expects(:update_repo_sources).with(uniq_nodes)
deployer.deploy(nodes)
end
deployer.deploy(nodes)
end
it 'should run pre node hooks once for node' do
Astute::PreNodeActions.any_instance.expects(:process).twice
it 'should enable puppet for all nodes' do
nodes = [{'uid' => 1, 'deployment_id' => 1}, {'uid' => 2}, {'uid' => 1}]
deployer.deploy(nodes)
end
it 'should run pre deploy hooks once for role' do
Astute::PreDeployActions.any_instance.expects(:process).times(3)
deployer.expects(:enable_puppet_deploy).with([1,2]).returns()
deployer.deploy(nodes)
end
deployer.deploy(nodes)
it 'should run post deploy hooks once for role' do
Astute::PostDeployActions.any_instance.expects(:process).times(3)
deployer.deploy(nodes)
end
end
it 'deploy nodes by order' do
@ -274,9 +264,6 @@ describe Astute::DeploymentEngine do
it 'number of nodes running in parallel should be limited' do
Astute.config.MAX_NODES_PER_CALL = 1
deployer.stubs(:generate_ssh_keys)
deployer.stubs(:upload_ssh_keys)
deployer.stubs(:sync_puppet_manifests)
nodes = [
{'uid' => 1, 'priority' => 10, 'role' => 'compute'},
@ -299,405 +286,4 @@ describe Astute::DeploymentEngine do
end
end
describe '#sync_puppet_manifests' do
before(:each) do
deployer.stubs(:deploy_piece)
deployer.stubs(:generate_ssh_keys)
deployer.stubs(:upload_ssh_keys)
deployer.stubs(:enable_puppet_deploy)
deployer.stubs(:sync_time)
end
let(:nodes) { [
{'uid' => 1,
'deployment_id' => 1,
'master_ip' => '10.20.0.2',
'puppet_modules_source' => 'rsync://10.20.0.2:/puppet/modules/',
'puppet_manifests_source' => 'rsync://10.20.0.2:/puppet/manifests/'
},
{'uid' => 2}
]
}
let(:mclient) do
mclient = mock_rpcclient(nodes)
Astute::MClient.any_instance.stubs(:rpcclient).returns(mclient)
Astute::MClient.any_instance.stubs(:log_result).returns(mclient)
Astute::MClient.any_instance.stubs(:check_results_with_retries).returns(mclient)
mclient
end
let(:master_ip) { nodes.first['master_ip'] }
it "should sync puppet modules and manifests mcollective client 'puppetsync'" do
mclient.expects(:rsync).with(:modules_source => "rsync://10.20.0.2:/puppet/modules/",
:manifests_source => "rsync://10.20.0.2:/puppet/manifests/"
)
deployer.deploy(nodes)
end
it 'should able to customize path for puppet modules and manifests' do
modules_source = 'rsync://10.20.0.2:/puppet/vX/modules/'
manifests_source = 'rsync://10.20.0.2:/puppet/vX/manifests/'
nodes.first['puppet_modules_source'] = modules_source
nodes.first['puppet_manifests_source'] = manifests_source
mclient.expects(:rsync).with(:modules_source => modules_source,
:manifests_source => manifests_source
)
deployer.deploy(nodes)
end
context 'retry sync if mcollective raise error and' do
it 'raise error if retry fail SYNC_RETRIES times' do
mclient.stubs(:rsync)
Astute::MClient.any_instance.stubs(:check_results_with_retries)
.raises(Astute::MClientError)
.times(Astute::DeploymentEngine::SYNC_RETRIES)
expect { deployer.deploy(nodes) }.to raise_error(Astute::MClientError)
end
it 'not raise error if mcollective return success less than SYNC_RETRIES attempts' do
mclient.stubs(:rsync)
Astute::MClient.any_instance.stubs(:check_results_with_retries)
.raises(Astute::MClientError)
.then.returns("")
expect { deployer.deploy(nodes) }.to_not raise_error(Astute::MClientError)
end
end
it 'should raise exception if modules/manifests schema of uri is not equal' do
nodes.first['puppet_manifests_source'] = 'rsync://10.20.0.2:/puppet/vX/modules/'
nodes.first['puppet_manifests_source'] = 'http://10.20.0.2:/puppet/vX/manifests/'
expect { deployer.deploy(nodes) }.to raise_error(Astute::DeploymentEngineError,
/Scheme for puppet_modules_source 'rsync' and puppet_manifests_source/)
end
it 'should raise exception if modules/manifests source uri is incorrect' do
nodes.first['puppet_manifests_source'] = ':/puppet/modules/'
expect { deployer.deploy(nodes) }.to raise_error(Astute::DeploymentEngineError,
/bad URI/)
end
it 'should raise exception if schema of uri is incorrect' do
nodes.first['puppet_modules_source'] = 'http2://localhost/puppet/modules/'
nodes.first['puppet_manifests_source'] = 'http2://localhost/puppet/manifests/'
mclient.expects(:rsync).never
expect { deployer.deploy(nodes) }.to raise_error(Astute::DeploymentEngineError,
/Unknown scheme /)
end
end
describe '#update_repo_sources' do
before(:each) do
deployer.stubs(:generate_ssh_keys)
deployer.stubs(:upload_ssh_keys)
deployer.stubs(:sync_puppet_manifests)
deployer.stubs(:enable_puppet_deploy)
deployer.stubs(:deploy_piece)
deployer.stubs(:sync_time)
end
let(:nodes) do
[
{'uid' => 1,
'deployment_id' => 1,
'cobbler' => {
'profile' => 'centos-x86_64'
},
'repo_metadata' => {
'Nailgun' => 'http://10.20.0.2:8080/centos/fuelweb/x86_64/',
}
},
{'uid' => 2}
]
end
let(:mclient) do
mclient = mock_rpcclient(nodes)
Astute::MClient.any_instance.stubs(:rpcclient).returns(mclient)
Astute::MClient.any_instance.stubs(:log_result).returns(mclient)
Astute::MClient.any_instance.stubs(:check_results_with_retries).returns(mclient)
mclient
end
context 'source configuration generation' do
before(:each) do
deployer.stubs(:regenerate_metadata)
end
it 'should generate correct config for centos' do
content = ["[nailgun]",
"name=Nailgun",
"baseurl=http://10.20.0.2:8080/centos/fuelweb/x86_64/",
"gpgcheck=0"].join("\n")
deployer.expects(:upload_repo_source).with(nodes, content)
deployer.deploy(nodes)
end
it 'should generate correct config for ubuntu' do
nodes.first['cobbler']['profile'] = 'ubuntu_1204_x86_64'
nodes.first['repo_metadata']['Nailgun'] =
'http://10.20.0.2:8080/ubuntu/fuelweb/x86_64 precise main'
content = "deb http://10.20.0.2:8080/ubuntu/fuelweb/x86_64 precise main"
deployer.expects(:upload_repo_source).with(nodes, content)
deployer.deploy(nodes)
end
it 'should raise error if os not recognized' do
nodes.first['cobbler']['profile'] = 'unknown'
expect {deployer.deploy(nodes)}.to raise_error(Astute::DeploymentEngineError,
/Unknown system/)
end
end # source configuration generation
context 'new source configuration uploading' do
let(:repo_content) { "repo conf" }
before(:each) do
deployer.stubs(:generate_repo_source).returns(repo_content)
deployer.stubs(:regenerate_metadata)
end
it 'should upload config in correct place for centos' do
mclient.expects(:upload).with(:path => '/etc/yum.repos.d/nailgun.repo',
:content => repo_content,
:user_owner => 'root',
:group_owner => 'root',
:permissions => '0644',
:dir_permissions => '0755',
:overwrite => true,
:parents => true
)
deployer.deploy(nodes)
end
it 'should upload config in correct place for ubuntu' do
nodes.first['cobbler']['profile'] = 'ubuntu_1204_x86_64'
mclient.expects(:upload).with(:path => '/etc/apt/sources.list',
:content => repo_content,
:user_owner => 'root',
:group_owner => 'root',
:permissions => '0644',
:dir_permissions => '0755',
:overwrite => true,
:parents => true
)
deployer.deploy(nodes)
end
end #new source configuration uploading
context 'metadata regeneration' do
let(:fail_return) { [{:data => {:exit_code => 1}}] }
before(:each) do
deployer.stubs(:sync_time)
deployer.stubs(:generate_repo_source)
deployer.stubs(:upload_repo_source)
end
let(:success_return) { [{:data => {:exit_code => 0}}] }
it 'should regenerate metadata for centos' do
mclient.expects(:execute).with(:cmd => 'yum clean all').returns(success_return)
deployer.deploy(nodes)
end
it 'should regenerate metadata for ubuntu' do
nodes.first['cobbler']['profile'] = 'ubuntu_1204_x86_64'
mclient.expects(:execute).with(:cmd => 'apt-get clean; apt-get update').returns(success_return)
deployer.deploy(nodes)
end
it 'should raise error if metadata not updated' do
nodes.first['cobbler']['profile'] = 'ubuntu_1204_x86_64'
mclient.expects(:execute).with(:cmd => 'apt-get clean; apt-get update').returns(fail_return).times(Astute.config[:MC_RETRIES])
expect { deployer.deploy(nodes) }.to raise_error(Astute::DeploymentEngineError,
/Run command:/)
end
it 'should retry metadata update several time if get error' do
nodes.first['cobbler']['profile'] = 'ubuntu_1204_x86_64'
mclient.expects(:execute).with(:cmd => 'apt-get clean; apt-get update').returns(fail_return)
.then.returns(success_return).twice
deployer.deploy(nodes)
end
end #'metadata regeneration'
end # update_repo_sources
describe '#generation and uploading of ssh keys' do
before(:each) do
Astute.config.PUPPET_SSH_KEYS = ['nova']
deployer.stubs(:sync_time)
deployer.stubs(:deploy_piece)
deployer.stubs(:sync_puppet_manifests)
deployer.stubs(:enable_puppet_deploy)
end
let(:nodes) { [{'uid' => 1, 'deployment_id' => 1}, {'uid' => 2}] }
it 'should use Astute.config to get the ssh names that need to generate' do
deployer.expects(:generate_ssh_keys).with(nodes.first['deployment_id'])
deployer.expects(:upload_ssh_keys).with([1, 2], nodes.first['deployment_id'])
deployer.deploy(nodes)
end
it 'should raise error if deployment_id is not set' do
nodes = [{'uid' => 1}, {'uid' => 2}]
expect { deployer.deploy(nodes) }.to raise_error('Deployment_id is missing')
end
context 'generation of ssh keys' do
before(:each) do
deployer.stubs(:upload_ssh_keys).with([1, 2], nodes.first['deployment_id'])
end
it 'should save files in correct place: KEY_DIR/<name of key>/' do
Engine.any_instance.stubs(:run_system_command).returns([0, "", ""])
Dir.mktmpdir do |temp_dir|
Astute::DeploymentEngine.const_set 'KEY_DIR', temp_dir
deployer.deploy(nodes)
expect { File.directory? File.join(temp_dir, 'nova') }.to be_true
end
end
it 'should raise error if directory for key was not created' do
FileUtils.stubs(:mkdir_p).returns(false)
File.stubs(:directory?).returns(false)
expect { deployer.deploy(nodes) }.to raise_error(Astute::DeploymentEngineError,
/Could not create directory/)
end
it 'should raise error if ssh key generation fail' do
FileUtils.stubs(:mkdir_p).returns(true)
File.stubs(:directory?).returns(true)
Engine.any_instance.stubs(:run_system_command).returns([1, "", ""])
expect { deployer.deploy(nodes) }.to raise_error(Astute::DeploymentEngineError,
/Could not generate ssh key! Command:/)
end
it 'should raise error if ssh key generation command not find' do
FileUtils.stubs(:mkdir_p).returns(true)
File.stubs(:directory?).returns(true)
Engine.any_instance.stubs(:run_system_command).returns([127, "Command not found", ""])
expect { deployer.deploy(nodes) }.to raise_error(Astute::DeploymentEngineError,
/Command not found/)
end
it 'should run ssh key generation with correct command' do
FileUtils.stubs(:mkdir_p).returns(true)
File.stubs(:directory?).returns(true)
key_path = File.join(Engine::KEY_DIR, nodes.first['deployment_id'].to_s, 'nova', 'nova')
cmd = "ssh-keygen -b 2048 -t rsa -N '' -f #{key_path} 2>&1"
Engine.any_instance.expects(:run_system_command).with(cmd).returns([0, "", ""])
deployer.deploy(nodes)
end
it 'should not overwrite files' do
Dir.mktmpdir do |temp_dir|
Astute::DeploymentEngine.const_set 'KEY_DIR', temp_dir
key_path = File.join(temp_dir,'nova', 'nova')
FileUtils.mkdir_p File.join(temp_dir,'nova')
File.open(key_path, 'w') { |file| file.write("say no overwrite") }
deployer.deploy(nodes)
expect { File.exist? File.join(key_path, 'nova', 'nova') }.to be_true
expect { File.read File.join(key_path, 'nova', 'nova') == "say no overwrite" }.to be_true
end
end
it 'should check next key if find existing' do
Astute.config.PUPPET_SSH_KEYS = ['nova', 'test']
nova_key_path = File.join(Engine::KEY_DIR, nodes.first['deployment_id'].to_s, 'nova', 'nova')
test_key_path = File.join(Engine::KEY_DIR, nodes.first['deployment_id'].to_s, 'test', 'test')
FileUtils.stubs(:mkdir_p).returns(true).twice
File.stubs(:directory?).returns(true).twice
File.stubs(:exist?).with(nova_key_path).returns(true)
File.stubs(:exist?).with(test_key_path).returns(false)
Engine.any_instance.expects(:run_system_command).returns([0, "", ""])
deployer.deploy(nodes)
end
end # end context
context 'upload ssh keys' do
before(:each) do
deployer.stubs(:generate_ssh_keys)
end
it "should upload ssh keys using mcollective client 'uploadfile'" do
mclient = mock_rpcclient(nodes)
Astute::MClient.any_instance.stubs(:rpcclient).returns(mclient)
Astute::MClient.any_instance.stubs(:log_result).returns(mclient)
Astute::MClient.any_instance.stubs(:check_results_with_retries).returns(mclient)
File.stubs(:read).returns("private key").then.returns("public key")
mclient.expects(:upload).with(:path => File.join(Engine::KEY_DIR, 'nova', 'nova'),
:content => "private key",
:user_owner => 'root',
:group_owner => 'root',
:permissions => '0600',
:dir_permissions => '0700',
:overwrite => true,
:parents => true
)
mclient.expects(:upload).with(:path => File.join(Engine::KEY_DIR, 'nova', 'nova.pub'),
:content => "public key",
:user_owner => 'root',
:group_owner => 'root',
:permissions => '0600',
:dir_permissions => '0700',
:overwrite => true,
:parents => true
)
deployer.deploy(nodes)
end
end # context
end # describe
describe '#sync_time' do
before(:each) do
deployer.stubs(:generate_ssh_keys)
deployer.stubs(:upload_ssh_keys)
deployer.stubs(:sync_puppet_manifests)
deployer.stubs(:enable_puppet_deploy)
deployer.stubs(:update_repo_sources)
deployer.stubs(:deploy_piece)
end
let(:nodes) { [{'uid' => 1, 'deployment_id' => 1}, {'uid' => 2}] }
it 'should sync time between cluster nodes' do
deployer.expects(:sync_time).with([1,2])
deployer.deploy(nodes)
end
it 'should not raise exception if fail' do
deployer.stubs(:run_shell_command_remotely).returns(false)
expect {deployer.deploy(nodes)}.to_not raise_error
end
it 'should try to sync several times if fail' do
deployer.stubs(:run_shell_command_remotely).returns(false)
.then.returns(true).twice
deployer.deploy(nodes)
end
end #sync_time
end

View File

@ -49,11 +49,7 @@ describe "NailyFact DeploymentEngine" do
before(:each) do
uniq_nodes_uid = deploy_data.map {|n| n['uid'] }.uniq
deploy_engine.stubs(:generate_ssh_keys).with(deploy_data.first['deployment_id'])
deploy_engine.stubs(:upload_ssh_keys).with(uniq_nodes_uid, deploy_data.first['deployment_id'])
deploy_engine.stubs(:sync_puppet_manifests).with(deploy_data.uniq { |n| n['uid'] })
deploy_engine.stubs(:enable_puppet_deploy).with(uniq_nodes_uid)
deploy_engine.stubs(:sync_time)
Astute::PreDeploymentActions.any_instance.stubs(:process).returns(nil)
Astute::PreDeployActions.any_instance.stubs(:process).returns(nil)
Astute::PreNodeActions.any_instance.stubs(:process).returns(nil)
Astute::PreDeployActions.any_instance.stubs(:process).returns(nil)
@ -65,7 +61,6 @@ describe "NailyFact DeploymentEngine" do
end
it "it should not raise an exception if deployment mode is unknown" do
deploy_engine.expects(:upload_facts).times(deploy_data.size)
Astute::PuppetdDeployer.stubs(:deploy).with(ctx, deploy_data, instance_of(Fixnum)).once
expect {deploy_engine.deploy(deploy_data)}.to_not raise_exception
end
@ -77,8 +72,6 @@ describe "NailyFact DeploymentEngine" do
end
it "should not raise any exception" do
deploy_engine.expects(:upload_facts).times(deploy_data.size)
# we got two calls, one for controller (high priority), and another for all computes (same low priority)
Astute::PuppetdDeployer.expects(:deploy).with(ctx, controller_nodes, instance_of(Fixnum)).once
Astute::PuppetdDeployer.expects(:deploy).with(ctx, compute_nodes, instance_of(Fixnum)).once
@ -99,7 +92,6 @@ describe "NailyFact DeploymentEngine" do
let(:node_amount) { deploy_data.size }
it "should prepare log parsing for every deploy call because node may be deployed several times" do
deploy_engine.expects(:upload_facts).times(node_amount)
ctx.deploy_log_parser.expects(:prepare).with(compute_nodes).once
ctx.deploy_log_parser.expects(:prepare).with(cinder_nodes).once
@ -109,7 +101,6 @@ describe "NailyFact DeploymentEngine" do
end
it "should generate and publish facts for every deploy call because node may be deployed several times" do
deploy_engine.expects(:upload_facts).times(node_amount)
ctx.deploy_log_parser.expects(:prepare).with(compute_nodes).once
ctx.deploy_log_parser.expects(:prepare).with(cinder_nodes).once
@ -125,8 +116,6 @@ describe "NailyFact DeploymentEngine" do
end
it "ha deploy should not raise any exception" do
deploy_engine.expects(:upload_facts).at_least_once
primary_controller = deploy_data.find { |n| n['role'] == 'primary-controller' }
Astute::PuppetdDeployer.expects(:deploy).with(ctx, [primary_controller], 2).once
@ -144,7 +133,6 @@ describe "NailyFact DeploymentEngine" do
end
it "ha deploy should not raise any exception if there are only one controller" do
deploy_engine.expects(:upload_facts).at_least_once
Astute::PuppetdDeployer.expects(:deploy).once
deploy_engine.deploy(deploy_data)

View File

@ -0,0 +1,42 @@
# Copyright 2014 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::EnablePuppetDeploy do
include SpecHelpers
let(:ctx) do
tctx = mock_ctx
tctx.stubs(:status).returns({})
tctx
end
let(:mclient) do
mclient = mock_rpcclient(deploy_data)
Astute::MClient.any_instance.stubs(:rpcclient).returns(mclient)
Astute::MClient.any_instance.stubs(:log_result).returns(mclient)
Astute::MClient.any_instance.stubs(:check_results_with_retries).returns(mclient)
mclient
end
let(:deploy_data) { [{'uid' => 1, 'deployment_id' => 1}, {'uid' => 2}] }
let(:enable_puppet_deploy) { Astute::EnablePuppetDeploy.new }
it 'should enable puppet for all nodes' do
mclient.expects(:enable)
enable_puppet_deploy.process(deploy_data, ctx)
end
end

View File

@ -0,0 +1,128 @@
# Copyright 2014 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::GenerateSshKeys do
include SpecHelpers
before(:each) do
Astute.config.PUPPET_SSH_KEYS = ['nova']
end
let(:ctx) do
tctx = mock_ctx
tctx.stubs(:status).returns({})
tctx
end
let(:deploy_data) { [{'uid' => 1, 'deployment_id' => 1}, {'uid' => 2}] }
let(:generate_ssh_keys) { Astute::GenerateSshKeys.new }
# it 'should use Astute.config to get the ssh names that need to generate' do
# deployer.expects(:generate_ssh_keys).with(nodes.first['deployment_id'])
# deployer.expects(:upload_ssh_keys).with([1, 2], nodes.first['deployment_id'])
# generate_ssh_keys(deploy_data, ctx)
# end
around(:each) do |example|
old_ssh_keys_dir = Astute.config.PUPPET_SSH_KEYS_DIR
example.run
Astute.config.PUPPET_SSH_KEYS_DIR = old_ssh_keys_dir
end
it 'should raise error if deployment_id is not set' do
nodes = [{'uid' => 1}, {'uid' => 2}]
expect { generate_ssh_keys.process(nodes, ctx) }.to raise_error('Deployment_id is missing')
end
it 'should save files in correct place: KEY_DIR/<name of key>/' do
generate_ssh_keys.stubs(:run_system_command).returns([0, "", ""])
Dir.mktmpdir do |temp_dir|
Astute.config.PUPPET_SSH_KEYS_DIR = temp_dir
generate_ssh_keys.process(deploy_data, ctx)
expect { File.directory? File.join(temp_dir, 'nova') }.to be_true
end
end
it 'should raise error if directory for key was not created' do
FileUtils.stubs(:mkdir_p).returns(false)
File.stubs(:directory?).returns(false)
expect { generate_ssh_keys.process(deploy_data, ctx) }.to raise_error(Astute::DeploymentEngineError,
/Could not create directory/)
end
it 'should raise error if ssh key generation fail' do
FileUtils.stubs(:mkdir_p).returns(true)
File.stubs(:directory?).returns(true)
generate_ssh_keys.stubs(:run_system_command).returns([1, "", ""])
expect { generate_ssh_keys.process(deploy_data, ctx) }.to raise_error(Astute::DeploymentEngineError,
/Could not generate ssh key! Command:/)
end
it 'should raise error if ssh key generation command not found' do
FileUtils.stubs(:mkdir_p).returns(true)
File.stubs(:directory?).returns(true)
generate_ssh_keys.stubs(:run_system_command).returns([127, "Command not found", ""])
expect { generate_ssh_keys.process(deploy_data, ctx) }.to raise_error(Astute::DeploymentEngineError,
/Command not found/)
end
it 'should run ssh key generation with correct command' do
FileUtils.stubs(:mkdir_p).returns(true)
File.stubs(:directory?).returns(true)
key_path = File.join(Astute.config.PUPPET_SSH_KEYS_DIR, deploy_data.first['deployment_id'].to_s, 'nova', 'nova')
cmd = "ssh-keygen -b 2048 -t rsa -N '' -f #{key_path} 2>&1"
generate_ssh_keys.expects(:run_system_command).with(cmd).returns([0, "", ""])
generate_ssh_keys.process(deploy_data, ctx)
end
it 'should not overwrite files' do
Dir.mktmpdir do |temp_dir|
Astute.config.PUPPET_SSH_KEYS_DIR = temp_dir
key_path = File.join(temp_dir,'nova', 'nova')
FileUtils.mkdir_p File.join(temp_dir,'nova')
File.open(key_path, 'w') { |file| file.write("say no overwrite") }
generate_ssh_keys.process(deploy_data, ctx)
expect { File.exist? File.join(key_path, 'nova', 'nova') }.to be_true
expect { File.read File.join(key_path, 'nova', 'nova') == "say no overwrite" }.to be_true
end
end
it 'should check next key if find existing' do
Astute.config.PUPPET_SSH_KEYS = ['nova', 'test']
nova_key_path = File.join(Astute.config.PUPPET_SSH_KEYS_DIR, deploy_data.first['deployment_id'].to_s, 'nova', 'nova')
test_key_path = File.join(Astute.config.PUPPET_SSH_KEYS_DIR, deploy_data.first['deployment_id'].to_s, 'test', 'test')
FileUtils.stubs(:mkdir_p).returns(true).twice
File.stubs(:directory?).returns(true).twice
File.stubs(:exist?).with(nova_key_path).returns(true)
File.stubs(:exist?).with(test_key_path).returns(false)
generate_ssh_keys.expects(:run_system_command).returns([0, "", ""])
generate_ssh_keys.process(deploy_data, ctx)
end
end

View File

@ -0,0 +1,106 @@
# Copyright 2014 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::SyncPuppetStuff do
include SpecHelpers
let(:ctx) do
tctx = mock_ctx
tctx.stubs(:status).returns({})
tctx
end
let(:sync_puppet_stuff) { Astute::SyncPuppetStuff.new }
let(:nodes) { [
{'uid' => 1,
'deployment_id' => 1,
'master_ip' => '10.20.0.2',
'puppet_modules_source' => 'rsync://10.20.0.2:/puppet/modules/',
'puppet_manifests_source' => 'rsync://10.20.0.2:/puppet/manifests/'
},
{'uid' => 2}
]
}
let(:mclient) do
mclient = mock_rpcclient(nodes)
Astute::MClient.any_instance.stubs(:rpcclient).returns(mclient)
Astute::MClient.any_instance.stubs(:log_result).returns(mclient)
Astute::MClient.any_instance.stubs(:check_results_with_retries).returns(mclient)
mclient
end
let(:master_ip) { nodes.first['master_ip'] }
it "should sync puppet modules and manifests mcollective client 'puppetsync'" do
mclient.expects(:rsync).with(:modules_source => "rsync://10.20.0.2:/puppet/modules/",
:manifests_source => "rsync://10.20.0.2:/puppet/manifests/"
)
sync_puppet_stuff.process(nodes, ctx)
end
it 'should able to customize path for puppet modules and manifests' do
modules_source = 'rsync://10.20.0.2:/puppet/vX/modules/'
manifests_source = 'rsync://10.20.0.2:/puppet/vX/manifests/'
nodes.first['puppet_modules_source'] = modules_source
nodes.first['puppet_manifests_source'] = manifests_source
mclient.expects(:rsync).with(:modules_source => modules_source,
:manifests_source => manifests_source
)
sync_puppet_stuff.process(nodes, ctx)
end
context 'retry sync if mcollective raise error and' do
it 'raise error if retry fail SYNC_RETRIES times' do
mclient.stubs(:rsync)
Astute::MClient.any_instance.stubs(:check_results_with_retries)
.raises(Astute::MClientError)
.times(Astute::DeploymentEngine::SYNC_RETRIES)
expect { sync_puppet_stuff.process(nodes, ctx) }.to raise_error(Astute::MClientError)
end
it 'not raise error if mcollective return success less than SYNC_RETRIES attempts' do
mclient.stubs(:rsync)
Astute::MClient.any_instance.stubs(:check_results_with_retries)
.raises(Astute::MClientError)
.then.returns("")
expect { sync_puppet_stuff.process(nodes, ctx) }.to_not raise_error(Astute::MClientError)
end
end
it 'should raise exception if modules/manifests schema of uri is not equal' do
nodes.first['puppet_manifests_source'] = 'rsync://10.20.0.2:/puppet/vX/modules/'
nodes.first['puppet_manifests_source'] = 'http://10.20.0.2:/puppet/vX/manifests/'
expect { sync_puppet_stuff.process(nodes, ctx) }.to raise_error(Astute::DeploymentEngineError,
/Scheme for puppet_modules_source 'rsync' and puppet_manifests_source/)
end
it 'should raise exception if modules/manifests source uri is incorrect' do
nodes.first['puppet_manifests_source'] = ':/puppet/modules/'
expect { sync_puppet_stuff.process(nodes, ctx) }.to raise_error(Astute::DeploymentEngineError,
/bad URI/)
end
it 'should raise exception if schema of uri is incorrect' do
nodes.first['puppet_modules_source'] = 'http2://localhost/puppet/modules/'
nodes.first['puppet_manifests_source'] = 'http2://localhost/puppet/manifests/'
mclient.expects(:rsync).never
expect { sync_puppet_stuff.process(nodes, ctx) }.to raise_error(Astute::DeploymentEngineError,
/Unknown scheme /)
end
end

View File

@ -0,0 +1,50 @@
# Copyright 2014 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::SyncTime do
include SpecHelpers
let(:ctx) do
tctx = mock_ctx
tctx.stubs(:status).returns({})
tctx
end
let(:deploy_data) { [{'uid' => 1, 'deployment_id' => 1}, {'uid' => 2}] }
let(:sync_time) { Astute::SyncTime.new }
it 'should sync time between cluster nodes' do
sync_time.expects(:run_shell_command_remotely).with(
ctx,
[1,2],
"ntpdate -u $(egrep '^server' /etc/ntp.conf | sed '/^#/d' | awk '{print $2}')"
).returns(true)
sync_time.process(deploy_data, ctx)
end
it 'should not raise exception if fail' do
sync_time.stubs(:run_shell_command_remotely).returns(false)
expect { sync_time.process(deploy_data, ctx) }.to_not raise_error
end
it 'should try to sync several times if fail' do
sync_time.stubs(:run_shell_command_remotely).returns(false)
.then.returns(true).twice
sync_time.process(deploy_data, ctx)
end
end

View File

@ -0,0 +1,168 @@
# Copyright 2014 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::UpdateRepoSources do
include SpecHelpers
before(:each) do
Astute.config.PUPPET_SSH_KEYS = ['nova']
end
let(:ctx) do
tctx = mock_ctx
tctx.stubs(:status).returns({})
tctx
end
let(:mclient) do
mclient = mock_rpcclient(nodes)
Astute::MClient.any_instance.stubs(:rpcclient).returns(mclient)
Astute::MClient.any_instance.stubs(:log_result).returns(mclient)
Astute::MClient.any_instance.stubs(:check_results_with_retries).returns(mclient)
mclient
end
let(:nodes) do
[
{'uid' => 1,
'deployment_id' => 1,
'cobbler' => {
'profile' => 'centos-x86_64'
},
'repo_metadata' => {
'Nailgun' => 'http://10.20.0.2:8080/centos/fuelweb/x86_64/',
}
},
{'uid' => 2}
]
end
let(:update_repo_sources) { Astute::UpdateRepoSources.new }
around(:each) do |example|
old_ssh_keys_dir = Astute.config.PUPPET_SSH_KEYS_DIR
example.run
Astute.config.PUPPET_SSH_KEYS_DIR = old_ssh_keys_dir
end
context 'source configuration generation' do
before(:each) do
update_repo_sources.stubs(:regenerate_metadata)
end
it 'should generate correct config for centos' do
content = ["[nailgun]",
"name=Nailgun",
"baseurl=http://10.20.0.2:8080/centos/fuelweb/x86_64/",
"gpgcheck=0"].join("\n")
update_repo_sources.expects(:upload_repo_source).with(ctx, nodes, content)
update_repo_sources.process(nodes, ctx)
end
it 'should generate correct config for ubuntu' do
nodes.first['cobbler']['profile'] = 'ubuntu_1204_x86_64'
nodes.first['repo_metadata']['Nailgun'] =
'http://10.20.0.2:8080/ubuntu/fuelweb/x86_64 precise main'
content = "deb http://10.20.0.2:8080/ubuntu/fuelweb/x86_64 precise main"
update_repo_sources.expects(:upload_repo_source).with(ctx, nodes, content)
update_repo_sources.process(nodes, ctx)
end
it 'should raise error if os not recognized' do
nodes.first['cobbler']['profile'] = 'unknown'
expect {update_repo_sources.process(nodes, ctx)}.to raise_error(
Astute::DeploymentEngineError, /Unknown system/)
end
end # source configuration generation
context 'new source configuration uploading' do
let(:repo_content) { "repo conf" }
before(:each) do
update_repo_sources.stubs(:generate_repo_source).returns(repo_content)
update_repo_sources.stubs(:regenerate_metadata)
end
it 'should upload config in correct place for centos' do
mclient.expects(:upload).with(:path => '/etc/yum.repos.d/nailgun.repo',
:content => repo_content,
:user_owner => 'root',
:group_owner => 'root',
:permissions => '0644',
:dir_permissions => '0755',
:overwrite => true,
:parents => true
)
update_repo_sources.process(nodes, ctx)
end
it 'should upload config in correct place for ubuntu' do
nodes.first['cobbler']['profile'] = 'ubuntu_1204_x86_64'
mclient.expects(:upload).with(:path => '/etc/apt/sources.list',
:content => repo_content,
:user_owner => 'root',
:group_owner => 'root',
:permissions => '0644',
:dir_permissions => '0755',
:overwrite => true,
:parents => true
)
update_repo_sources.process(nodes, ctx)
end
end #new source configuration uploading
context 'metadata regeneration' do
let(:fail_return) { [{:data => {:exit_code => 1}}] }
before(:each) do
update_repo_sources.stubs(:generate_repo_source)
update_repo_sources.stubs(:upload_repo_source)
end
let(:success_return) { [{:data => {:exit_code => 0}}] }
it 'should regenerate metadata for centos' do
mclient.expects(:execute).with(:cmd => 'yum clean all').returns(success_return)
update_repo_sources.process(nodes, ctx)
end
it 'should regenerate metadata for ubuntu' do
nodes.first['cobbler']['profile'] = 'ubuntu_1204_x86_64'
mclient.expects(:execute).with(:cmd => 'apt-get clean; apt-get update').returns(success_return)
update_repo_sources.process(nodes, ctx)
end
it 'should raise error if metadata not updated' do
nodes.first['cobbler']['profile'] = 'ubuntu_1204_x86_64'
mclient.expects(:execute).with(:cmd => 'apt-get clean; apt-get update').returns(fail_return).times(Astute.config[:MC_RETRIES])
expect { update_repo_sources.process(nodes, ctx) }.to raise_error(Astute::DeploymentEngineError,
/Run command:/)
end
it 'should retry metadata update several time if get error' do
nodes.first['cobbler']['profile'] = 'ubuntu_1204_x86_64'
mclient.expects(:execute).with(:cmd => 'apt-get clean; apt-get update').returns(fail_return)
.then.returns(success_return).twice
update_repo_sources.process(nodes, ctx)
end
end #'metadata regeneration'
end

View File

@ -0,0 +1,61 @@
# Copyright 2014 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::UploadSshKeys do
include SpecHelpers
before(:each) do
Astute.config.PUPPET_SSH_KEYS = ['nova']
end
let(:ctx) do
tctx = mock_ctx
tctx.stubs(:status).returns({})
tctx
end
let(:deploy_data) { [{'uid' => 1, 'deployment_id' => 1}, {'uid' => 2}] }
let(:upload_ssh_keys) { Astute::UploadSshKeys.new }
it "should upload ssh keys using mcollective client 'uploadfile'" do
mclient = mock_rpcclient(deploy_data)
Astute::MClient.any_instance.stubs(:rpcclient).returns(mclient)
Astute::MClient.any_instance.stubs(:log_result).returns(mclient)
Astute::MClient.any_instance.stubs(:check_results_with_retries).returns(mclient)
File.stubs(:read).returns("private key").then.returns("public key")
mclient.expects(:upload).with(:path => File.join(Astute.config.PUPPET_SSH_KEYS_DIR, 'nova', 'nova'),
:content => "private key",
:user_owner => 'root',
:group_owner => 'root',
:permissions => '0600',
:dir_permissions => '0700',
:overwrite => true,
:parents => true
)
mclient.expects(:upload).with(:path => File.join(Astute.config.PUPPET_SSH_KEYS_DIR, 'nova', 'nova.pub'),
:content => "public key",
:user_owner => 'root',
:group_owner => 'root',
:permissions => '0600',
:dir_permissions => '0700',
:overwrite => true,
:parents => true
)
upload_ssh_keys.process(deploy_data, ctx)
end
end

235
spec/unit/tasklib_spec.rb Normal file
View File

@ -0,0 +1,235 @@
# 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 "TaskLib DeploymentEngine" do
include SpecHelpers
let(:deploy_engine) do
Astute::DeploymentEngine::Tasklib.new(ctx)
end
let(:nodes) do
[
{
'uid' => '1',
'role' => 'primary-controller',
'tasks' => [
{'name' => 'pr_controller_1', 'description' => 'test1'},
{'name' => 'pr_controller_2', 'description' => 'test2'},
{'name' => 'controller_3', 'description' => 'test3'}
],
'fail_if_error' => true
},
{
'uid' => '2',
'role' => 'controller',
'tasks' => [
{'name' => 'controller_1', 'description' => 'test1'},
{'name' => 'controller_3', 'description' => 'test3'}
],
'fail_if_error' => false
}
]
end
let(:ctx) do
ctx = mock
ctx.stubs(:task_id)
ctx.stubs(:deploy_log_parser).returns(Astute::LogParser::NoParsing.new)
ctx.stubs(:status).returns({})
reporter = mock
reporter.stubs(:report)
up_reporter = Astute::ProxyReporter::DeploymentProxyReporter.new(reporter, nodes)
ctx.stubs(:reporter).returns(up_reporter)
ctx
end
let(:mclient) do
mclient = mock_rpcclient([nodes.first])
Astute::MClient.any_instance.stubs(:rpcclient).returns(mclient)
Astute::MClient.any_instance.stubs(:log_result).returns(mclient)
Astute::MClient.any_instance.stubs(:check_results_with_retries).returns(mclient)
mclient
end
context 'call structure' do
it 'run pre tasks hook' do
deploy_engine.stubs(:deploy_nodes)
deploy_engine.stubs(:post_tasklib_deploy)
deploy_engine.expects(:pre_tasklib_deploy).once
deploy_engine.deploy_piece(nodes)
end
it 'run tasks deploy' do
deploy_engine.stubs(:pre_tasklib_deploy)
deploy_engine.stubs(:post_tasklib_deploy).once
deploy_engine.expects(:deploy_nodes)
deploy_engine.deploy_piece(nodes)
end
it 'run after tasks hook' do
deploy_engine.stubs(:pre_tasklib_deploy)
deploy_engine.stubs(:deploy_nodes)
deploy_engine.expects(:post_tasklib_deploy).once
deploy_engine.deploy_piece(nodes)
end
end #'call structure'
context 'pre tasks hook' do
before(:each) do
deploy_engine.stubs(:deploy_nodes)
deploy_engine.stubs(:post_tasklib_deploy)
end
it 'send initial status for node' do
deploy_engine.stubs(:upload_facts)
ctx.reporter.expects(:report).with('nodes' => [
{'uid' => '1', 'status' => 'deploying', 'role' => 'primary-controller', 'progress' => 0},
{'uid' => '2', 'status' => 'deploying', 'role' => 'controller', 'progress' => 0}
]).once
deploy_engine.deploy_piece(nodes)
end
end
context 'post tasks hook' do
before(:each) do
deploy_engine.stubs(:pre_tasklib_deploy)
deploy_engine.stubs(:deploy_nodes)
end
it 'show spend time' do
deploy_engine.instance_variable_set(:@time_before, Time.now.to_i)
deploy_engine.deploy_piece(nodes)
end
end
context 'task flow' do
before(:each) do
deploy_engine.stubs(:pre_tasklib_deploy)
deploy_engine.stubs(:post_tasklib_deploy)
deploy_engine.stubs(:check_status)
.with('1', 'pr_controller_1')
.returns(:running)
.then.returns(:ended_successfully)
deploy_engine.stubs(:check_status).with('1', 'pr_controller_2')
.returns(:running)
.then.returns(:running)
.then.returns(:ended_successfully)
deploy_engine.stubs(:check_status).with('1', 'controller_3')
.returns(:ended_successfully)
deploy_engine.stubs(:check_status).with('2', 'controller_1')
.returns(:running)
.then.returns(:unexpected_error)
end
it 'run all tasks' do
deploy_engine.expects(:run_task).with('1', 'pr_controller_1')
deploy_engine.expects(:run_task).with('1', 'pr_controller_2')
deploy_engine.expects(:run_task).with('1', 'controller_3')
deploy_engine.expects(:run_task).with('2', 'controller_1')
ctx.stubs(:report_and_update_status)
deploy_engine.deploy_piece(nodes)
end
it 'report and update node status' do
deploy_engine.stubs(:run_task).times(4)
mock_calculator = mock
ctx.stubs(:deploy_log_parser).returns(mock_calculator)
mock_calculator.stubs(:progress_calculate).with(['1'], anything).returns(
['uid' => '1',
'progress' => 30]
)
mock_calculator.stubs(:progress_calculate).with(['2'], anything).returns(
['uid' => '2',
'progress' => 50]
)
ctx.expects(:report_and_update_status).with('nodes' => [
{'uid' => '1', 'status' => 'deploying', 'role' => 'primary-controller', 'progress' => 30, 'task' => 'pr_controller_1'},
{'uid' => '2', 'status' => 'deploying', 'role' => 'controller', 'progress' => 50, 'task' => 'controller_1'}])
ctx.expects(:report_and_update_status).with('nodes' => [
{'uid' => '1', 'status' => 'deploying', 'role' => 'primary-controller', 'progress' => 30, 'task' => 'pr_controller_2'}
]).twice
ctx.expects(:report_and_update_status).with('nodes' => [
{'uid' => '1', 'status' => 'ready', 'role' => 'primary-controller', 'progress' => 100, 'task' => 'controller_3'}
])
ctx.expects(:report_and_update_status).with('nodes' => [
{'uid' => '2', 'status' => 'error', 'error_type' => 'deploy', 'role' => 'controller', 'task' => 'controller_1'}
])
deploy_engine.deploy_piece(nodes)
end
it 'raise error if tasklib return known status without handler' do
deploy_engine.stubs(:run_task)
deploy_engine.stubs(:check_status)
.with('1', 'pr_controller_1')
.returns(:unknown_state)
expect { deploy_engine.deploy_piece(nodes) }.to raise_error(/Known status 'unknown_state', but handler not provided/)
end
it 'raise error if tasklib return unknown exit code' do
deploy_engine.stubs(:run_task)
ctx.stubs(:report_and_update_status)
deploy_engine.unstub(:check_status)
mclient.stubs(:execute).returns([{:data => {:exit_code => '12'}}])
expect { deploy_engine.deploy_piece(nodes) }.to raise_error(/Internal error. Unknown status '12'/)
end
it 'run task using tasklib' do
ctx.stubs(:report_and_update_status)
tasklib_mclient = mock('tasklib_mclient')
deploy_engine.stubs(:tasklib_mclient).with(is_a(String)).returns(tasklib_mclient)
tasklib_mclient.expects(:execute).with(has_entry(:cmd => regexp_matches(/taskcmd daemon/))).returns([]).times(4)
deploy_engine.deploy_piece(nodes)
end
it 'run with debug option if debug option is true' do
ctx.stubs(:report_and_update_status)
nodes.first['debug'] = true
tasklib_mclient = mock('tasklib_mclient')
deploy_engine.stubs(:tasklib_mclient).with(is_a(String)).returns(tasklib_mclient)
tasklib_mclient.expects(:execute).with(has_entry(:cmd => regexp_matches(/taskcmd --debug daemon/))).returns([]).times(4)
deploy_engine.deploy_piece(nodes)
end
end
end