Federation support
- Federation proxy lifecycle management - Federation lifecycle management - Federation job submission and management - Mount Azure File share for auto-rotated log persistence - FIFO within job support - Constraint matching - Federations can be created in "unique job id" mode requiring all submitted jobs via fed jobs add be unique across the entire federation - Supports nearly 15K actions per job (in non-unique job id mode) - Task dependency rewrite engine for federated jobs - Verify dependencies only within task group - Uniquely identify task dependencies - Allow tuning of scheduling behavior options - Package federation logic on proxy into Docker container - Full guide/walkthrough for federation feature - Refactor common code between monitor/fed proxy into resource - Other doc updates
This commit is contained in:
Родитель
977c2e920b
Коммит
52628d27cf
|
@ -155,16 +155,20 @@ def _submit_task_sub_collection(
|
|||
retry = []
|
||||
for result in results.value:
|
||||
if result.status == batchmodels.TaskAddStatus.client_error:
|
||||
de = [
|
||||
'{}: {}'.format(x.key, x.value)
|
||||
for x in result.error.values
|
||||
]
|
||||
de = None
|
||||
if result.error.values is not None:
|
||||
de = [
|
||||
'{}: {}'.format(x.key, x.value)
|
||||
for x in result.error.values
|
||||
]
|
||||
logger.error(
|
||||
('skipping retry of adding task {} as it '
|
||||
'returned a client error (code={} message={} {}) '
|
||||
'for job {}').format(
|
||||
result.task_id, result.error.code,
|
||||
result.error.message, ' '.join(de), job_id))
|
||||
result.error.message,
|
||||
' '.join(de) if de is not None else '',
|
||||
job_id))
|
||||
elif (result.status ==
|
||||
batchmodels.TaskAddStatus.server_error):
|
||||
retry.append(task_map[result.task_id])
|
||||
|
|
|
@ -0,0 +1,46 @@
|
|||
federation:
|
||||
storage_account_settings: mystorageaccount
|
||||
location: <Azure region, e.g., eastus>
|
||||
resource_group: my-federation-proxy-rg
|
||||
hostname_prefix: fed
|
||||
ssh:
|
||||
username: shipyard
|
||||
ssh_public_key: /path/to/rsa/publickey.pub
|
||||
ssh_public_key_data: ssh-rsa ...
|
||||
ssh_private_key: /path/to/rsa/privatekey
|
||||
generated_file_export_path: null
|
||||
public_ip:
|
||||
enabled: true
|
||||
static: false
|
||||
virtual_network:
|
||||
name: myvnet
|
||||
resource_group: my-vnet-resource-group
|
||||
existing_ok: false
|
||||
address_space: 10.0.0.0/16
|
||||
subnet:
|
||||
name: my-federation-proxy-subnet
|
||||
address_prefix: 10.0.0.0/24
|
||||
network_security:
|
||||
ssh:
|
||||
- '*'
|
||||
custom_inbound_rules:
|
||||
myrule:
|
||||
destination_port_range: 5000-5001
|
||||
protocol: '*'
|
||||
source_address_prefix:
|
||||
- 1.2.3.4
|
||||
- 5.6.7.0/24
|
||||
vm_size: STANDARD_D2_V2
|
||||
accelerated_networking: false
|
||||
proxy_options:
|
||||
polling_interval:
|
||||
federations: 15
|
||||
actions: 5
|
||||
logging:
|
||||
persistence: true
|
||||
level: debug
|
||||
filename: fedproxy.log
|
||||
scheduling:
|
||||
after_success:
|
||||
blackout_interval: 15
|
||||
evaluate_autoscale: true
|
|
@ -60,6 +60,37 @@ job_specifications:
|
|||
- jobdata*.bin
|
||||
blobxfer_extra_options: null
|
||||
default_working_dir: batch
|
||||
federation_constraints:
|
||||
pool:
|
||||
autoscale:
|
||||
allow: true
|
||||
exclusive: false
|
||||
low_priority_nodes:
|
||||
allow: true
|
||||
exclusive: false
|
||||
native: false
|
||||
windows: false
|
||||
location: eastus
|
||||
container_registries:
|
||||
private_docker_hub: true
|
||||
public:
|
||||
- my.public.registry.io
|
||||
max_active_task_backlog:
|
||||
ratio: null
|
||||
autoscale_exempt: true
|
||||
custom_image_arm_id: null
|
||||
virtual_network_arm_id: null
|
||||
compute_node:
|
||||
vm_size: STANDARD_F1
|
||||
cores:
|
||||
amount: 2
|
||||
schedulable_variance: null
|
||||
memory:
|
||||
amount: 512m
|
||||
schedulable_variance: null
|
||||
exclusive: false
|
||||
gpu: false
|
||||
infiniband: false
|
||||
tasks:
|
||||
- id: null
|
||||
docker_image: busybox
|
||||
|
|
556
convoy/batch.py
556
convoy/batch.py
|
@ -35,6 +35,7 @@ import concurrent.futures
|
|||
import datetime
|
||||
import fnmatch
|
||||
import getpass
|
||||
import json
|
||||
import logging
|
||||
import multiprocessing
|
||||
import os
|
||||
|
@ -42,11 +43,10 @@ try:
|
|||
import pathlib2 as pathlib
|
||||
except ImportError:
|
||||
import pathlib
|
||||
import pickle
|
||||
import ssl
|
||||
import sys
|
||||
import tempfile
|
||||
import time
|
||||
import uuid
|
||||
# non-stdlib imports
|
||||
import azure.batch.models as batchmodels
|
||||
import azure.mgmt.batch.models as mgmtbatchmodels
|
||||
|
@ -4033,16 +4033,20 @@ def _submit_task_sub_collection(
|
|||
retry = []
|
||||
for result in results.value:
|
||||
if result.status == batchmodels.TaskAddStatus.client_error:
|
||||
de = [
|
||||
'{}: {}'.format(x.key, x.value)
|
||||
for x in result.error.values
|
||||
]
|
||||
de = None
|
||||
if result.error.values is not None:
|
||||
de = [
|
||||
'{}: {}'.format(x.key, x.value)
|
||||
for x in result.error.values
|
||||
]
|
||||
logger.error(
|
||||
('skipping retry of adding task {} as it '
|
||||
'returned a client error (code={} message={} {}) '
|
||||
'for job {}').format(
|
||||
result.task_id, result.error.code,
|
||||
result.error.message, ' '.join(de), job_id))
|
||||
result.error.message,
|
||||
' '.join(de) if de is not None else '',
|
||||
job_id))
|
||||
elif (result.status ==
|
||||
batchmodels.TaskAddStatus.server_error):
|
||||
retry.append(task_map[result.task_id])
|
||||
|
@ -4097,24 +4101,25 @@ def _generate_non_native_env_dump(env_vars, envfile):
|
|||
|
||||
|
||||
def _construct_task(
|
||||
batch_client, blob_client, keyvault_client, config, bxfile,
|
||||
bs, native, is_windows, tempdisk, allow_run_on_missing,
|
||||
batch_client, blob_client, keyvault_client, config, federation_id,
|
||||
bxfile, bs, native, is_windows, tempdisk, allow_run_on_missing,
|
||||
docker_missing_images, singularity_missing_images, cloud_pool,
|
||||
pool, jobspec, job_id, job_env_vars, task_map, existing_tasklist,
|
||||
reserved_task_id, lasttaskid, is_merge_task, uses_task_dependencies,
|
||||
on_task_failure, _task):
|
||||
on_task_failure, container_image_refs, _task):
|
||||
# type: (batch.BatchServiceClient, azureblob.BlockBlobService,
|
||||
# azure.keyvault.KeyVaultClient, dict, tuple,
|
||||
# azure.keyvault.KeyVaultClient, dict, str, tuple,
|
||||
# settings.BatchShipyardSettings, bool, bool, str, bool,
|
||||
# list, list, batchmodels.CloudPool, settings.PoolSettings,
|
||||
# dict, str, dict, dict, list, str, str, bool, bool,
|
||||
# batchmodels.OnTaskFailure, dict) -> str
|
||||
# batchmodels.OnTaskFailure, set, dict) -> tuple
|
||||
"""Contruct a Batch task and add it to the task map
|
||||
:param batch_client: The batch client to use.
|
||||
:type batch_client: `azure.batch.batch_service_client.BatchServiceClient`
|
||||
:param azure.storage.blob.BlockBlobService blob_client: blob client
|
||||
:param azure.keyvault.KeyVaultClient keyvault_client: keyvault client
|
||||
:param dict config: configuration dict
|
||||
:param str federation_id: federation id
|
||||
:param tuple bxfile: blobxfer file
|
||||
:param settings.BatchShipyardSettings bs: batch shipyard settings
|
||||
:param bool native: native pool
|
||||
|
@ -4134,9 +4139,11 @@ def _construct_task(
|
|||
:param bool is_merge_task: is merge task
|
||||
:param bool uses_task_dependencies: uses task dependencies
|
||||
:param batchmodels.OntaskFailure on_task_failure: on task failure
|
||||
:param set container_image_refs: container image references
|
||||
:param dict _task: task spec
|
||||
:rtype: tuple
|
||||
:return: (list of committed task ids for job, task id added to task map)
|
||||
:return: (list of committed task ids for job, task id added to task map,
|
||||
instance count for task, has gpu task, has ib task)
|
||||
"""
|
||||
_task_id = settings.task_id(_task)
|
||||
if util.is_none_or_empty(_task_id):
|
||||
|
@ -4149,8 +4156,14 @@ def _construct_task(
|
|||
settings.set_task_name(_task, '{}-{}'.format(job_id, _task_id))
|
||||
del _task_id
|
||||
task = settings.task_settings(
|
||||
cloud_pool, config, pool, jobspec, _task)
|
||||
cloud_pool, config, pool, jobspec, _task, federation_id=federation_id)
|
||||
is_singularity = util.is_not_empty(task.singularity_image)
|
||||
if util.is_not_empty(federation_id):
|
||||
if is_singularity:
|
||||
container_image_refs.add(task.singularity_image)
|
||||
else:
|
||||
container_image_refs.add(task.docker_image)
|
||||
task_ic = 1
|
||||
# retrieve keyvault task env vars
|
||||
if util.is_not_empty(
|
||||
task.environment_variables_keyvault_secret_id):
|
||||
|
@ -4219,6 +4232,7 @@ def _construct_task(
|
|||
coordination_command_line=cc,
|
||||
common_resource_files=[],
|
||||
)
|
||||
task_ic = task.multi_instance.num_instances
|
||||
del cc
|
||||
# add common resource files for multi-instance
|
||||
if util.is_not_empty(task.multi_instance.resource_files):
|
||||
|
@ -4418,19 +4432,24 @@ def _construct_task(
|
|||
'duplicate task id detected: {} for job {}'.format(
|
||||
task.id, job_id))
|
||||
task_map[task.id] = batchtask
|
||||
return existing_tasklist, task.id
|
||||
return existing_tasklist, task.id, task_ic, task.gpu, task.infiniband
|
||||
|
||||
|
||||
def add_jobs(
|
||||
batch_client, blob_client, keyvault_client, config, autopool, jpfile,
|
||||
bxfile, recreate=False, tail=None):
|
||||
batch_client, blob_client, table_client, queue_client, keyvault_client,
|
||||
config, autopool, jpfile, bxfile, recreate=False, tail=None,
|
||||
federation_id=None):
|
||||
# type: (batch.BatchServiceClient, azureblob.BlockBlobService,
|
||||
# azure.cosmosdb.TableClient, azurequeue.QueueService,
|
||||
# azure.keyvault.KeyVaultClient, dict,
|
||||
# batchmodels.PoolSpecification, tuple, tuple, bool, str) -> None
|
||||
# batchmodels.PoolSpecification, tuple, tuple, bool, str,
|
||||
# str) -> None
|
||||
"""Add jobs
|
||||
:param batch_client: The batch client to use.
|
||||
:type batch_client: `azure.batch.batch_service_client.BatchServiceClient`
|
||||
:param azure.storage.blob.BlockBlobService blob_client: blob client
|
||||
:param azure.cosmosdb.TableService table_client: table client
|
||||
:param azure.storage.queue.QueueService queue_client: queue_client
|
||||
:param azure.keyvault.KeyVaultClient keyvault_client: keyvault client
|
||||
:param dict config: configuration dict
|
||||
:param batchmodels.PoolSpecification autopool: auto pool specification
|
||||
|
@ -4438,7 +4457,20 @@ def add_jobs(
|
|||
:param tuple bxfile: blobxfer file
|
||||
:param bool recreate: recreate job if completed
|
||||
:param str tail: tail specified file of last job/task added
|
||||
:param str federation_id: federation id
|
||||
"""
|
||||
# check option compatibility
|
||||
if util.is_not_empty(federation_id):
|
||||
if autopool is not None:
|
||||
raise RuntimeError(
|
||||
'cannot create an auto-pool job within a federation')
|
||||
if recreate:
|
||||
raise RuntimeError(
|
||||
'cannot recreate a job within a federation')
|
||||
if tail is not None:
|
||||
raise RuntimeError(
|
||||
'cannot tail task output for the specified file within '
|
||||
'a federation')
|
||||
# get the pool inter-node comm setting
|
||||
bs = settings.batch_shipyard_settings(config)
|
||||
pool = settings.pool_settings(config)
|
||||
|
@ -4451,7 +4483,7 @@ def add_jobs(
|
|||
except batchmodels.batch_error.BatchErrorException as ex:
|
||||
if 'The specified pool does not exist' in ex.message.value:
|
||||
cloud_pool = None
|
||||
if autopool is None:
|
||||
if autopool is None and util.is_none_or_empty(federation_id):
|
||||
logger.error('{} pool does not exist'.format(pool.id))
|
||||
if not util.confirm_action(
|
||||
config,
|
||||
|
@ -4467,8 +4499,8 @@ def add_jobs(
|
|||
singularity_images = settings.global_resources_singularity_images(config)
|
||||
lastjob = None
|
||||
lasttaskid = None
|
||||
jobschedule = None
|
||||
tasksadded = False
|
||||
raw_output = {}
|
||||
for jobspec in settings.job_specifications(config):
|
||||
job_id = settings.job_id(jobspec)
|
||||
lastjob = job_id
|
||||
|
@ -4478,6 +4510,7 @@ def add_jobs(
|
|||
# 3. if tasks have dependencies, set it if so
|
||||
# 4. if there are multi-instance tasks
|
||||
auto_complete = settings.job_auto_complete(jobspec)
|
||||
jobschedule = None
|
||||
multi_instance = False
|
||||
mi_docker_container_name = None
|
||||
reserved_task_id = None
|
||||
|
@ -4488,6 +4521,19 @@ def add_jobs(
|
|||
allow_run_on_missing = settings.job_allow_run_on_missing(jobspec)
|
||||
existing_tasklist = None
|
||||
has_merge_task = settings.job_has_merge_task(jobspec)
|
||||
max_instance_count_in_job = 0
|
||||
instances_required_in_job = 0
|
||||
# set federation overrides from constraints
|
||||
if util.is_not_empty(federation_id):
|
||||
fed_constraints = settings.job_federation_constraint_settings(
|
||||
jobspec, federation_id)
|
||||
if fed_constraints.pool.native is not None:
|
||||
native = fed_constraints.pool.native
|
||||
if fed_constraints.pool.windows is not None:
|
||||
is_windows = fed_constraints.pool.windows
|
||||
allow_run_on_missing = True
|
||||
else:
|
||||
fed_constraints = None
|
||||
for task in settings.job_tasks(config, jobspec):
|
||||
# check if task docker image is set in config.json
|
||||
di = settings.task_docker_image(task)
|
||||
|
@ -4559,7 +4605,7 @@ def add_jobs(
|
|||
)
|
||||
# construct job prep
|
||||
jpcmd = []
|
||||
if not native:
|
||||
if not native and util.is_none_or_empty(federation_id):
|
||||
if len(docker_missing_images) > 0 and allow_run_on_missing:
|
||||
# we don't want symmetric difference as we just want to
|
||||
# block on pre-loaded images only
|
||||
|
@ -4588,6 +4634,12 @@ def add_jobs(
|
|||
# job prep: digest any input_data
|
||||
addlcmds = data.process_input_data(config, bxfile, jobspec)
|
||||
if addlcmds is not None:
|
||||
if util.is_not_empty(federation_id):
|
||||
tfm = 'alfpark/batch-shipyard:{}-cargo'.format(__version__)
|
||||
if tfm in addlcmds:
|
||||
raise RuntimeError(
|
||||
'input_data:azure_batch is not supported at the '
|
||||
'job-level for federations')
|
||||
jpcmd.append(addlcmds)
|
||||
del addlcmds
|
||||
jptask = None
|
||||
|
@ -4687,7 +4739,8 @@ def add_jobs(
|
|||
batchmodels.OnAllTasksComplete.no_action
|
||||
)
|
||||
# check pool settings for kill job on completion
|
||||
if kill_job_on_completion:
|
||||
if (kill_job_on_completion and
|
||||
util.is_none_or_empty(federation_id)):
|
||||
if cloud_pool is not None:
|
||||
total_vms = (
|
||||
cloud_pool.current_dedicated_nodes +
|
||||
|
@ -4800,8 +4853,6 @@ def add_jobs(
|
|||
)
|
||||
del jscs
|
||||
del jscmdline
|
||||
else:
|
||||
jobschedule = None
|
||||
del recurrence
|
||||
# create job
|
||||
if jobschedule is None:
|
||||
|
@ -4821,9 +4872,15 @@ def add_jobs(
|
|||
],
|
||||
priority=settings.job_priority(jobspec),
|
||||
)
|
||||
logger.info('Adding job {} to pool {}'.format(job_id, pool.id))
|
||||
try:
|
||||
batch_client.job.add(job)
|
||||
if util.is_none_or_empty(federation_id):
|
||||
logger.info('Adding job {} to pool {}'.format(
|
||||
job_id, pool.id))
|
||||
batch_client.job.add(job)
|
||||
else:
|
||||
logger.info(
|
||||
'deferring adding job {} for federation {}'.format(
|
||||
job_id, federation_id))
|
||||
if settings.verbose(config) and jptask is not None:
|
||||
logger.debug('Job prep command: {}'.format(
|
||||
jptask.command_line))
|
||||
|
@ -4869,27 +4926,51 @@ def add_jobs(
|
|||
on_task_failure.value))
|
||||
else:
|
||||
raise
|
||||
del multi_instance
|
||||
del mi_docker_container_name
|
||||
# add all tasks under job
|
||||
container_image_refs = set()
|
||||
task_map = {}
|
||||
has_gpu_task = False
|
||||
has_ib_task = False
|
||||
for _task in settings.job_tasks(config, jobspec):
|
||||
existing_tasklist, lasttaskid = _construct_task(
|
||||
batch_client, blob_client, keyvault_client, config, bxfile,
|
||||
bs, native, is_windows, tempdisk, allow_run_on_missing,
|
||||
docker_missing_images, singularity_missing_images, cloud_pool,
|
||||
pool, jobspec, job_id, job_env_vars, task_map,
|
||||
existing_tasklist, reserved_task_id, lasttaskid, False,
|
||||
uses_task_dependencies, on_task_failure, _task)
|
||||
existing_tasklist, lasttaskid, lasttaskic, gpu, ib = \
|
||||
_construct_task(
|
||||
batch_client, blob_client, keyvault_client, config,
|
||||
federation_id, bxfile, bs, native, is_windows, tempdisk,
|
||||
allow_run_on_missing, docker_missing_images,
|
||||
singularity_missing_images, cloud_pool,
|
||||
pool, jobspec, job_id, job_env_vars, task_map,
|
||||
existing_tasklist, reserved_task_id, lasttaskid, False,
|
||||
uses_task_dependencies, on_task_failure,
|
||||
container_image_refs, _task
|
||||
)
|
||||
if not has_gpu_task and gpu:
|
||||
has_gpu_task = True
|
||||
if not has_ib_task and ib:
|
||||
has_ib_task = True
|
||||
instances_required_in_job += lasttaskic
|
||||
if lasttaskic > max_instance_count_in_job:
|
||||
max_instance_count_in_job = lasttaskic
|
||||
merge_task_id = None
|
||||
if has_merge_task:
|
||||
_task = settings.job_merge_task(jobspec)
|
||||
existing_tasklist, merge_task_id = _construct_task(
|
||||
batch_client, blob_client, keyvault_client, config, bxfile,
|
||||
bs, native, is_windows, tempdisk, allow_run_on_missing,
|
||||
docker_missing_images, singularity_missing_images, cloud_pool,
|
||||
pool, jobspec, job_id, job_env_vars, task_map,
|
||||
existing_tasklist, reserved_task_id, lasttaskid, True,
|
||||
uses_task_dependencies, on_task_failure, _task)
|
||||
existing_tasklist, merge_task_id, lasttaskic, gpu, ib = \
|
||||
_construct_task(
|
||||
batch_client, blob_client, keyvault_client, config,
|
||||
federation_id, bxfile, bs, native, is_windows, tempdisk,
|
||||
allow_run_on_missing, docker_missing_images,
|
||||
singularity_missing_images, cloud_pool,
|
||||
pool, jobspec, job_id, job_env_vars, task_map,
|
||||
existing_tasklist, reserved_task_id, lasttaskid, True,
|
||||
uses_task_dependencies, on_task_failure,
|
||||
container_image_refs, _task)
|
||||
if not has_gpu_task and gpu:
|
||||
has_gpu_task = True
|
||||
if not has_ib_task and ib:
|
||||
has_ib_task = True
|
||||
instances_required_in_job += lasttaskic
|
||||
if lasttaskic > max_instance_count_in_job:
|
||||
max_instance_count_in_job = lasttaskic
|
||||
# set dependencies on merge task
|
||||
merge_task = task_map.pop(merge_task_id)
|
||||
merge_task.depends_on = batchmodels.TaskDependencies(
|
||||
|
@ -4902,48 +4983,172 @@ def add_jobs(
|
|||
'please limit the the number of tasks').format(job_id))
|
||||
# add merge task into map
|
||||
task_map[merge_task_id] = merge_task
|
||||
# construct required registries for federation
|
||||
registries = construct_registry_list_for_federation(
|
||||
config, federation_id, fed_constraints, container_image_refs)
|
||||
del container_image_refs
|
||||
# submit job schedule if required
|
||||
if jobschedule is not None:
|
||||
taskmaploc = '{}jsrf-{}/{}'.format(
|
||||
bs.storage_entity_prefix, job_id, _TASKMAP_PICKLE_FILE)
|
||||
taskmaploc = 'jobschedules/{}/{}'.format(
|
||||
job_id, _TASKMAP_PICKLE_FILE)
|
||||
# pickle and upload task map
|
||||
f = tempfile.NamedTemporaryFile(mode='wb', delete=False)
|
||||
fname = f.name
|
||||
try:
|
||||
with open(fname, 'wb') as f:
|
||||
pickle.dump(task_map, f, protocol=pickle.HIGHEST_PROTOCOL)
|
||||
f.close()
|
||||
sas_urls = storage.upload_resource_files(
|
||||
blob_client, config, [(taskmaploc, fname)])
|
||||
finally:
|
||||
os.unlink(fname)
|
||||
del f
|
||||
del fname
|
||||
if len(sas_urls) != 1:
|
||||
raise RuntimeError('unexpected number of sas urls')
|
||||
sas_url = storage.pickle_and_upload(
|
||||
blob_client, task_map, taskmaploc, federation_id=federation_id)
|
||||
# attach as resource file to jm task
|
||||
jobschedule.job_specification.job_manager_task.resource_files.\
|
||||
append(
|
||||
batchmodels.ResourceFile(
|
||||
file_path=_TASKMAP_PICKLE_FILE,
|
||||
blob_source=next(iter(sas_urls.values())),
|
||||
blob_source=sas_url,
|
||||
file_mode='0640',
|
||||
)
|
||||
)
|
||||
# submit job schedule
|
||||
logger.info('Adding jobschedule {} to pool {}'.format(
|
||||
job_id, pool.id))
|
||||
batch_client.job_schedule.add(jobschedule)
|
||||
if util.is_none_or_empty(federation_id):
|
||||
logger.info('Adding jobschedule {} to pool {}'.format(
|
||||
job_id, pool.id))
|
||||
try:
|
||||
batch_client.job_schedule.add(jobschedule)
|
||||
except Exception:
|
||||
# delete uploaded task map
|
||||
storage.delete_resource_file(blob_client, taskmaploc)
|
||||
raise
|
||||
else:
|
||||
if storage.check_if_job_exists_in_federation(
|
||||
table_client, federation_id, jobschedule.id):
|
||||
# do not delete uploaded task map as the existing job
|
||||
# schedule will require it
|
||||
raise RuntimeError(
|
||||
'job schedule {} exists in federation id {}'.format(
|
||||
jobschedule.id, federation_id))
|
||||
kind = 'job_schedule'
|
||||
unique_id = uuid.uuid4()
|
||||
# ensure task dependencies are self-contained
|
||||
if uses_task_dependencies:
|
||||
try:
|
||||
task_map = rewrite_task_dependencies_for_federation(
|
||||
table_client, federation_id, jobschedule.id, kind,
|
||||
unique_id, task_map, merge_task_id)
|
||||
except Exception:
|
||||
# delete uploaded task map
|
||||
storage.delete_resource_file(
|
||||
blob_client, taskmaploc,
|
||||
federation_id=federation_id)
|
||||
raise
|
||||
# pickle and re-upload task map
|
||||
sas_url = storage.pickle_and_upload(
|
||||
blob_client, task_map, taskmaploc,
|
||||
federation_id=federation_id)
|
||||
logger.debug(
|
||||
'submitting job schedule {} for federation {}'.format(
|
||||
jobschedule.id, federation_id))
|
||||
# encapsulate job schedule/task map info in json
|
||||
queue_data, jsloc = \
|
||||
generate_info_metadata_for_federation_message(
|
||||
blob_client, config, unique_id, federation_id,
|
||||
fed_constraints, registries, kind, jobschedule.id,
|
||||
jobschedule, native, is_windows, auto_complete,
|
||||
multi_instance, uses_task_dependencies,
|
||||
has_gpu_task, has_ib_task, max_instance_count_in_job,
|
||||
instances_required_in_job, has_merge_task,
|
||||
merge_task_id, task_map
|
||||
)
|
||||
# enqueue action to global queue
|
||||
logger.debug('enqueuing action {} to federation {}'.format(
|
||||
unique_id, federation_id))
|
||||
try:
|
||||
storage.add_job_to_federation(
|
||||
table_client, queue_client, config, federation_id,
|
||||
unique_id, queue_data, kind)
|
||||
except Exception:
|
||||
# delete uploaded files
|
||||
storage.delete_resource_file(
|
||||
blob_client, taskmaploc, federation_id=federation_id)
|
||||
storage.delete_resource_file(
|
||||
blob_client, jsloc, federation_id=federation_id)
|
||||
raise
|
||||
# add to raw output
|
||||
if settings.raw(config):
|
||||
raw_output[jobschedule.id] = {
|
||||
'federation': {
|
||||
'id': federation_id,
|
||||
'storage': {
|
||||
'account': storage.get_storageaccount(),
|
||||
'endpoint':
|
||||
storage.get_storageaccount_endpoint(),
|
||||
},
|
||||
},
|
||||
'kind': kind,
|
||||
'action': 'add',
|
||||
'unique_id': str(unique_id),
|
||||
'tasks_per_recurrence': len(task_map),
|
||||
}
|
||||
else:
|
||||
# add task collection to job
|
||||
_add_task_collection(batch_client, job_id, task_map)
|
||||
# patch job if job autocompletion is needed
|
||||
if auto_complete:
|
||||
batch_client.job.patch(
|
||||
job_id=job_id,
|
||||
job_patch_parameter=batchmodels.JobPatchParameter(
|
||||
on_all_tasks_complete=batchmodels.
|
||||
OnAllTasksComplete.terminate_job))
|
||||
if util.is_none_or_empty(federation_id):
|
||||
_add_task_collection(batch_client, job_id, task_map)
|
||||
# patch job if job autocompletion is needed
|
||||
if auto_complete:
|
||||
batch_client.job.patch(
|
||||
job_id=job_id,
|
||||
job_patch_parameter=batchmodels.JobPatchParameter(
|
||||
on_all_tasks_complete=batchmodels.
|
||||
OnAllTasksComplete.terminate_job))
|
||||
else:
|
||||
if (storage.federation_requires_unique_job_ids(
|
||||
table_client, federation_id) and
|
||||
storage.check_if_job_exists_in_federation(
|
||||
table_client, federation_id, job_id)):
|
||||
raise RuntimeError(
|
||||
'job {} exists in federation id {} requiring unique '
|
||||
'job ids'.format(job_id, federation_id))
|
||||
kind = 'job'
|
||||
unique_id = uuid.uuid4()
|
||||
if uses_task_dependencies:
|
||||
task_map = rewrite_task_dependencies_for_federation(
|
||||
table_client, federation_id, job_id, kind, unique_id,
|
||||
task_map, merge_task_id)
|
||||
logger.debug('submitting job {} for federation {}'.format(
|
||||
job_id, federation_id))
|
||||
# encapsulate job/task map info in json
|
||||
queue_data, jloc = \
|
||||
generate_info_metadata_for_federation_message(
|
||||
blob_client, config, unique_id, federation_id,
|
||||
fed_constraints, registries, kind, job_id, job,
|
||||
native, is_windows, auto_complete, multi_instance,
|
||||
uses_task_dependencies, has_gpu_task,
|
||||
has_ib_task, max_instance_count_in_job,
|
||||
instances_required_in_job, has_merge_task,
|
||||
merge_task_id, task_map
|
||||
)
|
||||
# enqueue action to global queue
|
||||
logger.debug('enqueuing action {} to federation {}'.format(
|
||||
unique_id, federation_id))
|
||||
try:
|
||||
storage.add_job_to_federation(
|
||||
table_client, queue_client, config, federation_id,
|
||||
unique_id, queue_data, kind)
|
||||
except Exception:
|
||||
# delete uploaded files
|
||||
storage.delete_resource_file(
|
||||
blob_client, jloc, federation_id=federation_id)
|
||||
raise
|
||||
# add to raw output
|
||||
if settings.raw(config):
|
||||
raw_output[job_id] = {
|
||||
'federation': {
|
||||
'id': federation_id,
|
||||
'storage': {
|
||||
'account': storage.get_storageaccount(),
|
||||
'endpoint':
|
||||
storage.get_storageaccount_endpoint(),
|
||||
},
|
||||
},
|
||||
'kind': kind,
|
||||
'action': 'add',
|
||||
'unique_id': str(unique_id),
|
||||
'num_tasks': len(task_map),
|
||||
}
|
||||
tasksadded = True
|
||||
# tail file if specified
|
||||
if tail:
|
||||
|
@ -4955,3 +5160,216 @@ def add_jobs(
|
|||
stream_file_and_wait_for_task(
|
||||
batch_client, config, filespec='{},{},{}'.format(
|
||||
lastjob, lasttaskid, tail), disk=False)
|
||||
# output raw
|
||||
if util.is_not_empty(raw_output):
|
||||
print(json.dumps(raw_output, indent=4, sort_keys=True))
|
||||
|
||||
|
||||
def generate_info_metadata_for_federation_message(
|
||||
blob_client, config, unique_id, federation_id, fed_constraints,
|
||||
registries, kind, target, data, native, is_windows, auto_complete,
|
||||
multi_instance, uses_task_dependencies, has_gpu_task, has_ib_task,
|
||||
max_instance_count_in_job, instances_required_in_job, has_merge_task,
|
||||
merge_task_id, task_map):
|
||||
info = {
|
||||
'version': '1',
|
||||
'action': {
|
||||
'method': 'add',
|
||||
'kind': kind,
|
||||
},
|
||||
kind: {
|
||||
'id': target,
|
||||
'data': data,
|
||||
'constraints': {
|
||||
'pool': {
|
||||
'autoscale': {
|
||||
'allow': fed_constraints.pool.autoscale_allow,
|
||||
'exclusive': fed_constraints.pool.autoscale_exclusive,
|
||||
},
|
||||
'custom_image_arm_id':
|
||||
fed_constraints.pool.custom_image_arm_id,
|
||||
'location': fed_constraints.pool.location,
|
||||
'low_priority_nodes': {
|
||||
'allow': fed_constraints.pool.low_priority_nodes_allow,
|
||||
'exclusive':
|
||||
fed_constraints.pool.low_priority_nodes_exclusive,
|
||||
},
|
||||
'max_active_task_backlog': {
|
||||
'ratio':
|
||||
fed_constraints.pool.max_active_task_backlog_ratio,
|
||||
'autoscale_exempt':
|
||||
fed_constraints.pool.
|
||||
max_active_task_backlog_autoscale_exempt,
|
||||
},
|
||||
'native': native,
|
||||
'registries': registries,
|
||||
'virtual_network_arm_id':
|
||||
fed_constraints.pool.virtual_network_arm_id,
|
||||
'windows': is_windows,
|
||||
},
|
||||
'compute_node': {
|
||||
'vm_size': fed_constraints.compute_node.vm_size,
|
||||
'cores': {
|
||||
'amount': fed_constraints.compute_node.cores,
|
||||
'schedulable_variance':
|
||||
fed_constraints.compute_node.core_variance,
|
||||
},
|
||||
'memory': {
|
||||
'amount': fed_constraints.compute_node.memory,
|
||||
'schedulable_variance':
|
||||
fed_constraints.compute_node.memory_variance,
|
||||
},
|
||||
'exclusive': fed_constraints.compute_node.exclusive,
|
||||
'gpu': has_gpu_task or fed_constraints.compute_node.gpu,
|
||||
'infiniband': has_ib_task or
|
||||
fed_constraints.compute_node.infiniband,
|
||||
},
|
||||
'task': {
|
||||
'auto_complete': auto_complete,
|
||||
'has_multi_instance': multi_instance,
|
||||
'has_task_dependencies': uses_task_dependencies,
|
||||
'instance_counts': {
|
||||
'max': max_instance_count_in_job,
|
||||
'total': instances_required_in_job,
|
||||
},
|
||||
},
|
||||
},
|
||||
'task_naming': {
|
||||
'prefix': settings.autogenerated_task_id_prefix(config),
|
||||
'padding': settings.autogenerated_task_id_zfill(config),
|
||||
},
|
||||
},
|
||||
}
|
||||
if kind == 'jobschedule':
|
||||
info[kind]['constraints']['task'][
|
||||
'tasks_per_recurrence'] = len(task_map)
|
||||
elif kind == 'job':
|
||||
info['task_map'] = task_map
|
||||
if has_merge_task:
|
||||
info[kind]['constraints']['task']['merge_task_id'] = merge_task_id
|
||||
# pickle json and upload
|
||||
loc = 'messages/{}.pickle'.format(unique_id)
|
||||
sas_url = storage.pickle_and_upload(
|
||||
blob_client, info, loc, federation_id=federation_id)
|
||||
# construct queue message
|
||||
info = {
|
||||
'version': '1',
|
||||
'federation_id': federation_id,
|
||||
'target': target,
|
||||
'blob_data': sas_url,
|
||||
'uuid': str(unique_id),
|
||||
}
|
||||
return info, loc
|
||||
|
||||
|
||||
def construct_registry_list_for_federation(
|
||||
config, federation_id, fed_constraints, container_image_refs):
|
||||
if util.is_none_or_empty(federation_id):
|
||||
return None
|
||||
regs = settings.docker_registries(config, images=container_image_refs)
|
||||
# find docker hub repos
|
||||
dh_repos = set()
|
||||
for image in container_image_refs:
|
||||
tmp = image.split('/')
|
||||
if len(tmp) > 1:
|
||||
if '.' in tmp[0] or ':' in tmp[0] and tmp[0] != 'localhost':
|
||||
continue
|
||||
else:
|
||||
dh_repos.add('dockerhub-{}'.format(tmp[0]))
|
||||
if fed_constraints.pool.container_registries_private_docker_hub:
|
||||
req_regs = list(dh_repos)
|
||||
else:
|
||||
req_regs = []
|
||||
if util.is_not_empty(fed_constraints.pool.container_registries_public):
|
||||
pub_exclude = set(fed_constraints.pool.container_registries_public)
|
||||
else:
|
||||
pub_exclude = set()
|
||||
# filter registries according to constraints
|
||||
for cr in regs:
|
||||
if util.is_none_or_empty(cr.registry_server):
|
||||
continue
|
||||
else:
|
||||
if cr.registry_server not in pub_exclude:
|
||||
req_regs.append('{}-{}'.format(
|
||||
cr.registry_server, cr.user_name))
|
||||
return req_regs if util.is_not_empty(req_regs) else None
|
||||
|
||||
|
||||
def rewrite_task_dependencies_for_federation(
|
||||
table_client, federation_id, job_id, kind, unique_id, task_map,
|
||||
merge_task_id):
|
||||
# perform validation first
|
||||
# 1. no outside dependencies outside of task group
|
||||
# 2. for now, disallow task depends_on_range
|
||||
# TODO task depends_on range support:
|
||||
# - convert depends on range to explicit task depends on
|
||||
# 3. ensure the total length of dependencies for each task is less than
|
||||
# 64k chars
|
||||
ujid_req = storage.federation_requires_unique_job_ids(
|
||||
table_client, federation_id)
|
||||
uid = str(unique_id)[:8]
|
||||
all_tids = list(task_map.keys())
|
||||
task_remap = {}
|
||||
dep_len = 0
|
||||
for tid in task_map:
|
||||
if tid == merge_task_id:
|
||||
continue
|
||||
new_tid = '{}-{}'.format(tid, uid)
|
||||
if not ujid_req and len(new_tid) > 64:
|
||||
raise RuntimeError(
|
||||
'Cannot add unique suffix to task {} in {} {}. Please '
|
||||
'shorten the task id to a maximum of 55 characters.'.format(
|
||||
tid, kind, job_id))
|
||||
t = task_map[tid]
|
||||
if t.depends_on is not None:
|
||||
if util.is_not_empty(t.depends_on.task_ids):
|
||||
new_dep = []
|
||||
for x in t.depends_on.task_ids:
|
||||
if x not in all_tids:
|
||||
raise RuntimeError(
|
||||
'{} {} contains task dependencies not '
|
||||
'self-contained in task group bound for '
|
||||
'federation {}'.format(
|
||||
kind, job_id, federation_id))
|
||||
new_dep.append('{}-{}'.format(x, uid))
|
||||
if not ujid_req:
|
||||
t.depends_on = batchmodels.TaskDependencies(
|
||||
task_ids=new_dep
|
||||
)
|
||||
dep_len += len(''.join(new_dep))
|
||||
if util.is_not_empty(t.depends_on.task_id_ranges):
|
||||
raise RuntimeError(
|
||||
'{} {} contains task dependency ranges, which are not '
|
||||
'supported, bound for federation {}'.format(
|
||||
kind, job_id, federation_id))
|
||||
if not ujid_req:
|
||||
t.id = new_tid
|
||||
task_remap[tid] = t
|
||||
# passed self-containment check, can stop here for unique job id
|
||||
# federations
|
||||
if ujid_req:
|
||||
logger.debug(
|
||||
'federation {} requires unique job ids, not rewriting task '
|
||||
'dependencies for {} {}'.format(federation_id, kind, job_id))
|
||||
return task_map
|
||||
# remap merge task
|
||||
if util.is_not_empty(merge_task_id):
|
||||
new_tid = '{}-{}'.format(merge_task_id, uid)
|
||||
if len(new_tid) > 64:
|
||||
raise RuntimeError(
|
||||
'Cannot add unique suffix to merge task {} in {} {}. Please '
|
||||
'shorten the task id to a maximum of 55 characters.'.format(
|
||||
tid, kind, job_id))
|
||||
t = task_map[merge_task_id]
|
||||
t.depends_on = batchmodels.TaskDependencies(
|
||||
task_ids=list(task_remap.keys())
|
||||
)
|
||||
t.id = new_tid
|
||||
task_remap[new_tid] = t
|
||||
dep_len += len(new_tid)
|
||||
# check total dependency length
|
||||
if dep_len > 64000:
|
||||
raise RuntimeError(
|
||||
'Total number of dependencies for {} {} exceeds the maximum '
|
||||
'limit.'.format(kind, job_id))
|
||||
return task_remap
|
||||
|
|
|
@ -43,6 +43,7 @@ import azure.mgmt.network
|
|||
import azure.mgmt.resource
|
||||
import azure.mgmt.storage
|
||||
import azure.storage.blob as azureblob
|
||||
import azure.storage.queue as azurequeue
|
||||
# local imports
|
||||
from . import aad
|
||||
from . import settings
|
||||
|
@ -357,7 +358,7 @@ def create_storage_clients():
|
|||
# type: (None) -> tuple
|
||||
"""Create storage clients
|
||||
:rtype: tuple
|
||||
:return: blob_client, table_client
|
||||
:return: blob_client, table_client, queue_client
|
||||
"""
|
||||
account_name = storage.get_storageaccount()
|
||||
account_key = storage.get_storageaccount_key()
|
||||
|
@ -372,4 +373,9 @@ def create_storage_clients():
|
|||
account_key=account_key,
|
||||
endpoint_suffix=endpoint_suffix,
|
||||
)
|
||||
return blob_client, table_client
|
||||
queue_client = azurequeue.QueueService(
|
||||
account_name=account_name,
|
||||
account_key=account_key,
|
||||
endpoint_suffix=endpoint_suffix,
|
||||
)
|
||||
return blob_client, table_client, queue_client
|
||||
|
|
|
@ -54,6 +54,7 @@ util.setup_logger(logger)
|
|||
_SSH_KEY_PREFIX = 'id_rsa_shipyard'
|
||||
_REMOTEFS_SSH_KEY_PREFIX = '{}_remotefs'.format(_SSH_KEY_PREFIX)
|
||||
_MONITORING_SSH_KEY_PREFIX = '{}_monitoring'.format(_SSH_KEY_PREFIX)
|
||||
_FEDERATION_SSH_KEY_PREFIX = '{}_federation'.format(_SSH_KEY_PREFIX)
|
||||
# named tuples
|
||||
PfxSettings = collections.namedtuple(
|
||||
'PfxSettings', [
|
||||
|
@ -89,6 +90,15 @@ def get_monitoring_ssh_key_prefix():
|
|||
return _MONITORING_SSH_KEY_PREFIX
|
||||
|
||||
|
||||
def get_federation_ssh_key_prefix():
|
||||
# type: (None) -> str
|
||||
"""Get federation SSH key prefix
|
||||
:rtype: str
|
||||
:return: ssh key prefix for federation proxy
|
||||
"""
|
||||
return _FEDERATION_SSH_KEY_PREFIX
|
||||
|
||||
|
||||
def generate_rdp_password():
|
||||
# type: (None) -> str
|
||||
"""Generate an RDP password
|
||||
|
|
|
@ -0,0 +1,605 @@
|
|||
# Copyright (c) Microsoft Corporation
|
||||
#
|
||||
# All rights reserved.
|
||||
#
|
||||
# MIT License
|
||||
#
|
||||
# Permission is hereby granted, free of charge, to any person obtaining a
|
||||
# copy of this software and associated documentation files (the "Software"),
|
||||
# to deal in the Software without restriction, including without limitation
|
||||
# the rights to use, copy, modify, merge, publish, distribute, sublicense,
|
||||
# and/or sell copies of the Software, and to permit persons to whom the
|
||||
# Software is furnished to do so, subject to the following conditions:
|
||||
#
|
||||
# The above copyright notice and this permission notice shall be included in
|
||||
# all copies or substantial portions of the Software.
|
||||
#
|
||||
# THE SOFTWARE IS PROVIDED *AS IS*, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
|
||||
# IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
|
||||
# FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
|
||||
# AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
|
||||
# LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
|
||||
# FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
|
||||
# DEALINGS IN THE SOFTWARE.
|
||||
|
||||
# compat imports
|
||||
from __future__ import (
|
||||
absolute_import, division, print_function
|
||||
)
|
||||
from builtins import ( # noqa
|
||||
bytes, dict, int, list, object, range, str, ascii, chr, hex, input,
|
||||
next, oct, open, pow, round, super, filter, map, zip)
|
||||
# stdlib imports
|
||||
import functools
|
||||
import logging
|
||||
import json
|
||||
import os
|
||||
import uuid
|
||||
# non-stdlib imports
|
||||
import azure.mgmt.authorization.models as authmodels
|
||||
import msrestazure.azure_exceptions
|
||||
|
||||
# local imports
|
||||
from . import crypto
|
||||
from . import remotefs
|
||||
from . import resource
|
||||
from . import settings
|
||||
from . import storage
|
||||
from . import util
|
||||
from .version import __version__
|
||||
|
||||
# create logger
|
||||
logger = logging.getLogger(__name__)
|
||||
util.setup_logger(logger)
|
||||
|
||||
|
||||
def _create_virtual_machine_extension(
|
||||
compute_client, config, vm_resource, bootstrap_file, blob_urls,
|
||||
vm_name, private_ips, fqdn, offset, verbose=False):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient,
|
||||
# settings.VmResource, str, List[str], str, List[str], str,
|
||||
# int, bool) -> msrestazure.azure_operation.AzureOperationPoller
|
||||
"""Create a virtual machine extension
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param settings.VmResource vm_resource: VM resource
|
||||
:param str bootstrap_file: bootstrap file
|
||||
:param list blob_urls: blob urls
|
||||
:param str vm_name: vm name
|
||||
:param list private_ips: list of static private ips
|
||||
:param str fqdn: fqdn if public ip available
|
||||
:param int offset: vm number
|
||||
:param bool verbose: verbose logging
|
||||
:rtype: msrestazure.azure_operation.AzureOperationPoller
|
||||
:return: msrestazure.azure_operation.AzureOperationPoller
|
||||
"""
|
||||
bs = settings.batch_shipyard_settings(config)
|
||||
fpo = settings.federation_proxy_options_settings(config)
|
||||
# construct vm extensions
|
||||
vm_ext_name = settings.generate_virtual_machine_extension_name(
|
||||
vm_resource, offset)
|
||||
# try to get storage account resource group
|
||||
ssel = settings.federation_storage_account_settings(config)
|
||||
rg = settings.credentials_storage(config, ssel).resource_group
|
||||
# construct bootstrap command
|
||||
cmd = './{bsf}{a}{log}{p}{r}{s}{v}'.format(
|
||||
bsf=bootstrap_file[0],
|
||||
a=' -a {}'.format(settings.determine_cloud_type_from_aad(config)),
|
||||
log=' -l {}:{}:{}'.format(
|
||||
fpo.log_persistence, fpo.log_level, fpo.log_filename),
|
||||
p=' -p {}:{}'.format(
|
||||
fpo.scheduling_after_success_blackout_interval,
|
||||
fpo.scheduling_after_success_evaluate_autoscale),
|
||||
r=' -r {}:{}'.format(
|
||||
fpo.federations_polling_interval, fpo.actions_polling_interval),
|
||||
s=' -s {}:{}:{}'.format(
|
||||
storage.get_storageaccount(),
|
||||
rg if util.is_not_empty(rg) else '',
|
||||
bs.storage_entity_prefix
|
||||
),
|
||||
v=' -v {}'.format(__version__),
|
||||
)
|
||||
if verbose:
|
||||
logger.debug('bootstrap command: {}'.format(cmd))
|
||||
logger.debug('creating virtual machine extension: {}'.format(vm_ext_name))
|
||||
return compute_client.virtual_machine_extensions.create_or_update(
|
||||
resource_group_name=vm_resource.resource_group,
|
||||
vm_name=vm_name,
|
||||
vm_extension_name=vm_ext_name,
|
||||
extension_parameters=compute_client.virtual_machine_extensions.models.
|
||||
VirtualMachineExtension(
|
||||
location=vm_resource.location,
|
||||
publisher='Microsoft.Azure.Extensions',
|
||||
virtual_machine_extension_type='CustomScript',
|
||||
type_handler_version='2.0',
|
||||
auto_upgrade_minor_version=True,
|
||||
settings={
|
||||
'fileUris': blob_urls,
|
||||
},
|
||||
protected_settings={
|
||||
'commandToExecute': cmd,
|
||||
'storageAccountName': storage.get_storageaccount(),
|
||||
'storageAccountKey': storage.get_storageaccount_key(),
|
||||
},
|
||||
),
|
||||
)
|
||||
|
||||
|
||||
def create_federation_proxy(
|
||||
auth_client, resource_client, compute_client, network_client,
|
||||
blob_client, table_client, queue_client, config, resources_path,
|
||||
bootstrap_file, federation_files):
|
||||
# type: (azure.mgmt.authorization.AuthorizationManagementClient,
|
||||
# azure.mgmt.resource.resources.ResourceManagementClient,
|
||||
# azure.mgmt.compute.ComputeManagementClient,
|
||||
# azure.mgmt.network.NetworkManagementClient,
|
||||
# azure.storage.blob.BlockBlobService,
|
||||
# azure.cosmosdb.table.TableService,
|
||||
# azure.storage.queue.QueueService,
|
||||
# dict, pathlib.Path, Tuple[str, pathlib.Path],
|
||||
# List[Tuple[str, pathlib.Path]]) -> None
|
||||
"""Create a federation proxy
|
||||
:param azure.mgmt.authorization.AuthorizationManagementClient auth_client:
|
||||
auth client
|
||||
:param azure.mgmt.resource.resources.ResourceManagementClient
|
||||
resource_client: resource client
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param azure.storage.blob.BlockBlobService blob_client: blob client
|
||||
:param azure.cosmosdb.table.TableService table_client: table client
|
||||
:param azure.storage.queue.QueueService queue_client: queue client
|
||||
:param dict config: configuration dict
|
||||
:param pathlib.Path: resources path
|
||||
:param Tuple[str, pathlib.Path] bootstrap_file: customscript bootstrap file
|
||||
:param List[Tuple[str, pathlib.Path]] federation_files: federation files
|
||||
"""
|
||||
fs = settings.federation_settings(config)
|
||||
# get subscription id for msi
|
||||
sub_id = settings.credentials_management(config).subscription_id
|
||||
if util.is_none_or_empty(sub_id):
|
||||
raise ValueError('Management subscription id not specified')
|
||||
# check if cluster already exists
|
||||
logger.debug('checking if federation proxy exists')
|
||||
try:
|
||||
vm = compute_client.virtual_machines.get(
|
||||
resource_group_name=fs.resource_group,
|
||||
vm_name=settings.generate_virtual_machine_name(fs, 0)
|
||||
)
|
||||
raise RuntimeError(
|
||||
'Existing virtual machine {} found for federation proxy'.format(
|
||||
vm.id))
|
||||
except msrestazure.azure_exceptions.CloudError as e:
|
||||
if e.status_code == 404:
|
||||
pass
|
||||
else:
|
||||
raise
|
||||
# confirm before proceeding
|
||||
if not util.confirm_action(config, 'create federation proxy'):
|
||||
return
|
||||
# create resource group if it doesn't exist
|
||||
resource.create_resource_group(
|
||||
resource_client, fs.resource_group, fs.location)
|
||||
# create storage containers
|
||||
storage.create_storage_containers_nonbatch(
|
||||
blob_client, table_client, queue_client, 'federation')
|
||||
# create file share for log persistence
|
||||
bs = settings.batch_shipyard_settings(config)
|
||||
storage.create_file_share_saskey(
|
||||
settings.credentials_storage(
|
||||
config,
|
||||
bs.storage_account_settings,
|
||||
),
|
||||
'{}fedlogs'.format(bs.storage_entity_prefix),
|
||||
'ingress',
|
||||
create_share=True,
|
||||
)
|
||||
# create global lock
|
||||
storage.create_global_lock_blob(blob_client, 'federation')
|
||||
# upload scripts to blob storage for customscript vm extension
|
||||
blob_urls = storage.upload_for_nonbatch(
|
||||
blob_client, federation_files, 'federation')
|
||||
# async operation dictionary
|
||||
async_ops = {}
|
||||
# create nsg
|
||||
async_ops['nsg'] = resource.AsyncOperation(functools.partial(
|
||||
resource.create_network_security_group, network_client, fs))
|
||||
# use dynamic ips for private
|
||||
private_ips = None
|
||||
logger.debug('using dynamic private ip address allocation')
|
||||
# create virtual network and subnet if specified
|
||||
vnet, subnet = resource.create_virtual_network_and_subnet(
|
||||
resource_client, network_client,
|
||||
fs.virtual_network.resource_group, fs.location,
|
||||
fs.virtual_network)
|
||||
# create public ips
|
||||
pips = None
|
||||
if fs.public_ip.enabled:
|
||||
async_ops['pips'] = {}
|
||||
async_ops['pips'][0] = resource.AsyncOperation(functools.partial(
|
||||
resource.create_public_ip, network_client, fs, 0))
|
||||
logger.debug('waiting for public ips to provision')
|
||||
pips = {}
|
||||
for offset in async_ops['pips']:
|
||||
pip = async_ops['pips'][offset].result()
|
||||
logger.info(
|
||||
('public ip: {} [provisioning_state={} ip_address={} '
|
||||
'public_ip_allocation={}]').format(
|
||||
pip.id, pip.provisioning_state,
|
||||
pip.ip_address, pip.public_ip_allocation_method))
|
||||
pips[offset] = pip
|
||||
else:
|
||||
logger.info('public ip is disabled for monitoring resource')
|
||||
# get nsg
|
||||
logger.debug('waiting for network security group to provision')
|
||||
nsg = async_ops['nsg'].result()
|
||||
# create nics
|
||||
async_ops['nics'] = {}
|
||||
async_ops['nics'][0] = resource.AsyncOperation(functools.partial(
|
||||
resource.create_network_interface, network_client, fs, subnet, nsg,
|
||||
private_ips, pips, 0))
|
||||
# wait for nics to be created
|
||||
logger.debug('waiting for network interfaces to provision')
|
||||
nics = {}
|
||||
for offset in async_ops['nics']:
|
||||
nic = async_ops['nics'][offset].result()
|
||||
logger.info(
|
||||
('network interface: {} [provisioning_state={} private_ip={} '
|
||||
'private_ip_allocation_method={} network_security_group={} '
|
||||
'accelerated_networking={}]').format(
|
||||
nic.id, nic.provisioning_state,
|
||||
nic.ip_configurations[0].private_ip_address,
|
||||
nic.ip_configurations[0].private_ip_allocation_method,
|
||||
nsg.name if nsg is not None else None,
|
||||
nic.enable_accelerated_networking))
|
||||
nics[offset] = nic
|
||||
# read or generate ssh keys
|
||||
if util.is_not_empty(fs.ssh.ssh_public_key_data):
|
||||
key_data = fs.ssh.ssh_public_key_data
|
||||
else:
|
||||
# create universal ssh key for all vms if not specified
|
||||
ssh_pub_key = fs.ssh.ssh_public_key
|
||||
if ssh_pub_key is None:
|
||||
_, ssh_pub_key = crypto.generate_ssh_keypair(
|
||||
fs.ssh.generated_file_export_path,
|
||||
crypto.get_federation_ssh_key_prefix())
|
||||
# read public key data
|
||||
with ssh_pub_key.open('rb') as fd:
|
||||
key_data = fd.read().decode('utf8')
|
||||
ssh_pub_key = compute_client.virtual_machines.models.SshPublicKey(
|
||||
path='/home/{}/.ssh/authorized_keys'.format(fs.ssh.username),
|
||||
key_data=key_data,
|
||||
)
|
||||
# create vms
|
||||
async_ops['vms'] = {}
|
||||
async_ops['vms'][0] = resource.AsyncOperation(functools.partial(
|
||||
resource.create_virtual_machine, compute_client, fs, None, nics,
|
||||
None, ssh_pub_key, 0, enable_msi=True))
|
||||
# wait for vms to be created
|
||||
logger.info(
|
||||
'waiting for {} virtual machines to provision'.format(
|
||||
len(async_ops['vms'])))
|
||||
vms = {}
|
||||
for offset in async_ops['vms']:
|
||||
vms[offset] = async_ops['vms'][offset].result()
|
||||
logger.debug('{} virtual machines created'.format(len(vms)))
|
||||
# create role assignments for msi identity
|
||||
logger.debug('assigning roles to msi identity')
|
||||
sub_scope = '/subscriptions/{}/'.format(sub_id)
|
||||
cont_role = None
|
||||
for role in auth_client.role_definitions.list(
|
||||
sub_scope, filter='roleName eq \'Contributor\''):
|
||||
cont_role = role.id
|
||||
break
|
||||
if cont_role is None:
|
||||
raise RuntimeError('Role Id not found for Reader')
|
||||
role_assign = auth_client.role_assignments.create(
|
||||
scope=sub_scope,
|
||||
role_assignment_name=uuid.uuid4(),
|
||||
parameters=authmodels.RoleAssignmentCreateParameters(
|
||||
role_definition_id=cont_role,
|
||||
principal_id=vms[0].identity.principal_id
|
||||
),
|
||||
)
|
||||
if settings.verbose(config):
|
||||
logger.debug('reader role assignment: {}'.format(role_assign))
|
||||
cont_role = None
|
||||
for role in auth_client.role_definitions.list(
|
||||
sub_scope, filter='roleName eq \'Reader and Data Access\''):
|
||||
cont_role = role.id
|
||||
break
|
||||
if cont_role is None:
|
||||
raise RuntimeError('Role Id not found for Reader and Data Access')
|
||||
role_assign = auth_client.role_assignments.create(
|
||||
scope=sub_scope,
|
||||
role_assignment_name=uuid.uuid4(),
|
||||
parameters=authmodels.RoleAssignmentCreateParameters(
|
||||
role_definition_id=cont_role,
|
||||
principal_id=vms[0].identity.principal_id
|
||||
),
|
||||
)
|
||||
if settings.verbose(config):
|
||||
logger.debug('reader and data access role assignment: {}'.format(
|
||||
role_assign))
|
||||
# get ip info for vm
|
||||
if util.is_none_or_empty(pips):
|
||||
fqdn = None
|
||||
ipinfo = 'private_ip_address={}'.format(
|
||||
nics[offset].ip_configurations[0].private_ip_address)
|
||||
else:
|
||||
# refresh public ip for vm
|
||||
pip = network_client.public_ip_addresses.get(
|
||||
resource_group_name=fs.resource_group,
|
||||
public_ip_address_name=pips[offset].name,
|
||||
)
|
||||
fqdn = pip.dns_settings.fqdn
|
||||
ipinfo = 'fqdn={} public_ip_address={}'.format(fqdn, pip.ip_address)
|
||||
# install msi vm extension
|
||||
async_ops['vmext'] = {}
|
||||
async_ops['vmext'][0] = resource.AsyncOperation(
|
||||
functools.partial(
|
||||
resource.create_msi_virtual_machine_extension, compute_client, fs,
|
||||
vms[0].name, 0, settings.verbose(config)),
|
||||
max_retries=0,
|
||||
)
|
||||
logger.debug('waiting for virtual machine msi extensions to provision')
|
||||
for offset in async_ops['vmext']:
|
||||
async_ops['vmext'][offset].result()
|
||||
# install vm extension
|
||||
async_ops['vmext'][0] = resource.AsyncOperation(
|
||||
functools.partial(
|
||||
_create_virtual_machine_extension, compute_client, config, fs,
|
||||
bootstrap_file, blob_urls, vms[0].name,
|
||||
private_ips, fqdn, 0, settings.verbose(config)),
|
||||
max_retries=0,
|
||||
)
|
||||
logger.debug('waiting for virtual machine extensions to provision')
|
||||
for offset in async_ops['vmext']:
|
||||
# get vm extension result
|
||||
vm_ext = async_ops['vmext'][offset].result()
|
||||
vm = vms[offset]
|
||||
logger.info(
|
||||
('virtual machine: {} [provisioning_state={}/{} '
|
||||
'vm_size={} {}]').format(
|
||||
vm.id, vm.provisioning_state, vm_ext.provisioning_state,
|
||||
vm.hardware_profile.vm_size, ipinfo))
|
||||
|
||||
|
||||
def delete_federation_proxy(
|
||||
resource_client, compute_client, network_client, blob_client,
|
||||
table_client, queue_client, config, delete_virtual_network=False,
|
||||
delete_resource_group=False, generate_from_prefix=False, wait=False):
|
||||
# type: (azure.mgmt.resource.resources.ResourceManagementClient,
|
||||
# azure.mgmt.compute.ComputeManagementClient,
|
||||
# azure.mgmt.network.NetworkManagementClient,
|
||||
# azure.storage.blob.BlockBlobService,
|
||||
# azure.cosmosdb.table.TableService,
|
||||
# azure.storage.queue.QueueService,
|
||||
# dict, bool, bool, bool, bool) -> None
|
||||
"""Delete a federation proxy
|
||||
:param azure.mgmt.resource.resources.ResourceManagementClient
|
||||
resource_client: resource client
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param azure.storage.blob.BlockBlobService blob_client: blob client
|
||||
:param azure.cosmosdb.table.TableService table_client: table client
|
||||
:param azure.storage.queue.QueueService queue_client: queue client
|
||||
:param dict config: configuration dict
|
||||
:param bool delete_virtual_network: delete vnet
|
||||
:param bool delete_resource_group: delete resource group
|
||||
:param bool generate_from_prefix: generate resources from hostname prefix
|
||||
:param bool wait: wait for completion
|
||||
"""
|
||||
fs = settings.federation_settings(config)
|
||||
# delete rg if specified
|
||||
if delete_resource_group:
|
||||
if util.confirm_action(
|
||||
config, 'delete resource group {}'.format(
|
||||
fs.resource_group)):
|
||||
logger.info('deleting resource group {}'.format(
|
||||
fs.resource_group))
|
||||
async_delete = resource_client.resource_groups.delete(
|
||||
resource_group_name=fs.resource_group)
|
||||
if wait:
|
||||
logger.debug('waiting for resource group {} to delete'.format(
|
||||
fs.resource_group))
|
||||
async_delete.result()
|
||||
logger.info('resource group {} deleted'.format(
|
||||
fs.resource_group))
|
||||
return
|
||||
if not util.confirm_action(config, 'delete federation proxy'):
|
||||
return
|
||||
# get vms and cache for concurent async ops
|
||||
resources = {}
|
||||
i = 0
|
||||
vm_name = settings.generate_virtual_machine_name(fs, i)
|
||||
try:
|
||||
vm = compute_client.virtual_machines.get(
|
||||
resource_group_name=fs.resource_group,
|
||||
vm_name=vm_name,
|
||||
)
|
||||
except msrestazure.azure_exceptions.CloudError as e:
|
||||
if e.status_code == 404:
|
||||
logger.warning('virtual machine {} not found'.format(vm_name))
|
||||
if generate_from_prefix:
|
||||
logger.warning(
|
||||
'OS and data disks for this virtual machine will not '
|
||||
'be deleted, please use "fs disks del" to delete '
|
||||
'those resources if desired')
|
||||
resources[i] = {
|
||||
'vm': settings.generate_virtual_machine_name(fs, i),
|
||||
'as': None,
|
||||
'nic': settings.generate_network_interface_name(fs, i),
|
||||
'pip': settings.generate_public_ip_name(fs, i),
|
||||
'subnet': None,
|
||||
'nsg': settings.generate_network_security_group_name(fs),
|
||||
'vnet': None,
|
||||
'os_disk': None,
|
||||
}
|
||||
else:
|
||||
raise
|
||||
else:
|
||||
# get resources connected to vm
|
||||
nic, pip, subnet, vnet, nsg = \
|
||||
resource.get_resource_names_from_virtual_machine(
|
||||
compute_client, network_client, fs, vm)
|
||||
resources[i] = {
|
||||
'vm': vm.name,
|
||||
'arm_id': vm.id,
|
||||
'id': vm.vm_id,
|
||||
'nic': nic,
|
||||
'pip': pip,
|
||||
'subnet': subnet,
|
||||
'nsg': nsg,
|
||||
'vnet': vnet,
|
||||
'os_disk': vm.storage_profile.os_disk.name,
|
||||
}
|
||||
# unset virtual network if not specified to delete
|
||||
if not delete_virtual_network:
|
||||
resources[i]['subnet'] = None
|
||||
resources[i]['vnet'] = None
|
||||
if len(resources) == 0:
|
||||
logger.warning('no resources deleted')
|
||||
return
|
||||
if settings.verbose(config):
|
||||
logger.debug('deleting the following resources:{}{}'.format(
|
||||
os.linesep, json.dumps(resources, sort_keys=True, indent=4)))
|
||||
# delete storage containers, only delete global blob container
|
||||
storage.delete_storage_containers_nonbatch(
|
||||
blob_client, None, None, 'federation')
|
||||
# create async op holder
|
||||
async_ops = {}
|
||||
# delete vms
|
||||
async_ops['vms'] = {}
|
||||
for key in resources:
|
||||
vm_name = resources[key]['vm']
|
||||
async_ops['vms'][vm_name] = resource.AsyncOperation(functools.partial(
|
||||
resource.delete_virtual_machine, compute_client,
|
||||
fs.resource_group, vm_name), retry_conflict=True)
|
||||
logger.info(
|
||||
'waiting for {} virtual machines to delete'.format(
|
||||
len(async_ops['vms'])))
|
||||
for vm_name in async_ops['vms']:
|
||||
async_ops['vms'][vm_name].result()
|
||||
logger.info('{} virtual machines deleted'.format(len(async_ops['vms'])))
|
||||
# delete nics
|
||||
async_ops['nics'] = {}
|
||||
for key in resources:
|
||||
nic_name = resources[key]['nic']
|
||||
async_ops['nics'][nic_name] = resource.AsyncOperation(
|
||||
functools.partial(
|
||||
resource.delete_network_interface, network_client,
|
||||
fs.resource_group, nic_name),
|
||||
retry_conflict=True
|
||||
)
|
||||
# wait for nics to delete
|
||||
logger.debug('waiting for {} network interfaces to delete'.format(
|
||||
len(async_ops['nics'])))
|
||||
for nic_name in async_ops['nics']:
|
||||
async_ops['nics'][nic_name].result()
|
||||
logger.info('{} network interfaces deleted'.format(len(async_ops['nics'])))
|
||||
# delete os disks
|
||||
async_ops['os_disk'] = []
|
||||
for key in resources:
|
||||
os_disk = resources[key]['os_disk']
|
||||
if util.is_none_or_empty(os_disk):
|
||||
continue
|
||||
async_ops['os_disk'].append(remotefs.delete_managed_disks(
|
||||
resource_client, compute_client, config, os_disk,
|
||||
resource_group=fs.resource_group, wait=False,
|
||||
confirm_override=True))
|
||||
# delete nsg
|
||||
deleted = set()
|
||||
async_ops['nsg'] = {}
|
||||
for key in resources:
|
||||
nsg_name = resources[key]['nsg']
|
||||
if nsg_name in deleted:
|
||||
continue
|
||||
deleted.add(nsg_name)
|
||||
async_ops['nsg'][nsg_name] = resource.AsyncOperation(functools.partial(
|
||||
resource.delete_network_security_group, network_client,
|
||||
fs.resource_group, nsg_name), retry_conflict=True)
|
||||
deleted.clear()
|
||||
# delete public ips
|
||||
async_ops['pips'] = {}
|
||||
for key in resources:
|
||||
pip_name = resources[key]['pip']
|
||||
if util.is_none_or_empty(pip_name):
|
||||
continue
|
||||
async_ops['pips'][pip_name] = resource.AsyncOperation(
|
||||
functools.partial(
|
||||
resource.delete_public_ip, network_client,
|
||||
fs.resource_group, pip_name),
|
||||
retry_conflict=True
|
||||
)
|
||||
logger.debug('waiting for {} public ips to delete'.format(
|
||||
len(async_ops['pips'])))
|
||||
for pip_name in async_ops['pips']:
|
||||
async_ops['pips'][pip_name].result()
|
||||
logger.info('{} public ips deleted'.format(len(async_ops['pips'])))
|
||||
# delete subnets
|
||||
async_ops['subnets'] = {}
|
||||
for key in resources:
|
||||
subnet_name = resources[key]['subnet']
|
||||
vnet_name = resources[key]['vnet']
|
||||
if util.is_none_or_empty(subnet_name) or subnet_name in deleted:
|
||||
continue
|
||||
deleted.add(subnet_name)
|
||||
async_ops['subnets'][subnet_name] = resource.AsyncOperation(
|
||||
functools.partial(
|
||||
resource.delete_subnet, network_client,
|
||||
fs.virtual_network.resource_group, vnet_name, subnet_name),
|
||||
retry_conflict=True
|
||||
)
|
||||
logger.debug('waiting for {} subnets to delete'.format(
|
||||
len(async_ops['subnets'])))
|
||||
for subnet_name in async_ops['subnets']:
|
||||
async_ops['subnets'][subnet_name].result()
|
||||
logger.info('{} subnets deleted'.format(len(async_ops['subnets'])))
|
||||
deleted.clear()
|
||||
# delete vnet
|
||||
async_ops['vnets'] = {}
|
||||
for key in resources:
|
||||
vnet_name = resources[key]['vnet']
|
||||
if util.is_none_or_empty(vnet_name) or vnet_name in deleted:
|
||||
continue
|
||||
deleted.add(vnet_name)
|
||||
async_ops['vnets'][vnet_name] = resource.AsyncOperation(
|
||||
functools.partial(
|
||||
resource.delete_virtual_network, network_client,
|
||||
fs.virtual_network.resource_group, vnet_name),
|
||||
retry_conflict=True
|
||||
)
|
||||
deleted.clear()
|
||||
# delete boot diagnostics storage containers
|
||||
for key in resources:
|
||||
try:
|
||||
vm_name = resources[key]['vm']
|
||||
vm_id = resources[key]['id']
|
||||
except KeyError:
|
||||
pass
|
||||
else:
|
||||
storage.delete_storage_containers_boot_diagnostics(
|
||||
blob_client, vm_name, vm_id)
|
||||
# wait for all async ops to complete
|
||||
if wait:
|
||||
logger.debug('waiting for network security groups to delete')
|
||||
for nsg_name in async_ops['nsg']:
|
||||
async_ops['nsg'][nsg_name].result()
|
||||
logger.info('{} network security groups deleted'.format(
|
||||
len(async_ops['nsg'])))
|
||||
logger.debug('waiting for virtual networks to delete')
|
||||
for vnet_name in async_ops['vnets']:
|
||||
async_ops['vnets'][vnet_name].result()
|
||||
logger.info('{} virtual networks deleted'.format(
|
||||
len(async_ops['vnets'])))
|
||||
logger.debug('waiting for managed os disks to delete')
|
||||
count = 0
|
||||
for os_disk_set in async_ops['os_disk']:
|
||||
for os_disk in os_disk_set:
|
||||
os_disk_set[os_disk].result()
|
||||
count += 1
|
||||
logger.info('{} managed os disks deleted'.format(count))
|
475
convoy/fleet.py
475
convoy/fleet.py
|
@ -47,6 +47,7 @@ from . import autoscale
|
|||
from . import batch
|
||||
from . import crypto
|
||||
from . import data
|
||||
from . import federation
|
||||
from . import keyvault
|
||||
from . import misc
|
||||
from . import monitor
|
||||
|
@ -254,6 +255,17 @@ _CONFIGURABLE_MONITORING_FILES = {
|
|||
'nginx': _MONITORINGNGINXCONF_FILE,
|
||||
'dashboard': _MONITORINGGRAFANADASHBOARD_FILE,
|
||||
}
|
||||
_FEDERATIONPREP_FILE = (
|
||||
'shipyard_federation_bootstrap.sh',
|
||||
pathlib.Path(_ROOT_PATH, 'scripts/shipyard_federation_bootstrap.sh')
|
||||
)
|
||||
_FEDERATIONSERVICES_FILE = (
|
||||
'docker-compose.yml',
|
||||
pathlib.Path(_ROOT_PATH, 'federation/docker-compose.yml')
|
||||
)
|
||||
_ALL_FEDERATION_FILES = [
|
||||
_FEDERATIONPREP_FILE, _FEDERATIONSERVICES_FILE,
|
||||
]
|
||||
|
||||
|
||||
def initialize_globals(verbose):
|
||||
|
@ -274,15 +286,17 @@ def initialize_globals(verbose):
|
|||
_RESOURCES_PATH))
|
||||
|
||||
|
||||
def populate_global_settings(config, fs_storage, pool_id=None):
|
||||
# type: (dict, bool) -> None
|
||||
def populate_global_settings(config, fs_storage, pool_id=None, sc=None):
|
||||
# type: (dict, bool, str, settings.StorageCredentialsSettings) -> None
|
||||
"""Populate global settings from config
|
||||
:param dict config: configuration dict
|
||||
:param bool fs_storage: adjust for fs context
|
||||
:param str pool_id: pool id override
|
||||
:param settings.StorageCredentialsSettings sc: storage creds
|
||||
"""
|
||||
bs = settings.batch_shipyard_settings(config)
|
||||
sc = settings.credentials_storage(config, bs.storage_account_settings)
|
||||
if sc is None:
|
||||
sc = settings.credentials_storage(config, bs.storage_account_settings)
|
||||
if fs_storage:
|
||||
# set postfix to empty for now, it will be populated with the
|
||||
# storage cluster during the actual calls
|
||||
|
@ -1448,7 +1462,7 @@ def _construct_pool_object(
|
|||
)
|
||||
)
|
||||
# upload resource files
|
||||
sas_urls = storage.upload_resource_files(blob_client, config, _rflist)
|
||||
sas_urls = storage.upload_resource_files(blob_client, _rflist)
|
||||
del _rflist
|
||||
# remove temporary az mount files created
|
||||
if azureblob_vd:
|
||||
|
@ -1504,6 +1518,10 @@ def _construct_pool_object(
|
|||
name=settings.get_metadata_version_name(),
|
||||
value=__version__,
|
||||
),
|
||||
batchmodels.MetadataItem(
|
||||
name='BATCH_SHIPYARD_NATIVE_CONTAINER_POOL',
|
||||
value='1' if native else '0',
|
||||
),
|
||||
],
|
||||
task_scheduling_policy=task_scheduling_policy,
|
||||
certificate_references=[]
|
||||
|
@ -1884,8 +1902,7 @@ def _setup_glusterfs(
|
|||
appcmd.append('gluster volume set {} {}'.format(
|
||||
settings.get_gluster_default_volume_name(), vo))
|
||||
# upload script
|
||||
sas_urls = storage.upload_resource_files(
|
||||
blob_client, config, [shell_script])
|
||||
sas_urls = storage.upload_resource_files(blob_client, [shell_script])
|
||||
# get pool current dedicated
|
||||
pool = batch_client.pool.get(pool_id)
|
||||
batchtask = batchmodels.TaskAddParameter(
|
||||
|
@ -3700,8 +3717,8 @@ def action_jobs_add(
|
|||
# add jobs
|
||||
is_windows = settings.is_windows_pool(config)
|
||||
batch.add_jobs(
|
||||
batch_client, blob_client, keyvault_client, config, autopool,
|
||||
_IMAGE_BLOCK_FILE,
|
||||
batch_client, blob_client, None, None, keyvault_client, config,
|
||||
autopool, _IMAGE_BLOCK_FILE,
|
||||
_BLOBXFER_WINDOWS_FILE if is_windows else _BLOBXFER_FILE,
|
||||
recreate, tail)
|
||||
|
||||
|
@ -4337,8 +4354,9 @@ def action_monitor_ssh(
|
|||
"""
|
||||
_check_compute_client(compute_client)
|
||||
_check_network_client(network_client)
|
||||
monitor.ssh_monitoring_resource(
|
||||
compute_client, network_client, config, tty, command)
|
||||
vm_res = settings.monitoring_settings(config)
|
||||
resource.ssh_to_virtual_machine_resource(
|
||||
compute_client, network_client, vm_res, tty, command)
|
||||
|
||||
|
||||
def action_monitor_suspend(compute_client, config, wait):
|
||||
|
@ -4350,7 +4368,9 @@ def action_monitor_suspend(compute_client, config, wait):
|
|||
:param bool wait: wait for suspension to complete
|
||||
"""
|
||||
_check_compute_client(compute_client)
|
||||
monitor.suspend_monitoring_resource(compute_client, config, wait)
|
||||
vm_res = settings.monitoring_settings(config)
|
||||
resource.suspend_virtual_machine_resource(
|
||||
compute_client, config, vm_res, wait=wait)
|
||||
|
||||
|
||||
def action_monitor_start(compute_client, config, wait):
|
||||
|
@ -4362,7 +4382,25 @@ def action_monitor_start(compute_client, config, wait):
|
|||
:param bool wait: wait for restart to complete
|
||||
"""
|
||||
_check_compute_client(compute_client)
|
||||
monitor.start_monitoring_resource(compute_client, config, wait)
|
||||
vm_res = settings.monitoring_settings(config)
|
||||
resource.start_virtual_machine_resource(
|
||||
compute_client, config, vm_res, wait=wait)
|
||||
|
||||
|
||||
def action_monitor_status(compute_client, network_client, config):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient,
|
||||
# azure.mgmt.network.NetworkManagementClient, dict) -> None
|
||||
"""Action: Monitor Status
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param dict config: configuration dict
|
||||
"""
|
||||
_check_compute_client(compute_client)
|
||||
vm_res = settings.monitoring_settings(config)
|
||||
resource.stat_virtual_machine_resource(
|
||||
compute_client, network_client, config, vm_res)
|
||||
|
||||
|
||||
def action_monitor_destroy(
|
||||
|
@ -4402,3 +4440,416 @@ def action_monitor_destroy(
|
|||
table_client, config, delete_virtual_network=delete_virtual_network,
|
||||
delete_resource_group=delete_all_resources,
|
||||
generate_from_prefix=generate_from_prefix, wait=wait)
|
||||
|
||||
|
||||
def action_fed_proxy_create(
|
||||
auth_client, resource_client, compute_client, network_client,
|
||||
blob_client, table_client, queue_client, config):
|
||||
# type: (azure.mgmt.authorization.AuthorizationManagementClient,
|
||||
# azure.mgmt.resource.resources.ResourceManagementClient,
|
||||
# azure.mgmt.compute.ComputeManagementClient,
|
||||
# azure.mgmt.network.NetworkManagementClient,
|
||||
# azure.storage.blob.BlockBlobService,
|
||||
# azure.cosmosdb.table.TableService,
|
||||
# azure.storage.queue.QueueService, dict) -> None
|
||||
"""Action: Fed Proxy Create
|
||||
:param azure.mgmt.authorization.AuthorizationManagementClient auth_client:
|
||||
auth client
|
||||
:param azure.mgmt.resource.resources.ResourceManagementClient
|
||||
resource_client: resource client
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param azure.storage.blob.BlockBlobService blob_client: blob client
|
||||
:param azure.cosmosdb.table.TableService table_client: table client
|
||||
:param azure.storage.queue.QueueService queue_client: queue client
|
||||
:param dict config: configuration dict
|
||||
"""
|
||||
_check_resource_client(resource_client)
|
||||
_check_compute_client(compute_client)
|
||||
_check_network_client(network_client)
|
||||
# ensure aad creds are populated
|
||||
mgmt_aad = settings.credentials_management(config)
|
||||
if (util.is_none_or_empty(mgmt_aad.subscription_id) or
|
||||
util.is_none_or_empty(mgmt_aad.aad.authority_url)):
|
||||
raise ValueError('management aad credentials are invalid')
|
||||
federation.create_federation_proxy(
|
||||
auth_client, resource_client, compute_client, network_client,
|
||||
blob_client, table_client, queue_client, config, _RESOURCES_PATH,
|
||||
_FEDERATIONPREP_FILE, _ALL_FEDERATION_FILES)
|
||||
|
||||
|
||||
def action_fed_proxy_ssh(
|
||||
compute_client, network_client, config, tty, command):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient,
|
||||
# azure.mgmt.network.NetworkManagementClient, dict,
|
||||
# bool, tuple) -> None
|
||||
"""Action: Fed Proxy Ssh
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param dict config: configuration dict
|
||||
:param bool tty: allocate pseudo-tty
|
||||
:param tuple command: command
|
||||
"""
|
||||
_check_compute_client(compute_client)
|
||||
_check_network_client(network_client)
|
||||
vm_res = settings.federation_settings(config)
|
||||
resource.ssh_to_virtual_machine_resource(
|
||||
compute_client, network_client, vm_res, tty, command)
|
||||
|
||||
|
||||
def action_fed_proxy_suspend(compute_client, config, wait):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient, dict, bool) -> None
|
||||
"""Action: Fed Proxy Suspend
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param dict config: configuration dict
|
||||
:param bool wait: wait for suspension to complete
|
||||
"""
|
||||
_check_compute_client(compute_client)
|
||||
vm_res = settings.federation_settings(config)
|
||||
resource.suspend_virtual_machine_resource(
|
||||
compute_client, config, vm_res, wait=wait)
|
||||
|
||||
|
||||
def action_fed_proxy_start(compute_client, config, wait):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient, dict, bool) -> None
|
||||
"""Action: Fed Proxy Start
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param dict config: configuration dict
|
||||
:param bool wait: wait for restart to complete
|
||||
"""
|
||||
_check_compute_client(compute_client)
|
||||
vm_res = settings.federation_settings(config)
|
||||
resource.start_virtual_machine_resource(
|
||||
compute_client, config, vm_res, wait=wait)
|
||||
|
||||
|
||||
def action_fed_proxy_status(compute_client, network_client, config):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient,
|
||||
# azure.mgmt.network.NetworkManagementClient, dict) -> None
|
||||
"""Action: Fed Proxy Status
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param dict config: configuration dict
|
||||
"""
|
||||
_check_compute_client(compute_client)
|
||||
vm_res = settings.federation_settings(config)
|
||||
resource.stat_virtual_machine_resource(
|
||||
compute_client, network_client, config, vm_res)
|
||||
|
||||
|
||||
def action_fed_proxy_destroy(
|
||||
resource_client, compute_client, network_client, blob_client,
|
||||
table_client, queue_client, config, delete_all_resources,
|
||||
delete_virtual_network, generate_from_prefix, wait):
|
||||
# type: (azure.mgmt.resource.resources.ResourceManagementClient,
|
||||
# azure.mgmt.compute.ComputeManagementClient,
|
||||
# azure.mgmt.network.NetworkManagementClient,
|
||||
# azure.storage.blob.BlockBlobService,
|
||||
# azure.cosmosdb.table.TableService,
|
||||
# azure.storage.queue.QueueService, dict, bool, bool,
|
||||
# bool, bool) -> None
|
||||
"""Action: Fed Proxy Destroy
|
||||
:param azure.mgmt.resource.resources.ResourceManagementClient
|
||||
resource_client: resource client
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param azure.storage.blob.BlockBlobService blob_client: blob client
|
||||
:param azure.cosmosdb.table.TableService table_client: table client
|
||||
:param azure.storage.queue.QueueService queue_client: queue client
|
||||
:param dict config: configuration dict
|
||||
:param bool delete_all_resources: delete all resources
|
||||
:param bool delete_virtual_network: delete virtual network
|
||||
:param bool generate_from_prefix: generate resources from hostname prefix
|
||||
:param bool wait: wait for deletion to complete
|
||||
"""
|
||||
_check_resource_client(resource_client)
|
||||
_check_compute_client(compute_client)
|
||||
_check_network_client(network_client)
|
||||
if (generate_from_prefix and
|
||||
(delete_all_resources or delete_virtual_network)):
|
||||
raise ValueError(
|
||||
'Cannot specify generate_from_prefix and a delete_* option')
|
||||
federation.delete_federation_proxy(
|
||||
resource_client, compute_client, network_client, blob_client,
|
||||
table_client, queue_client, config,
|
||||
delete_virtual_network=delete_virtual_network,
|
||||
delete_resource_group=delete_all_resources,
|
||||
generate_from_prefix=generate_from_prefix, wait=wait)
|
||||
|
||||
|
||||
def action_fed_create(
|
||||
blob_client, table_client, queue_client, config, federation_id,
|
||||
force, unique_jobs):
|
||||
# type: (azure.storage.blob.BlockBlobService,
|
||||
# azure.cosmosdb.table.TableService,
|
||||
# azure.storage.queue.QueueService, dict, str, bool, bool) -> None
|
||||
"""Action: Fed Create
|
||||
:param azure.storage.blob.BlockBlobService blob_client: blob client
|
||||
:param azure.cosmosdb.table.TableService table_client: table client
|
||||
:param azure.storage.queue.QueueService queue_client: queue client
|
||||
:param dict config: configuration dict
|
||||
:param str federation_id: federation id
|
||||
:param bool force: force creation
|
||||
:param bool unique_jobs: unique job ids required
|
||||
"""
|
||||
if util.is_none_or_empty(federation_id):
|
||||
raise ValueError('federation id is invalid')
|
||||
logger.info('creating federation id: {}'.format(federation_id))
|
||||
storage.create_federation_id(
|
||||
blob_client, table_client, queue_client, config, federation_id.lower(),
|
||||
force, unique_jobs)
|
||||
|
||||
|
||||
def action_fed_list(
|
||||
table_client, config, federation_id):
|
||||
# type: (azure.cosmosdb.table.TableService, dict, List[str]) -> None
|
||||
"""Action: Fed List
|
||||
:param azure.cosmosdb.table.TableService table_client: table client
|
||||
:param dict config: configuration dict
|
||||
:param List[str] federation_id: federation ids
|
||||
"""
|
||||
storage.list_federations(table_client, config, federation_id)
|
||||
|
||||
|
||||
def action_fed_destroy(
|
||||
blob_client, table_client, queue_client, config, federation_id):
|
||||
# type: (azure.storage.blob.BlockBlobService,
|
||||
# azure.cosmosdb.table.TableService,
|
||||
# azure.storage.queue.QueueService, dict, str) -> None
|
||||
"""Action: Fed Destroy
|
||||
:param azure.storage.blob.BlockBlobService blob_client: blob client
|
||||
:param azure.cosmosdb.table.TableService table_client: table client
|
||||
:param azure.storage.queue.QueueService queue_client: queue client
|
||||
:param dict config: configuration dict
|
||||
:param str federation_id: federation id
|
||||
"""
|
||||
if util.is_none_or_empty(federation_id):
|
||||
raise ValueError('federation id is invalid')
|
||||
if not util.confirm_action(
|
||||
config,
|
||||
msg='destroy federation id {}, all queued jobs for the '
|
||||
'federation will be deleted'.format(federation_id)):
|
||||
return
|
||||
logger.info('destroying federation id: {}'.format(federation_id))
|
||||
storage.destroy_federation_id(
|
||||
blob_client, table_client, queue_client, config, federation_id.lower())
|
||||
|
||||
|
||||
def action_fed_pool_add(
|
||||
batch_client, table_client, config, federation_id, batch_service_url,
|
||||
pools):
|
||||
# type: (batchsc.BatchServiceClient,
|
||||
# azure.cosmosdb.table.TableService, dict, str, str,
|
||||
# List[str]) -> None
|
||||
"""Action: Fed Pool Add
|
||||
:param azure.batch.batch_service_client.BatchServiceClient batch_client:
|
||||
batch client
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param azure.cosmosdb.table.TableService table_client: table client
|
||||
:param dict config: configuration dict
|
||||
:param str federation_id: federation id
|
||||
:param str batch_service_url: Batch service url to use instead
|
||||
:param list pools: list of pool ids to add to federation
|
||||
"""
|
||||
if util.is_none_or_empty(federation_id):
|
||||
raise ValueError('federation id is invalid')
|
||||
# ensure that we are operating in AAD mode for batch
|
||||
if batch_client is not None:
|
||||
bc = settings.credentials_batch(config)
|
||||
_check_for_batch_aad(bc, 'add pool(s) to federation')
|
||||
if util.is_none_or_empty(pools):
|
||||
pools = [settings.pool_id(config)]
|
||||
# check for pool existence
|
||||
for poolid in pools:
|
||||
try:
|
||||
batch_client.pool.get(poolid)
|
||||
except batchmodels.batch_error.BatchErrorException as ex:
|
||||
if 'The specified pool does not exist' in ex.message.value:
|
||||
raise ValueError(
|
||||
'pool {} does not exist for account {}'.format(
|
||||
bc.account_service_url))
|
||||
else:
|
||||
raise
|
||||
else:
|
||||
if util.is_none_or_empty(pools):
|
||||
logger.error('no pools specified to add to federation')
|
||||
return
|
||||
storage.add_pool_to_federation(
|
||||
table_client, config, federation_id.lower(), batch_service_url, pools)
|
||||
|
||||
|
||||
def action_fed_pool_remove(
|
||||
batch_client, table_client, config, federation_id, all,
|
||||
batch_service_url, pools):
|
||||
# type: (batchsc.BatchServiceClient,
|
||||
# azure.cosmosdb.table.TableService, dict, str, bool, str,
|
||||
# List[str]) -> None
|
||||
"""Action: Fed Pool Remove
|
||||
:param azure.batch.batch_service_client.BatchServiceClient batch_client:
|
||||
batch client
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param azure.cosmosdb.table.TableService table_client: table client
|
||||
:param dict config: configuration dict
|
||||
:param str federation_id: federation id
|
||||
:param bool all: all pools
|
||||
:param str batch_service_url: Batch service url to use instead
|
||||
:param list pools: list of pool ids to add to federation
|
||||
"""
|
||||
if util.is_none_or_empty(federation_id):
|
||||
raise ValueError('federation id is invalid')
|
||||
# ensure that we are operating in AAD mode for batch
|
||||
if batch_client is not None:
|
||||
bc = settings.credentials_batch(config)
|
||||
_check_for_batch_aad(bc, 'add pool(s) to federation')
|
||||
if util.is_none_or_empty(pools):
|
||||
pools = [settings.pool_id(config)]
|
||||
if util.is_none_or_empty(pools) and not all:
|
||||
logger.error('no pools specified to remove from federation')
|
||||
return
|
||||
elif util.is_not_empty(pools) and all:
|
||||
raise ValueError('cannot specify both --all and --poolid')
|
||||
storage.remove_pool_from_federation(
|
||||
table_client, config, federation_id.lower(), all, batch_service_url,
|
||||
pools)
|
||||
|
||||
|
||||
def action_fed_jobs_add(
|
||||
batch_client, keyvault_client, blob_client, table_client,
|
||||
queue_client, config, federation_id):
|
||||
# type: (azure.storage.blob.BlockBlobService,
|
||||
# azure.cosmosdb.table.TableService,
|
||||
# azure.storage.queue.QueueService, dict, str) -> None
|
||||
"""Action: Fed Jobs Add
|
||||
:param azure.storage.blob.BlockBlobService blob_client: blob client
|
||||
:param azure.cosmosdb.table.TableService table_client: table client
|
||||
:param azure.storage.queue.QueueService queue_client: queue client
|
||||
:param dict config: configuration dict
|
||||
:param str federation_id: federation id
|
||||
"""
|
||||
if util.is_none_or_empty(federation_id):
|
||||
raise ValueError('federation id is invalid')
|
||||
is_windows = settings.is_windows_pool(config)
|
||||
batch.add_jobs(
|
||||
batch_client, blob_client, table_client, queue_client, keyvault_client,
|
||||
config, None, _IMAGE_BLOCK_FILE,
|
||||
_BLOBXFER_WINDOWS_FILE if is_windows else _BLOBXFER_FILE,
|
||||
recreate=False, tail=None, federation_id=federation_id)
|
||||
|
||||
|
||||
def action_fed_jobs_list(
|
||||
table_client, config, federation_id, jobid, jobscheduleid):
|
||||
# type: (azure.cosmosdb.table.TableService,
|
||||
# dict, str, str, str) -> None
|
||||
"""Action: Fed Jobs List
|
||||
:param azure.storage.blob.BlockBlobService blob_client: blob client
|
||||
:param azure.cosmosdb.table.TableService table_client: table client
|
||||
:param azure.storage.queue.QueueService queue_client: queue client
|
||||
:param dict config: configuration dict
|
||||
:param str federation_id: federation id
|
||||
:param str jobid: job id
|
||||
:param str jobscheduleid: job schedule id
|
||||
"""
|
||||
if jobid is not None and jobscheduleid is not None:
|
||||
raise ValueError('cannot specify both --jobid and --jobscheduleid')
|
||||
storage.list_jobs_in_federation(
|
||||
table_client, config, federation_id, jobid, jobscheduleid)
|
||||
|
||||
|
||||
def action_fed_jobs_del_or_term(
|
||||
blob_client, table_client, queue_client, config, delete, federation_id,
|
||||
jobid, jobscheduleid, all_jobs, all_jobschedules, force):
|
||||
# type: (azure.storage.blob.BlockBlobService,
|
||||
# azure.cosmosdb.table.TableService,
|
||||
# azure.storage.queue.QueueService, dict, bool, str, str,
|
||||
# bool, bool, bool) -> None
|
||||
"""Action: Fed Jobs Del or Term
|
||||
:param azure.storage.blob.BlockBlobService blob_client: blob client
|
||||
:param azure.cosmosdb.table.TableService table_client: table client
|
||||
:param azure.storage.queue.QueueService queue_client: queue client
|
||||
:param dict config: configuration dict
|
||||
:param bool delete: delete instead of terminate
|
||||
:param str federation_id: federation id
|
||||
:param str jobid: job id
|
||||
:param str jobscheduleid: job schedule id
|
||||
:param bool all_jobs all jobs
|
||||
:param bool all_jobschedules: all job schedules
|
||||
:param bool force: force
|
||||
"""
|
||||
if all_jobs and all_jobschedules:
|
||||
raise ValueError(
|
||||
'cannot specify both --all-jobs and --alljobschedules')
|
||||
elif all_jobs:
|
||||
if util.is_not_empty(jobid) or util.is_not_empty(jobscheduleid):
|
||||
raise ValueError(
|
||||
'cannot specify both --all-jobs and --jobid or '
|
||||
'--jobscheduleid')
|
||||
elif all_jobschedules:
|
||||
if util.is_not_empty(jobid) or util.is_not_empty(jobscheduleid):
|
||||
raise ValueError(
|
||||
'cannot specify both --all-jobschedules and --jobscheduleid '
|
||||
'or --jobid')
|
||||
else:
|
||||
if util.is_not_empty(jobid) and util.is_not_empty(jobscheduleid):
|
||||
raise ValueError('cannot specify both --jobid and --jobscheduleid')
|
||||
elif (util.is_none_or_empty(jobid) and
|
||||
util.is_none_or_empty(jobscheduleid)):
|
||||
_jobs = settings.job_specifications(config)
|
||||
js = []
|
||||
jobs = []
|
||||
for job in _jobs:
|
||||
if settings.job_recurrence(job):
|
||||
js.append(settings.job_id(job))
|
||||
else:
|
||||
jobs.append(settings.job_id(job))
|
||||
del _jobs
|
||||
if util.is_not_empty(js):
|
||||
storage.delete_or_terminate_job_from_federation(
|
||||
blob_client, table_client, queue_client, config, delete,
|
||||
federation_id, None, js, False, False, force)
|
||||
del js
|
||||
if util.is_not_empty(jobs):
|
||||
storage.delete_or_terminate_job_from_federation(
|
||||
blob_client, table_client, queue_client, config, delete,
|
||||
federation_id, jobs, None, False, False, force)
|
||||
return
|
||||
storage.delete_or_terminate_job_from_federation(
|
||||
blob_client, table_client, queue_client, config, delete, federation_id,
|
||||
jobid, jobscheduleid, all_jobs, all_jobschedules, force)
|
||||
|
||||
|
||||
def action_fed_jobs_zap(blob_client, config, federation_id, unique_id):
|
||||
# type: (azure.storage.blob.BlockBlobService,
|
||||
# dict, str, str) -> None
|
||||
"""Action: Fed Jobs Zap
|
||||
:param azure.storage.blob.BlockBlobService blob_client: blob client
|
||||
:param dict config: configuration dict
|
||||
:param str federation_id: federation id
|
||||
:param str unique_id: unique id
|
||||
"""
|
||||
if util.is_none_or_empty(federation_id):
|
||||
raise ValueError('federation id is invalid')
|
||||
if util.is_none_or_empty(unique_id):
|
||||
raise ValueError('unique id is invalid')
|
||||
if not util.confirm_action(
|
||||
config,
|
||||
msg='zap unique id {} in federation {}, this may result in '
|
||||
'orphaned jobs and/or data'.format(unique_id, federation_id)):
|
||||
return
|
||||
storage.zap_unique_id_from_federation(
|
||||
blob_client, config, federation_id, unique_id)
|
||||
|
|
|
@ -34,10 +34,6 @@ import functools
|
|||
import logging
|
||||
import json
|
||||
import os
|
||||
try:
|
||||
import pathlib2 as pathlib
|
||||
except ImportError:
|
||||
import pathlib
|
||||
import uuid
|
||||
# non-stdlib imports
|
||||
import azure.mgmt.authorization.models as authmodels
|
||||
|
@ -188,7 +184,7 @@ def create_monitoring_resource(
|
|||
'lets encrypt enabled')
|
||||
# create storage container
|
||||
storage.create_storage_containers_nonbatch(
|
||||
blob_client, table_client, 'monitoring')
|
||||
blob_client, table_client, None, 'monitoring')
|
||||
# configure yaml files and write to resources
|
||||
if servconf.lets_encrypt_enabled and ms.public_ip.enabled:
|
||||
with monitoring_files['compose'][1].open('r') as f:
|
||||
|
@ -576,7 +572,7 @@ def delete_monitoring_resource(
|
|||
os.linesep, json.dumps(resources, sort_keys=True, indent=4)))
|
||||
# delete storage container
|
||||
storage.delete_storage_containers_nonbatch(
|
||||
blob_client, table_client, 'monitoring')
|
||||
blob_client, table_client, None, 'monitoring')
|
||||
# create async op holder
|
||||
async_ops = {}
|
||||
# delete vms
|
||||
|
@ -710,147 +706,3 @@ def delete_monitoring_resource(
|
|||
os_disk_set[os_disk].result()
|
||||
count += 1
|
||||
logger.info('{} managed os disks deleted'.format(count))
|
||||
|
||||
|
||||
def _get_ssh_info(
|
||||
compute_client, network_client, config, nic=None, pip=None):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient,
|
||||
# azure.mgmt.network.NetworkManagementClient, dict,
|
||||
# networkmodes.NetworkInterface, networkmodels.PublicIPAddress) ->
|
||||
# Tuple[pathlib.Path, int, str, str]
|
||||
"""Get SSH info to a resource monitor
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param dict config: configuration dict
|
||||
:param networkmodels.NetworkInterface nic: network interface
|
||||
:param networkmodels.PublicIPAddress pip: public ip
|
||||
:rtype: tuple
|
||||
:return (ssh private key, port, username, ip)
|
||||
"""
|
||||
ms = settings.monitoring_settings(config)
|
||||
vm_name = settings.generate_virtual_machine_name(ms, 0)
|
||||
try:
|
||||
vm = compute_client.virtual_machines.get(
|
||||
resource_group_name=ms.resource_group,
|
||||
vm_name=vm_name,
|
||||
)
|
||||
except msrestazure.azure_exceptions.CloudError as e:
|
||||
if e.status_code == 404:
|
||||
raise RuntimeError('virtual machine {} not found'.format(vm_name))
|
||||
else:
|
||||
raise
|
||||
# get connection ip
|
||||
if ms.public_ip.enabled:
|
||||
# get pip connected to vm
|
||||
if pip is None:
|
||||
_, pip = resource.get_nic_and_pip_from_virtual_machine(
|
||||
network_client, ms.resource_group, vm)
|
||||
ip_address = pip.ip_address
|
||||
else:
|
||||
if nic is None:
|
||||
nic, _ = resource.get_nic_and_pip_from_virtual_machine(
|
||||
network_client, ms.resource_group, vm)
|
||||
ip_address = nic.ip_configurations[0].private_ip_address
|
||||
# return connection info for vm
|
||||
if ms.ssh.ssh_private_key is not None:
|
||||
ssh_priv_key = ms.ssh.ssh_private_key
|
||||
else:
|
||||
ssh_priv_key = pathlib.Path(
|
||||
ms.ssh.generated_file_export_path,
|
||||
crypto.get_monitoring_ssh_key_prefix())
|
||||
if not ssh_priv_key.exists():
|
||||
raise RuntimeError('SSH private key file not found at: {}'.format(
|
||||
ssh_priv_key))
|
||||
return ssh_priv_key, 22, vm.os_profile.admin_username, ip_address
|
||||
|
||||
|
||||
def ssh_monitoring_resource(
|
||||
compute_client, network_client, config, tty, command):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient,
|
||||
# azure.mgmt.network.NetworkManagementClient, dict,
|
||||
# bool, tuple) -> None
|
||||
"""SSH to a node in resource monitor
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param dict config: configuration dict
|
||||
:param bool tty: allocate pseudo-tty
|
||||
:param tuple command: command to execute
|
||||
"""
|
||||
ssh_priv_key, port, username, ip = _get_ssh_info(
|
||||
compute_client, network_client, config)
|
||||
crypto.connect_or_exec_ssh_command(
|
||||
ip, port, ssh_priv_key, username, tty=tty, command=command)
|
||||
|
||||
|
||||
def suspend_monitoring_resource(compute_client, config, wait=False):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient, dict, bool) -> None
|
||||
"""Suspend a monitoring resource
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param dict config: configuration dict
|
||||
:param bool wait: wait for suspension to complete
|
||||
"""
|
||||
ms = settings.monitoring_settings(config)
|
||||
vm_name = settings.generate_virtual_machine_name(ms, 0)
|
||||
try:
|
||||
vm = compute_client.virtual_machines.get(
|
||||
resource_group_name=ms.resource_group,
|
||||
vm_name=vm_name,
|
||||
)
|
||||
except msrestazure.azure_exceptions.CloudError as e:
|
||||
if e.status_code == 404:
|
||||
logger.error('virtual machine {} not found'.format(vm_name))
|
||||
return
|
||||
else:
|
||||
raise
|
||||
if not util.confirm_action(
|
||||
config, 'suspend monitoring resource {}'.format(vm.name)):
|
||||
return
|
||||
# deallocate vm
|
||||
async_op = resource.AsyncOperation(functools.partial(
|
||||
resource.deallocate_virtual_machine, compute_client,
|
||||
ms.resource_group, vm.name), retry_conflict=True)
|
||||
if wait:
|
||||
logger.info('waiting for virtual machine {} to deallocate'.format(
|
||||
vm.name))
|
||||
async_op.result()
|
||||
logger.info('virtual machine {} deallocated'.format(vm.name))
|
||||
|
||||
|
||||
def start_monitoring_resource(compute_client, config, wait=False):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient, dict, bool) -> None
|
||||
"""Starts a suspended monitoring resource
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param dict config: configuration dict
|
||||
:param bool wait: wait for restart to complete
|
||||
"""
|
||||
ms = settings.monitoring_settings(config)
|
||||
vm_name = settings.generate_virtual_machine_name(ms, 0)
|
||||
try:
|
||||
vm = compute_client.virtual_machines.get(
|
||||
resource_group_name=ms.resource_group,
|
||||
vm_name=vm_name,
|
||||
)
|
||||
except msrestazure.azure_exceptions.CloudError as e:
|
||||
if e.status_code == 404:
|
||||
logger.error('virtual machine {} not found'.format(vm_name))
|
||||
return
|
||||
else:
|
||||
raise
|
||||
if not util.confirm_action(
|
||||
config, 'start suspended monitoring resource {}'.format(vm.name)):
|
||||
return
|
||||
# start vm
|
||||
async_op = resource.AsyncOperation(functools.partial(
|
||||
resource.start_virtual_machine, compute_client,
|
||||
ms.resource_group, vm.name))
|
||||
if wait:
|
||||
logger.info('waiting for virtual machine {} to start'.format(
|
||||
vm.name))
|
||||
async_op.result()
|
||||
logger.info('virtual machine {} started'.format(vm.name))
|
||||
|
|
|
@ -542,7 +542,8 @@ def create_storage_cluster(
|
|||
resource_client, rfs.storage_cluster.resource_group,
|
||||
rfs.storage_cluster.location)
|
||||
# create storage container
|
||||
storage.create_storage_containers_nonbatch(blob_client, None, 'remotefs')
|
||||
storage.create_storage_containers_nonbatch(
|
||||
blob_client, None, None, 'remotefs')
|
||||
# upload scripts to blob storage for customscript vm extension
|
||||
blob_urls = storage.upload_for_nonbatch(
|
||||
blob_client, remotefs_files, 'remotefs')
|
||||
|
@ -791,7 +792,8 @@ def resize_storage_cluster(
|
|||
config, 'resize storage cluster {}'.format(sc_id)):
|
||||
return False
|
||||
# re-create storage container in case it got deleted
|
||||
storage.create_storage_containers_nonbatch(blob_client, None, 'remotefs')
|
||||
storage.create_storage_containers_nonbatch(
|
||||
blob_client, None, None, 'remotefs')
|
||||
# upload scripts to blob storage for customscript vm extension
|
||||
blob_urls = storage.upload_for_nonbatch(
|
||||
blob_client, remotefs_files, 'remotefs')
|
||||
|
@ -1368,7 +1370,8 @@ def delete_storage_cluster(
|
|||
logger.debug('deleting the following resources:{}{}'.format(
|
||||
os.linesep, json.dumps(resources, sort_keys=True, indent=4)))
|
||||
# delete storage container
|
||||
storage.delete_storage_containers_nonbatch(blob_client, None, 'remotefs')
|
||||
storage.delete_storage_containers_nonbatch(
|
||||
blob_client, None, None, 'remotefs')
|
||||
# create async op holder
|
||||
async_ops = {}
|
||||
# delete vms
|
||||
|
|
|
@ -31,7 +31,13 @@ from builtins import ( # noqa
|
|||
next, oct, open, pow, round, super, filter, map, zip)
|
||||
# stdlib imports
|
||||
import functools
|
||||
import json
|
||||
import logging
|
||||
import os
|
||||
try:
|
||||
import pathlib2 as pathlib
|
||||
except ImportError:
|
||||
import pathlib
|
||||
import random
|
||||
import time
|
||||
# non-stdlib imports
|
||||
|
@ -40,6 +46,7 @@ import azure.mgmt.resource.resources.models as rgmodels
|
|||
import msrest.exceptions
|
||||
import msrestazure.azure_exceptions
|
||||
# local imports
|
||||
from . import crypto
|
||||
from . import settings
|
||||
from . import storage
|
||||
from . import util
|
||||
|
@ -934,6 +941,116 @@ def deallocate_virtual_machine(compute_client, rg_name, vm_name):
|
|||
)
|
||||
|
||||
|
||||
def get_ssh_info(compute_client, network_client, vm_res, nic=None, pip=None):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient,
|
||||
# azure.mgmt.network.NetworkManagementClient,
|
||||
# settings.VmResource, networkmodes.NetworkInterface,
|
||||
# networkmodels.PublicIPAddress) ->
|
||||
# Tuple[pathlib.Path, int, str, str]
|
||||
"""Get SSH info to a federation proxy
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param settings.VmResource vm_res: resource
|
||||
:param networkmodels.NetworkInterface nic: network interface
|
||||
:param networkmodels.PublicIPAddress pip: public ip
|
||||
:rtype: tuple
|
||||
:return (ssh private key, port, username, ip)
|
||||
"""
|
||||
vm_name = settings.generate_virtual_machine_name(vm_res, 0)
|
||||
try:
|
||||
vm = compute_client.virtual_machines.get(
|
||||
resource_group_name=vm_res.resource_group,
|
||||
vm_name=vm_name,
|
||||
)
|
||||
except msrestazure.azure_exceptions.CloudError as e:
|
||||
if e.status_code == 404:
|
||||
raise RuntimeError('virtual machine {} not found'.format(vm_name))
|
||||
else:
|
||||
raise
|
||||
# get connection ip
|
||||
if vm_res.public_ip.enabled:
|
||||
# get pip connected to vm
|
||||
if pip is None:
|
||||
_, pip = get_nic_and_pip_from_virtual_machine(
|
||||
network_client, vm_res.resource_group, vm)
|
||||
ip_address = pip.ip_address
|
||||
else:
|
||||
if nic is None:
|
||||
nic, _ = get_nic_and_pip_from_virtual_machine(
|
||||
network_client, vm_res.resource_group, vm)
|
||||
ip_address = nic.ip_configurations[0].private_ip_address
|
||||
# return connection info for vm
|
||||
if vm_res.ssh.ssh_private_key is not None:
|
||||
ssh_priv_key = vm_res.ssh.ssh_private_key
|
||||
else:
|
||||
ssh_priv_key = pathlib.Path(
|
||||
vm_res.ssh.generated_file_export_path,
|
||||
crypto.get_federation_ssh_key_prefix())
|
||||
if not ssh_priv_key.exists():
|
||||
raise RuntimeError('SSH private key file not found at: {}'.format(
|
||||
ssh_priv_key))
|
||||
return ssh_priv_key, 22, vm.os_profile.admin_username, ip_address
|
||||
|
||||
|
||||
def ssh_to_virtual_machine_resource(
|
||||
compute_client, network_client, vm_res, tty, command):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient,
|
||||
# azure.mgmt.network.NetworkManagementClient,
|
||||
# settings.VmResource, bool, tuple) -> None
|
||||
"""SSH to a node in federation proxy
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param settings.VmResource vm_res: resource
|
||||
:param bool tty: allocate pseudo-tty
|
||||
:param tuple command: command to execute
|
||||
"""
|
||||
ssh_priv_key, port, username, ip = get_ssh_info(
|
||||
compute_client, network_client, vm_res)
|
||||
crypto.connect_or_exec_ssh_command(
|
||||
ip, port, ssh_priv_key, username, tty=tty, command=command)
|
||||
|
||||
|
||||
def suspend_virtual_machine_resource(
|
||||
compute_client, config, vm_res, wait=False):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient, dict,
|
||||
# settings.VmResource, bool) -> None
|
||||
"""Suspend a monitoring resource
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param dict config: configuration dict
|
||||
:param settings.VmResource vm_res: resource
|
||||
:param bool wait: wait for suspension to complete
|
||||
"""
|
||||
vm_name = settings.generate_virtual_machine_name(vm_res, 0)
|
||||
try:
|
||||
vm = compute_client.virtual_machines.get(
|
||||
resource_group_name=vm_res.resource_group,
|
||||
vm_name=vm_name,
|
||||
)
|
||||
except msrestazure.azure_exceptions.CloudError as e:
|
||||
if e.status_code == 404:
|
||||
logger.error('virtual machine {} not found'.format(vm_name))
|
||||
return
|
||||
else:
|
||||
raise
|
||||
if not util.confirm_action(
|
||||
config, 'suspend virtual machine resource {}'.format(vm.name)):
|
||||
return
|
||||
# deallocate vm
|
||||
async_op = AsyncOperation(functools.partial(
|
||||
deallocate_virtual_machine, compute_client,
|
||||
vm_res.resource_group, vm.name), retry_conflict=True)
|
||||
if wait:
|
||||
logger.info('waiting for virtual machine {} to deallocate'.format(
|
||||
vm.name))
|
||||
async_op.result()
|
||||
logger.info('virtual machine {} deallocated'.format(vm.name))
|
||||
|
||||
|
||||
def start_virtual_machine(compute_client, rg_name, vm_name):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient, str, str) ->
|
||||
# msrestazure.azure_operation.AzureOperationPoller
|
||||
|
@ -950,3 +1067,131 @@ def start_virtual_machine(compute_client, rg_name, vm_name):
|
|||
resource_group_name=rg_name,
|
||||
vm_name=vm_name,
|
||||
)
|
||||
|
||||
|
||||
def start_virtual_machine_resource(
|
||||
compute_client, config, vm_res, wait=False):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient, dict,
|
||||
# settings.VmResource, bool) -> None
|
||||
"""Starts a suspended virtual machine resource
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param dict config: configuration dict
|
||||
:param settings.VmResource vm_res: resource
|
||||
:param bool wait: wait for restart to complete
|
||||
"""
|
||||
vm_name = settings.generate_virtual_machine_name(vm_res, 0)
|
||||
try:
|
||||
vm = compute_client.virtual_machines.get(
|
||||
resource_group_name=vm_res.resource_group,
|
||||
vm_name=vm_name,
|
||||
)
|
||||
except msrestazure.azure_exceptions.CloudError as e:
|
||||
if e.status_code == 404:
|
||||
logger.error('virtual machine {} not found'.format(vm_name))
|
||||
return
|
||||
else:
|
||||
raise
|
||||
if not util.confirm_action(
|
||||
config, 'start suspended virtual machine resource {}'.format(
|
||||
vm.name)):
|
||||
return
|
||||
# start vm
|
||||
async_op = AsyncOperation(functools.partial(
|
||||
start_virtual_machine, compute_client,
|
||||
vm_res.resource_group, vm.name))
|
||||
if wait:
|
||||
logger.info('waiting for virtual machine {} to start'.format(
|
||||
vm.name))
|
||||
async_op.result()
|
||||
logger.info('virtual machine {} started'.format(vm.name))
|
||||
|
||||
|
||||
def stat_virtual_machine_resource(
|
||||
compute_client, network_client, config, vm_res):
|
||||
# type: (azure.mgmt.compute.ComputeManagementClient,
|
||||
# azure.mgmt.network.NetworkManagementClient, dict,
|
||||
# settings.VmResource) -> None
|
||||
"""Retrieve status of a virtual machine resource
|
||||
:param azure.mgmt.compute.ComputeManagementClient compute_client:
|
||||
compute client
|
||||
:param azure.mgmt.network.NetworkManagementClient network_client:
|
||||
network client
|
||||
:param dict config: configuration dict
|
||||
:param settings.VmResource vm_res: resource
|
||||
"""
|
||||
# retrieve all vms
|
||||
vm_name = settings.generate_virtual_machine_name(vm_res, 0)
|
||||
try:
|
||||
vm = compute_client.virtual_machines.get(
|
||||
resource_group_name=vm_res.resource_group,
|
||||
vm_name=vm_name,
|
||||
expand=compute_client.virtual_machines.models.
|
||||
InstanceViewTypes.instance_view,
|
||||
)
|
||||
except msrestazure.azure_exceptions.CloudError as e:
|
||||
if e.status_code == 404:
|
||||
logger.error('virtual machine {} not found'.format(vm_name))
|
||||
return
|
||||
else:
|
||||
raise
|
||||
# fetch vm status
|
||||
vmstatus = {}
|
||||
powerstate = None
|
||||
for status in vm.instance_view.statuses:
|
||||
if status.code.startswith('PowerState'):
|
||||
powerstate = status.code
|
||||
diskstates = []
|
||||
if util.is_not_empty(vm.instance_view.disks):
|
||||
for disk in vm.instance_view.disks:
|
||||
for status in disk.statuses:
|
||||
diskstates.append(status.code)
|
||||
# get nic/pip connected to vm
|
||||
nic, pip = get_nic_and_pip_from_virtual_machine(
|
||||
network_client, vm_res.resource_group, vm)
|
||||
# get resource names (pass cached data to prevent another lookup)
|
||||
_, _, subnet, vnet, nsg = \
|
||||
get_resource_names_from_virtual_machine(
|
||||
compute_client, network_client, vm_res, vm, nic=nic, pip=pip)
|
||||
# stat data disks
|
||||
disks = {}
|
||||
total_size_gb = 0
|
||||
for dd in vm.storage_profile.data_disks:
|
||||
total_size_gb += dd.disk_size_gb
|
||||
disks[dd.name] = {
|
||||
'lun': dd.lun,
|
||||
'caching': str(dd.caching),
|
||||
'disk_size_gb': dd.disk_size_gb,
|
||||
'type': str(dd.managed_disk.storage_account_type),
|
||||
}
|
||||
disks['disk_array_size_gb'] = total_size_gb
|
||||
vmstatus[vm.name] = {
|
||||
'vm_size': vm.hardware_profile.vm_size,
|
||||
'powerstate': powerstate,
|
||||
'provisioning_state': vm.provisioning_state,
|
||||
'availability_set':
|
||||
vm.availability_set.id.split('/')[-1]
|
||||
if vm.availability_set is not None else None,
|
||||
'update_domain/fault_domain': '{}/{}'.format(
|
||||
vm.instance_view.platform_update_domain,
|
||||
vm.instance_view.platform_fault_domain),
|
||||
'fqdn': pip.dns_settings.fqdn if pip is not None else None,
|
||||
'public_ip_address': pip.ip_address if pip is not None else None,
|
||||
'public_ip_allocation':
|
||||
pip.public_ip_allocation_method if pip is not None else None,
|
||||
'private_ip_address': nic.ip_configurations[0].private_ip_address,
|
||||
'private_ip_allocation':
|
||||
nic.ip_configurations[0].private_ip_allocation_method,
|
||||
'admin_username': vm.os_profile.admin_username,
|
||||
'accelerated_networking': nic.enable_accelerated_networking,
|
||||
'virtual_network': vnet,
|
||||
'subnet': subnet,
|
||||
'network_security_group': nsg,
|
||||
'data_disks': disks,
|
||||
}
|
||||
log = '{}'.format(json.dumps(vmstatus, sort_keys=True, indent=4))
|
||||
if settings.raw(config):
|
||||
print(log)
|
||||
else:
|
||||
logger.info('virtual machine resource status:{}{}'.format(
|
||||
os.linesep, log))
|
||||
|
|
|
@ -78,16 +78,9 @@ _GPU_INSTANCES = _GPU_COMPUTE_INSTANCES.union(_GPU_VISUALIZATION_INSTANCES)
|
|||
_RDMA_INSTANCES = frozenset((
|
||||
# standard_a
|
||||
'standard_a8', 'standard_a9',
|
||||
# standard_h
|
||||
'standard_h16r', 'standard_h16mr',
|
||||
# standard_nc
|
||||
'standard_nc24r',
|
||||
# standard_nc_v2
|
||||
'standard_nc24rs_v2',
|
||||
# standard_nc_v3
|
||||
'standard_nc24rs_v3',
|
||||
# standard_nd
|
||||
'standard_nd24rs',
|
||||
))
|
||||
_RDMA_INSTANCE_SUFFIXES = frozenset((
|
||||
'r', 'rs', 'rs_v2', 'rs_v3',
|
||||
))
|
||||
_PREMIUM_STORAGE_INSTANCE_PREFIXES = frozenset((
|
||||
'standard_ds', 'standard_gs',
|
||||
|
@ -334,6 +327,32 @@ ResourceFileSettings = collections.namedtuple(
|
|||
'file_path', 'blob_source', 'file_mode',
|
||||
]
|
||||
)
|
||||
CustomMountFstabSettings = collections.namedtuple(
|
||||
'CustomMountFstabSettings', [
|
||||
'fs_spec', 'fs_vfstype', 'fs_mntops', 'fs_freq', 'fs_passno',
|
||||
]
|
||||
)
|
||||
FederationPoolConstraintSettings = collections.namedtuple(
|
||||
'FederationPoolConstraintSettings', [
|
||||
'native', 'windows', 'location', 'custom_image_arm_id',
|
||||
'virtual_network_arm_id', 'low_priority_nodes_allow',
|
||||
'low_priority_nodes_exclusive', 'autoscale_allow',
|
||||
'autoscale_exclusive', 'container_registries_private_docker_hub',
|
||||
'container_registries_public', 'max_active_task_backlog_ratio',
|
||||
'max_active_task_backlog_autoscale_exempt',
|
||||
]
|
||||
)
|
||||
FederationComputeNodeConstraintSettings = collections.namedtuple(
|
||||
'FederationComputeNodeConstraintSettings', [
|
||||
'vm_size', 'cores', 'core_variance', 'memory', 'memory_variance',
|
||||
'exclusive', 'gpu', 'infiniband',
|
||||
]
|
||||
)
|
||||
FederationConstraintSettings = collections.namedtuple(
|
||||
'FederationConstraintSettings', [
|
||||
'pool', 'compute_node',
|
||||
]
|
||||
)
|
||||
ManagedDisksSettings = collections.namedtuple(
|
||||
'ManagedDisksSettings', [
|
||||
'location', 'resource_group', 'premium', 'disk_size_gb', 'disk_names',
|
||||
|
@ -410,9 +429,12 @@ MonitoringVmSettings = collections.namedtuple(
|
|||
'accelerated_networking',
|
||||
]
|
||||
)
|
||||
CustomMountFstabSettings = collections.namedtuple(
|
||||
'CustomMountFstabSettings', [
|
||||
'fs_spec', 'fs_vfstype', 'fs_mntops', 'fs_freq', 'fs_passno',
|
||||
FederationProxyOptionsSettings = collections.namedtuple(
|
||||
'FederationProxyOptionsSettings', [
|
||||
'federations_polling_interval', 'actions_polling_interval',
|
||||
'log_persistence', 'log_level', 'log_filename',
|
||||
'scheduling_after_success_blackout_interval',
|
||||
'scheduling_after_success_evaluate_autoscale',
|
||||
]
|
||||
)
|
||||
|
||||
|
@ -425,6 +447,8 @@ class VmResource(object):
|
|||
# type: (VmResource, str, str, str, str, PublicIpSettings,
|
||||
# VirtualNetworkSettings, NetworkSecuritySettings, SSHSettings,
|
||||
# bool) -> None
|
||||
if location is None or ' ' in location:
|
||||
raise ValueError('invalid location specified')
|
||||
self.location = location
|
||||
self.resource_group = resource_group
|
||||
self.hostname_prefix = hostname_prefix
|
||||
|
@ -725,7 +749,10 @@ def is_rdma_pool(vm_size):
|
|||
:rtype: bool
|
||||
:return: if rdma is present
|
||||
"""
|
||||
if vm_size.lower() in _RDMA_INSTANCES:
|
||||
vsl = vm_size.lower()
|
||||
if vsl in _RDMA_INSTANCES:
|
||||
return True
|
||||
elif any(vsl.endswith(x) for x in _RDMA_INSTANCE_SUFFIXES):
|
||||
return True
|
||||
return False
|
||||
|
||||
|
@ -737,11 +764,10 @@ def is_premium_storage_vm_size(vm_size):
|
|||
:rtype: bool
|
||||
:return: if vm size is premium storage compatible
|
||||
"""
|
||||
if any([vm_size.lower().endswith(x)
|
||||
for x in _PREMIUM_STORAGE_INSTANCE_SUFFIXES]):
|
||||
vsl = vm_size.lower()
|
||||
if any(vsl.endswith(x) for x in _PREMIUM_STORAGE_INSTANCE_SUFFIXES):
|
||||
return True
|
||||
elif any([vm_size.lower().startswith(x)
|
||||
for x in _PREMIUM_STORAGE_INSTANCE_PREFIXES]):
|
||||
elif any(vsl.startswith(x) for x in _PREMIUM_STORAGE_INSTANCE_PREFIXES):
|
||||
return True
|
||||
return False
|
||||
|
||||
|
@ -1573,6 +1599,25 @@ def credentials_management(config):
|
|||
)
|
||||
|
||||
|
||||
def parse_batch_service_url(account_service_url, test_cluster=False):
|
||||
# type: (str, bool) -> Tuple[str, str]
|
||||
"""Parse batch service url into account name and location
|
||||
:param str account_service_url: account url
|
||||
:param bool test_cluster: test cluster
|
||||
:rtype: tuple
|
||||
:return: account, location
|
||||
"""
|
||||
# parse location from url
|
||||
tmp = account_service_url.split('.')
|
||||
location = tmp[1].lower()
|
||||
# parse account name from url
|
||||
if test_cluster:
|
||||
account = account_service_url.split('/')[-1]
|
||||
else:
|
||||
account = tmp[0].split('/')[-1]
|
||||
return account, location
|
||||
|
||||
|
||||
def credentials_batch(config):
|
||||
# type: (dict) -> BatchCredentialsSettings
|
||||
"""Get Batch credentials
|
||||
|
@ -1596,14 +1641,8 @@ def credentials_batch(config):
|
|||
creds['management'], 'subscription_id')
|
||||
except (KeyError, TypeError):
|
||||
subscription_id = None
|
||||
# parse location from url
|
||||
tmp = account_service_url.split('.')
|
||||
location = tmp[1]
|
||||
# parse account name from url
|
||||
if test_cluster:
|
||||
account = account_service_url.split('/')[-1]
|
||||
else:
|
||||
account = tmp[0].split('/')[-1]
|
||||
account, location = parse_batch_service_url(
|
||||
account_service_url, test_cluster=test_cluster)
|
||||
aad = _aad_credentials(
|
||||
creds,
|
||||
'batch',
|
||||
|
@ -1993,26 +2032,28 @@ def batch_shipyard_encryption_public_key_pem(config):
|
|||
return pem
|
||||
|
||||
|
||||
def docker_registries(config):
|
||||
# type: (dict) -> list
|
||||
def docker_registries(config, images=None):
|
||||
# type: (dict, List[str]) -> list
|
||||
"""Get Docker registries specified
|
||||
:param dict config: configuration object
|
||||
:param list images: list of images to base return
|
||||
:rtype: list
|
||||
:return: list of batchmodels.ContainerRegistry objects
|
||||
"""
|
||||
servers = []
|
||||
# get fallback docker registry
|
||||
bs = batch_shipyard_settings(config)
|
||||
if util.is_not_empty(bs.fallback_registry):
|
||||
servers.append(bs.fallback_registry)
|
||||
# get additional docker registries
|
||||
try:
|
||||
servers.extend(
|
||||
config['global_resources']['additional_registries']['docker'])
|
||||
except KeyError:
|
||||
pass
|
||||
if images is None:
|
||||
# get fallback docker registry
|
||||
bs = batch_shipyard_settings(config)
|
||||
if util.is_not_empty(bs.fallback_registry):
|
||||
servers.append(bs.fallback_registry)
|
||||
# get additional docker registries
|
||||
try:
|
||||
servers.extend(
|
||||
config['global_resources']['additional_registries']['docker'])
|
||||
except KeyError:
|
||||
pass
|
||||
images = global_resources_docker_images(config)
|
||||
# parse images for servers
|
||||
images = global_resources_docker_images(config)
|
||||
for image in images:
|
||||
tmp = image.split('/')
|
||||
if len(tmp) > 1:
|
||||
|
@ -2049,21 +2090,24 @@ def docker_registries(config):
|
|||
return registries
|
||||
|
||||
|
||||
def singularity_registries(config):
|
||||
# type: (dict) -> list
|
||||
def singularity_registries(config, images=None):
|
||||
# type: (dict, List[str]) -> list
|
||||
"""Get Singularity registries specified
|
||||
:param dict config: configuration object
|
||||
:param list images: list of images to base return
|
||||
:rtype: list
|
||||
:return: list of batchmodels.ContainerRegistry objects
|
||||
"""
|
||||
servers = []
|
||||
try:
|
||||
servers.extend(
|
||||
config['global_resources']['additional_registries']['singularity'])
|
||||
except KeyError:
|
||||
pass
|
||||
if images is None:
|
||||
try:
|
||||
servers.extend(
|
||||
config['global_resources']['additional_registries'][
|
||||
'singularity'])
|
||||
except KeyError:
|
||||
pass
|
||||
images = global_resources_singularity_images(config)
|
||||
# parse images for servers
|
||||
images = global_resources_singularity_images(config)
|
||||
for image in images:
|
||||
tmp = image.split('/')
|
||||
if len(tmp) > 1:
|
||||
|
@ -3012,6 +3056,145 @@ def job_allow_run_on_missing(conf):
|
|||
return allow
|
||||
|
||||
|
||||
def job_federation_constraint_settings(conf, federation_id):
|
||||
# type: (dict, str) -> dict
|
||||
"""Gets federation constraints
|
||||
:param dict conf: job configuration object
|
||||
:param str federation_id: federation id
|
||||
:rtype: dict
|
||||
:return: federation constraints
|
||||
"""
|
||||
if util.is_none_or_empty(federation_id):
|
||||
return None
|
||||
fc_conf = _kv_read_checked(conf, 'federation_constraints', default={})
|
||||
pool_conf = _kv_read_checked(fc_conf, 'pool', default={})
|
||||
native = _kv_read(pool_conf, 'native')
|
||||
windows = _kv_read(pool_conf, 'windows')
|
||||
if windows and native is not None and not native:
|
||||
raise ValueError(
|
||||
'cannot set constraint windows as true and native as false')
|
||||
pool_location = _kv_read_checked(pool_conf, 'location')
|
||||
if pool_location is not None:
|
||||
if ' ' in pool_location:
|
||||
raise ValueError(
|
||||
'pool:location "{}" is invalid, please ensure proper region '
|
||||
'name and not its display name'.format(pool_location))
|
||||
pool_location = pool_location.lower()
|
||||
pool_custom_image_arm_id = _kv_read_checked(
|
||||
pool_conf, 'custom_image_arm_id')
|
||||
if pool_custom_image_arm_id is not None:
|
||||
pool_custom_image_arm_id = pool_custom_image_arm_id.lower()
|
||||
pool_virtual_network_arm_id = _kv_read_checked(
|
||||
pool_conf, 'virtual_network_arm_id')
|
||||
if pool_virtual_network_arm_id is not None:
|
||||
pool_virtual_network_arm_id = pool_virtual_network_arm_id.lower()
|
||||
pool_lp_conf = _kv_read_checked(
|
||||
pool_conf, 'low_priority_nodes', default={})
|
||||
lp_allow = _kv_read(pool_lp_conf, 'allow', default=True)
|
||||
lp_exclusive = _kv_read(pool_lp_conf, 'exclusive', default=False)
|
||||
if not lp_allow and lp_exclusive:
|
||||
raise ValueError(
|
||||
'cannot set constraint low_priority:allow to false and '
|
||||
'low_priority:exclusive to true')
|
||||
pool_as_conf = _kv_read_checked(
|
||||
pool_conf, 'autoscale', default={})
|
||||
autoscale_allow = _kv_read(pool_as_conf, 'allow', default=True)
|
||||
autoscale_exclusive = _kv_read(pool_as_conf, 'exclusive', default=False)
|
||||
if not autoscale_allow and autoscale_exclusive:
|
||||
raise ValueError(
|
||||
'cannot set constraint autoscale:allow to false and '
|
||||
'autoscale:exclusive to true')
|
||||
pool_reg_conf = _kv_read_checked(
|
||||
pool_conf, 'container_registries', default={})
|
||||
pool_mab_conf = _kv_read_checked(
|
||||
pool_conf, 'max_active_task_backlog', default={})
|
||||
matbr = _kv_read(pool_mab_conf, 'ratio')
|
||||
if matbr is not None:
|
||||
matbr = float(matbr)
|
||||
if matbr < 0:
|
||||
raise ValueError(
|
||||
'cannot set constraint max_active_task_backlog:ratio to '
|
||||
'a negative value')
|
||||
matbae = _kv_read(
|
||||
pool_mab_conf, 'autoscale_exempt', default=True)
|
||||
node_conf = _kv_read_checked(fc_conf, 'compute_node', default={})
|
||||
vm_size = _kv_read_checked(node_conf, 'vm_size')
|
||||
if vm_size is not None:
|
||||
vm_size = vm_size.lower()
|
||||
core_conf = _kv_read_checked(node_conf, 'cores', default={})
|
||||
node_cores = _kv_read(core_conf, 'amount')
|
||||
if util.is_not_empty(vm_size) and node_cores is not None:
|
||||
raise ValueError(
|
||||
'cannot specify both vm_size and cores for compute_node '
|
||||
'constraint')
|
||||
node_core_variance = _kv_read(core_conf, 'schedulable_variance')
|
||||
if node_core_variance is not None:
|
||||
node_core_variance = float(node_core_variance)
|
||||
if node_core_variance < 0:
|
||||
raise ValueError(
|
||||
'cannot specify a negative cores:schedulable_variance')
|
||||
memory_conf = _kv_read(node_conf, 'memory', default={})
|
||||
node_memory = _kv_read_checked(memory_conf, 'amount')
|
||||
if util.is_not_empty(vm_size) and node_memory is not None:
|
||||
raise ValueError(
|
||||
'cannot specify both vm_size and memory for compute_node '
|
||||
'constraint')
|
||||
if node_memory is not None:
|
||||
node_memory = node_memory.lower()
|
||||
if node_memory[-1] not in ('b', 'k', 'm', 'g', 't'):
|
||||
raise ValueError(
|
||||
'federation_constraints:compute_node:memory has invalid '
|
||||
'suffix')
|
||||
if int(node_memory[:-1]) <= 0:
|
||||
raise ValueError(
|
||||
'federation_constraints:compute_node:memory is a '
|
||||
'non-positive value')
|
||||
node_memory_variance = _kv_read(memory_conf, 'schedulable_variance')
|
||||
if node_memory_variance is not None:
|
||||
node_memory_variance = float(node_memory_variance)
|
||||
if node_memory_variance < 0:
|
||||
raise ValueError(
|
||||
'cannot specify a negative memory:schedulable_variance')
|
||||
node_gpu = _kv_read(node_conf, 'gpu')
|
||||
if node_gpu and util.is_not_empty(vm_size) and not is_gpu_pool(vm_size):
|
||||
raise ValueError(
|
||||
'cannot specify gpu=True while vm_size does not have GPUs')
|
||||
node_ib = _kv_read(node_conf, 'infiniband')
|
||||
if node_ib and util.is_not_empty(vm_size) and not is_rdma_pool(vm_size):
|
||||
raise ValueError(
|
||||
'cannot specify infiniband=True while vm_size does not have '
|
||||
'RDMA/IB')
|
||||
return FederationConstraintSettings(
|
||||
pool=FederationPoolConstraintSettings(
|
||||
native=native,
|
||||
windows=windows,
|
||||
location=pool_location,
|
||||
custom_image_arm_id=pool_custom_image_arm_id,
|
||||
virtual_network_arm_id=pool_virtual_network_arm_id,
|
||||
low_priority_nodes_allow=lp_allow,
|
||||
low_priority_nodes_exclusive=lp_exclusive,
|
||||
autoscale_allow=autoscale_allow,
|
||||
autoscale_exclusive=autoscale_exclusive,
|
||||
container_registries_private_docker_hub=_kv_read(
|
||||
pool_reg_conf, 'private_docker_hub', default=False),
|
||||
container_registries_public=_kv_read_checked(
|
||||
pool_reg_conf, 'public'),
|
||||
max_active_task_backlog_ratio=matbr,
|
||||
max_active_task_backlog_autoscale_exempt=matbae,
|
||||
),
|
||||
compute_node=FederationComputeNodeConstraintSettings(
|
||||
vm_size=vm_size,
|
||||
cores=node_cores,
|
||||
core_variance=node_core_variance,
|
||||
memory=node_memory,
|
||||
memory_variance=node_memory_variance,
|
||||
exclusive=_kv_read(node_conf, 'exclusive', default=False),
|
||||
gpu=node_gpu,
|
||||
infiniband=node_ib,
|
||||
),
|
||||
)
|
||||
|
||||
|
||||
def job_has_merge_task(conf):
|
||||
# type: (dict) -> bool
|
||||
"""Determines if job has a merge task
|
||||
|
@ -3161,15 +3344,17 @@ def set_task_id(conf, id):
|
|||
conf['id'] = id
|
||||
|
||||
|
||||
def task_settings(cloud_pool, config, poolconf, jobspec, conf):
|
||||
def task_settings(
|
||||
cloud_pool, config, poolconf, jobspec, conf, federation_id=None):
|
||||
# type: (azure.batch.models.CloudPool, dict, PoolSettings, dict,
|
||||
# dict) -> TaskSettings
|
||||
# dict, str) -> TaskSettings
|
||||
"""Get task settings
|
||||
:param azure.batch.models.CloudPool cloud_pool: cloud pool object
|
||||
:param dict config: configuration dict
|
||||
:param PoolSettings poolconf: pool settings
|
||||
:param dict jobspec: job specification
|
||||
:param dict conf: task configuration object
|
||||
:param str federation_id: federation id
|
||||
:rtype: TaskSettings
|
||||
:return: task settings
|
||||
"""
|
||||
|
@ -3228,6 +3413,20 @@ def task_settings(cloud_pool, config, poolconf, jobspec, conf):
|
|||
image_reference.publisher.lower()
|
||||
offer = cloud_pool.virtual_machine_configuration.\
|
||||
image_reference.offer.lower()
|
||||
# get federation job constraint overrides
|
||||
if util.is_not_empty(federation_id):
|
||||
fed_constraints = job_federation_constraint_settings(
|
||||
jobspec, federation_id)
|
||||
if fed_constraints.pool.native is not None:
|
||||
native = fed_constraints.pool.native
|
||||
if fed_constraints.pool.windows is not None:
|
||||
is_windows = fed_constraints.pool.windows
|
||||
is_custom_image = util.is_not_empty(
|
||||
fed_constraints.pool.custom_image_arm_id)
|
||||
if is_multi_instance_task(conf):
|
||||
inter_node_comm = True
|
||||
else:
|
||||
fed_constraints = None
|
||||
# get depends on
|
||||
try:
|
||||
depends_on = conf['depends_on']
|
||||
|
@ -3590,6 +3789,14 @@ def task_settings(cloud_pool, config, poolconf, jobspec, conf):
|
|||
gpu = False
|
||||
# adjust for gpu settings
|
||||
if gpu:
|
||||
if util.is_not_empty(federation_id):
|
||||
# ensure that the job-level constraint does not conflict with
|
||||
# job/task level requirements
|
||||
if (fed_constraints.compute_node.gpu is not None and
|
||||
not fed_constraints.compute_node.gpu):
|
||||
raise ValueError(
|
||||
'job or task requirement of gpu conflicts with '
|
||||
'compute_node:gpu federation constraint')
|
||||
if not is_gpu_pool(vm_size):
|
||||
raise RuntimeError(
|
||||
('cannot initialize a gpu task on nodes without '
|
||||
|
@ -3615,6 +3822,22 @@ def task_settings(cloud_pool, config, poolconf, jobspec, conf):
|
|||
infiniband = False
|
||||
# adjust for infiniband
|
||||
if infiniband:
|
||||
# adjust ib with fed constraints (normalize to a known base config)
|
||||
if util.is_not_empty(federation_id):
|
||||
# ensure that the job-level constraint does not conflict with
|
||||
# job/task level requirements
|
||||
if (fed_constraints.compute_node.infiniband is not None and
|
||||
not fed_constraints.compute_node.infiniband):
|
||||
raise ValueError(
|
||||
'job or task requirement of infiniband conflicts with '
|
||||
'compute_node:infiniband federation constraint')
|
||||
# set publisher and offer (or node agent)
|
||||
if infiniband:
|
||||
if is_custom_image:
|
||||
node_agent = 'batch.node.centos'
|
||||
else:
|
||||
publisher = 'openlogic'
|
||||
offer = 'centos-hpc'
|
||||
if not inter_node_comm:
|
||||
raise RuntimeError(
|
||||
('cannot initialize an infiniband task on a '
|
||||
|
@ -3767,6 +3990,11 @@ def task_settings(cloud_pool, config, poolconf, jobspec, conf):
|
|||
# get num instances
|
||||
num_instances = conf['multi_instance']['num_instances']
|
||||
if not isinstance(num_instances, int):
|
||||
if util.is_not_empty(federation_id):
|
||||
raise ValueError(
|
||||
'cannot specify a non-integral value "{}" for '
|
||||
'num_instances for a multi-instance task destined for '
|
||||
'a federation'.format(num_instances))
|
||||
# TODO remove deprecation path
|
||||
if (num_instances == 'pool_specification_vm_count_dedicated' or
|
||||
num_instances == 'pool_specification_vm_count'):
|
||||
|
@ -4310,16 +4538,14 @@ def monitoring_settings(config):
|
|||
if util.is_none_or_empty(conf):
|
||||
raise KeyError
|
||||
except KeyError:
|
||||
raise ValueError(
|
||||
'monitoring settings are invalid or missing from global '
|
||||
'configuration')
|
||||
raise ValueError('monitoring settings are invalid')
|
||||
location = conf['location']
|
||||
if util.is_none_or_empty(location):
|
||||
raise ValueError('invalid location in monitoring:batch')
|
||||
raise ValueError('invalid location in monitoring')
|
||||
# monitoring vm settings
|
||||
rg = _kv_read_checked(conf, 'resource_group')
|
||||
if util.is_none_or_empty(rg):
|
||||
raise ValueError('invalid resource_group in monitoring:batch')
|
||||
raise ValueError('invalid resource_group in monitoring')
|
||||
vm_size = _kv_read_checked(conf, 'vm_size')
|
||||
hostname_prefix = _kv_read_checked(conf, 'hostname_prefix')
|
||||
accel_net = _kv_read(conf, 'accelerated_networking', False)
|
||||
|
@ -4364,17 +4590,7 @@ def monitoring_settings(config):
|
|||
raise ValueError(
|
||||
'expected list for prometheus network security rule')
|
||||
if 'custom_inbound_rules' in ns_conf:
|
||||
# reserve keywords (current and expected possible future support)
|
||||
_reserved = frozenset([
|
||||
'ssh', 'nfs', 'glusterfs', 'smb', 'cifs', 'samba', 'zfs',
|
||||
'beegfs', 'cephfs',
|
||||
])
|
||||
for key in ns_conf['custom_inbound_rules']:
|
||||
# ensure key is not reserved
|
||||
if key.lower() in _reserved:
|
||||
raise ValueError(
|
||||
('custom inbound rule of name {} conflicts with a '
|
||||
'reserved name {}').format(key, _reserved))
|
||||
ns_inbound[key] = InboundNetworkSecurityRule(
|
||||
destination_port_range=_kv_read_checked(
|
||||
ns_conf['custom_inbound_rules'][key],
|
||||
|
@ -4442,6 +4658,185 @@ def monitoring_settings(config):
|
|||
)
|
||||
|
||||
|
||||
def federation_proxy_options_settings(config):
|
||||
# type: (dict) -> FederationProxyOptionsSettings
|
||||
"""Get federation proxy options settings
|
||||
:param dict config: configuration dict
|
||||
:rtype: FederationProxyOptionsSettings
|
||||
:return: federation proxy options settings
|
||||
"""
|
||||
try:
|
||||
conf = config['federation']['proxy_options']
|
||||
except KeyError:
|
||||
conf = {}
|
||||
pi_conf = _kv_read_checked(conf, 'polling_interval', default={})
|
||||
fpi = _kv_read(pi_conf, 'federations', 30)
|
||||
if fpi < 5:
|
||||
raise ValueError(
|
||||
'the polling_interval:federations value can not be less than 5')
|
||||
api = _kv_read(pi_conf, 'actions', 5)
|
||||
if api < 5:
|
||||
raise ValueError(
|
||||
'the polling_interval:actions value can not be less than 5')
|
||||
log_conf = _kv_read_checked(conf, 'logging', default={})
|
||||
sched_conf = _kv_read_checked(conf, 'scheduling', default={})
|
||||
as_conf = _kv_read_checked(sched_conf, 'after_success', default={})
|
||||
sasbi = _kv_read(as_conf, 'blackout_interval', 15)
|
||||
if sasbi < 2:
|
||||
raise ValueError(
|
||||
'the scheduling:after_success:blackout_interval value can not '
|
||||
'be less than 2')
|
||||
return FederationProxyOptionsSettings(
|
||||
federations_polling_interval=str(fpi),
|
||||
actions_polling_interval=str(api),
|
||||
log_persistence=_kv_read(log_conf, 'persistence', True),
|
||||
log_level=_kv_read_checked(log_conf, 'level', 'debug'),
|
||||
log_filename=_kv_read_checked(log_conf, 'filename', 'fedproxy.log'),
|
||||
scheduling_after_success_blackout_interval=sasbi,
|
||||
scheduling_after_success_evaluate_autoscale=_kv_read(
|
||||
as_conf, 'evaluate_autoscale', True),
|
||||
)
|
||||
|
||||
|
||||
def federation_settings(config):
|
||||
# type: (dict) -> VmResource
|
||||
"""Get federation settings
|
||||
:param dict config: configuration dict
|
||||
:rtype: VmResource
|
||||
:return: VM resource settings
|
||||
"""
|
||||
# general settings
|
||||
try:
|
||||
conf = config['federation']
|
||||
if util.is_none_or_empty(conf):
|
||||
raise KeyError
|
||||
except KeyError:
|
||||
raise ValueError('federation settings are invalid or missing')
|
||||
location = conf['location']
|
||||
if util.is_none_or_empty(location):
|
||||
raise ValueError('invalid location in federation')
|
||||
# vm settings
|
||||
rg = _kv_read_checked(conf, 'resource_group')
|
||||
if util.is_none_or_empty(rg):
|
||||
raise ValueError('invalid resource_group in federation')
|
||||
vm_size = _kv_read_checked(conf, 'vm_size')
|
||||
hostname_prefix = _kv_read_checked(conf, 'hostname_prefix')
|
||||
accel_net = _kv_read(conf, 'accelerated_networking', False)
|
||||
# public ip settings
|
||||
pip_conf = _kv_read_checked(conf, 'public_ip', {})
|
||||
pip_enabled = _kv_read(pip_conf, 'enabled', True)
|
||||
pip_static = _kv_read(pip_conf, 'static', False)
|
||||
# sc network security settings
|
||||
ns_conf = conf['network_security']
|
||||
ns_inbound = {
|
||||
'ssh': InboundNetworkSecurityRule(
|
||||
destination_port_range='22',
|
||||
source_address_prefix=_kv_read_checked(ns_conf, 'ssh', ['*']),
|
||||
protocol='tcp',
|
||||
),
|
||||
}
|
||||
if not isinstance(ns_inbound['ssh'].source_address_prefix, list):
|
||||
raise ValueError('expected list for ssh network security rule')
|
||||
if 'custom_inbound_rules' in ns_conf:
|
||||
for key in ns_conf['custom_inbound_rules']:
|
||||
ns_inbound[key] = InboundNetworkSecurityRule(
|
||||
destination_port_range=_kv_read_checked(
|
||||
ns_conf['custom_inbound_rules'][key],
|
||||
'destination_port_range'),
|
||||
source_address_prefix=_kv_read_checked(
|
||||
ns_conf['custom_inbound_rules'][key],
|
||||
'source_address_prefix'),
|
||||
protocol=_kv_read_checked(
|
||||
ns_conf['custom_inbound_rules'][key], 'protocol'),
|
||||
)
|
||||
if not isinstance(ns_inbound[key].source_address_prefix, list):
|
||||
raise ValueError(
|
||||
'expected list for network security rule {} '
|
||||
'source_address_prefix'.format(key))
|
||||
# ssh settings
|
||||
ssh_conf = conf['ssh']
|
||||
ssh_username = _kv_read_checked(ssh_conf, 'username')
|
||||
ssh_public_key = _kv_read_checked(ssh_conf, 'ssh_public_key')
|
||||
if util.is_not_empty(ssh_public_key):
|
||||
ssh_public_key = pathlib.Path(ssh_public_key)
|
||||
ssh_public_key_data = _kv_read_checked(ssh_conf, 'ssh_public_key_data')
|
||||
ssh_private_key = _kv_read_checked(ssh_conf, 'ssh_private_key')
|
||||
if util.is_not_empty(ssh_private_key):
|
||||
ssh_private_key = pathlib.Path(ssh_private_key)
|
||||
if (ssh_public_key is not None and
|
||||
util.is_not_empty(ssh_public_key_data)):
|
||||
raise ValueError('cannot specify both an SSH public key file and data')
|
||||
if (ssh_public_key is None and
|
||||
util.is_none_or_empty(ssh_public_key_data) and
|
||||
ssh_private_key is not None):
|
||||
raise ValueError(
|
||||
'cannot specify an SSH private key with no public key specified')
|
||||
ssh_gen_file_path = _kv_read_checked(
|
||||
ssh_conf, 'generated_file_export_path', '.')
|
||||
return VmResource(
|
||||
location=location,
|
||||
resource_group=rg,
|
||||
hostname_prefix=hostname_prefix,
|
||||
virtual_network=virtual_network_settings(
|
||||
conf,
|
||||
default_resource_group=rg,
|
||||
default_existing_ok=False,
|
||||
default_create_nonexistant=True,
|
||||
),
|
||||
network_security=NetworkSecuritySettings(
|
||||
inbound=ns_inbound,
|
||||
),
|
||||
vm_size=vm_size,
|
||||
accelerated_networking=accel_net,
|
||||
public_ip=PublicIpSettings(
|
||||
enabled=pip_enabled,
|
||||
static=pip_static,
|
||||
),
|
||||
ssh=SSHSettings(
|
||||
username=ssh_username,
|
||||
expiry_days=9999,
|
||||
ssh_public_key=ssh_public_key,
|
||||
ssh_public_key_data=ssh_public_key_data,
|
||||
ssh_private_key=ssh_private_key,
|
||||
generate_docker_tunnel_script=False,
|
||||
generated_file_export_path=ssh_gen_file_path,
|
||||
hpn_server_swap=False,
|
||||
allow_docker_access=False,
|
||||
),
|
||||
)
|
||||
|
||||
|
||||
def federation_storage_account_settings(config):
|
||||
# type: (dict) ->str
|
||||
"""Get federation storage account settings selector
|
||||
:param dict config: configuration dict
|
||||
:rtype: str
|
||||
:return: federation storage settings link
|
||||
"""
|
||||
try:
|
||||
conf = config['federation']
|
||||
if util.is_none_or_empty(conf):
|
||||
raise KeyError
|
||||
except KeyError:
|
||||
raise ValueError('federation settings are invalid or missing')
|
||||
ssel = _kv_read_checked(conf, 'storage_account_settings')
|
||||
if util.is_none_or_empty(ssel):
|
||||
raise ValueError(
|
||||
'federation storage_account_settings are invalid or missing')
|
||||
return ssel
|
||||
|
||||
|
||||
def federation_credentials_storage(config):
|
||||
# type: (dict) -> StorageCredentialsSettings
|
||||
"""Get federation storage account settings
|
||||
:param dict config: configuration dict
|
||||
:rtype: StorageCredentialsSettings
|
||||
:return: federation storage cred settings
|
||||
"""
|
||||
return credentials_storage(
|
||||
config, federation_storage_account_settings(config))
|
||||
|
||||
|
||||
def generate_availability_set_name(vr):
|
||||
# type: (VmResource) -> str
|
||||
"""Generate an availabilty set name
|
||||
|
|
1253
convoy/storage.py
1253
convoy/storage.py
Разница между файлами не показана из-за своего большого размера
Загрузить разницу
|
@ -58,6 +58,7 @@ class ConfigType(enum.Enum):
|
|||
Jobs = 4,
|
||||
RemoteFS = 5,
|
||||
Monitor = 6,
|
||||
Federation = 7,
|
||||
|
||||
|
||||
# global defines
|
||||
|
@ -87,6 +88,10 @@ _SCHEMAS = {
|
|||
'name': 'Monitor',
|
||||
'schema': pathlib.Path(_ROOT_PATH, 'schemas/monitor.yaml'),
|
||||
},
|
||||
ConfigType.Federation: {
|
||||
'name': 'Federation',
|
||||
'schema': pathlib.Path(_ROOT_PATH, 'schemas/federation.yaml'),
|
||||
},
|
||||
}
|
||||
|
||||
# configure loggers
|
||||
|
|
|
@ -23,6 +23,9 @@ Batch Shipyard.
|
|||
Batch Shipyard resource monitoring configuration. This configuration is
|
||||
entirely optional unless using the resource monitoring capabilities of
|
||||
Batch Shipyard.
|
||||
7. [Federation](17-batch-shipyard-configuration-federation.md) -
|
||||
Batch Shipyard federation configuration. This configuration is entirely
|
||||
optional unless using the federation capabilities of Batch Shipyard.
|
||||
|
||||
Note that all potential properties are described here and that specifying
|
||||
all such properties may result in invalid configuration as some properties
|
||||
|
|
|
@ -81,6 +81,37 @@ job_specifications:
|
|||
- jobdata*.bin
|
||||
blobxfer_extra_options: null
|
||||
default_working_dir: batch
|
||||
federation_constraints:
|
||||
pool:
|
||||
autoscale:
|
||||
allow: true
|
||||
exclusive: false
|
||||
low_priority_nodes:
|
||||
allow: true
|
||||
exclusive: false
|
||||
native: false
|
||||
windows: false
|
||||
location: eastus
|
||||
container_registries:
|
||||
private_docker_hub: true
|
||||
public:
|
||||
- my.public.registry.io
|
||||
max_active_task_backlog:
|
||||
ratio: null
|
||||
autoscale_exempt: true
|
||||
custom_image_arm_id: null
|
||||
virtual_network_arm_id: null
|
||||
compute_node:
|
||||
vm_size: STANDARD_F1
|
||||
cores:
|
||||
amount: 2
|
||||
schedulable_variance: null
|
||||
memory:
|
||||
amount: 512m
|
||||
schedulable_variance: null
|
||||
exclusive: false
|
||||
gpu: false
|
||||
infiniband: false
|
||||
tasks:
|
||||
- id: null
|
||||
docker_image: busybox
|
||||
|
@ -490,6 +521,143 @@ container runtime through either `additional_docker_run_options` or
|
|||
that property takes precedence over this option. Note that this option does
|
||||
not work in `native` mode currently; `native` mode will always override this
|
||||
option to `batch`.
|
||||
* (optional) `federation_constraints` defines properties to apply to the job
|
||||
and all tasks (i.e., the task group) when submitting the job to a federation.
|
||||
Please see the [federation guide](68-batch-shipyard-federation.md) for more
|
||||
information and terminology definitions.
|
||||
* (optional) `pool` properties defines pool-level constraints for the
|
||||
job.
|
||||
* (optional) `autoscale` properties define autoscale constraints.
|
||||
`allow` cannot be set to `false` and `exclusive` to `true`.
|
||||
* (required) `allow` specifies if the job is allowed to be
|
||||
scheduled on an autoscale-enabled pool. The default, if not
|
||||
specified, is `true`.
|
||||
* (required) `exclusive` specifies if the job can only be
|
||||
scheduled to an autoscale-enabled pool. The default, if not
|
||||
specified, is `false`.
|
||||
* (optional) `low_priority_nodes` properties define low priority
|
||||
constraints. Both of these properties are enforced on a best-effort
|
||||
basis. `allow` cannot be set to `false` and `exclusive` to `true`.
|
||||
* (required) `allow` specifies if the job is allowed to run on
|
||||
pools with low priority nodes. The default, if not specified,
|
||||
is `true`.
|
||||
* (required) `exclusive` specifies if the job can only be
|
||||
scheduled to pools with low priority nodes. The default,
|
||||
if not specified, is `false`.
|
||||
* (optional) `native` specifies if the job must be run on native
|
||||
container pools. The default, if not specified, is `null` which
|
||||
adopts the pool settings as submitted with the job.
|
||||
* (optional) `windows` specifies if the job must run on Windows
|
||||
pools. The default, if not specified, is `null` which adopts the
|
||||
pool settings as submitted with the job.
|
||||
* (optional) `location` specifies a specific location constraint.
|
||||
The location must be a proper ARM name and not the "display name".
|
||||
For example, the region "East US" is the display name, while the
|
||||
proper name is `eastus`. To get a listing of ARM region names, they
|
||||
can be queried via the Azure CLI command
|
||||
`az account list-locations -o table`. The default, if not specified,
|
||||
is `null` which does not impose a region constraint.
|
||||
* (optional) `container_registries` defines container registry
|
||||
constraints. All container images specified within each task group
|
||||
for a job must have the corresponding container registry login
|
||||
information loaded for each possible target pool within the
|
||||
federation; the image itself does not need to be pre-loaded on each
|
||||
individual compaute node (but can be pre-loaded to improve task
|
||||
startup times - please see the global configuration doc for more
|
||||
information under `global_resources`).
|
||||
* (optional) `private_docker_hub` specifies if any container image
|
||||
within the task group that reside on Docker Hub require a
|
||||
Docker Hub login. The default, if not specified, is `false` -
|
||||
which means all container images that refer to Docker Hub are
|
||||
public and require no login.
|
||||
* (optional) `public` is a list of registries that do not require
|
||||
a login. If desired, you can create non-Docker Hub repos that
|
||||
do not have logins. You will need to specify any tasks within the
|
||||
group referencing images that do not require logins.
|
||||
* (optional) `max_active_task_backlog` defines the maximum active
|
||||
task backlog constraint.
|
||||
* (optional) `ratio` is the maximum ratio as defined by:
|
||||
`active tasks in pool / available scheduling slots in pool`.
|
||||
Thus, if there are 8 active tasks waiting, with 4 available
|
||||
scheduling slots for a particluar pool, the backlog ratio for
|
||||
this pool would be `2`. Thus if this property was set to `1`,
|
||||
then this particular pool would be skipped as a potential
|
||||
target for the job. The default, if not specified, is `null`
|
||||
or that there is no `ratio` and scheduling of task groups can
|
||||
target any pool with any backlog amount.
|
||||
* (optional) `autoscale_exempt` specifies if autoscale-enabled
|
||||
pools should not be subject to this constraint while in
|
||||
allocation state `steady` and the number of available slots
|
||||
is 0. This is to ensure that autoscale pools would be able to
|
||||
be targetted for jobs even if the pool currently contains no
|
||||
nodes. The default, if not specified, is `true`.
|
||||
* (optional) `custom_image_arm_id` defines the custom image ARM
|
||||
image id that must be used as the pool host OS for this task group
|
||||
to be scheduled.
|
||||
* (optional) `virtual_network_arm_id` defines that a pool must be
|
||||
joined with this specific ARM subnet id to be scheduled with this
|
||||
task group.
|
||||
* (optional) `compute_node` are compute node level specific constraints.
|
||||
* (optional) `vm_size` defines an
|
||||
[Azure Batch supported Azure VM size](https://docs.microsoft.com/azure/batch/batch-pool-vm-sizes)
|
||||
that this task group must be scheduled to. The default, if not
|
||||
specified, is `null` or no restriction. This property is mutually
|
||||
exclusive of `cores` and/or `memory` constraints below.
|
||||
* (optional) `cores` defines scheduling behavior for the number of
|
||||
cores required for each task within the task group. This property
|
||||
is mutually exclusive of the `vm_size` constraint.
|
||||
* (optional) `amount` is the number of cores required for each
|
||||
task in the task group. Azure Batch schedules to scheduling
|
||||
slots and does not schedule to cores. This limitation leads to
|
||||
the next constraint. The default, if not specified, is
|
||||
`null` or no restriction.
|
||||
* (optional) `schedulable_variance` is the maximum
|
||||
"over-provisioned" core capacity allowed for each task which
|
||||
constructs a valid compute node core range target of
|
||||
[amount, amount * (1 + schedulable_variance)]. For example,
|
||||
if `amount` is set to `2` and this property is set to `1` (which
|
||||
is equivalent to 100% of the specified `amount`) then the job's
|
||||
tasks will only target pools with compute nodes with a range of
|
||||
cores [2, 4], inclusive. The default value, if not specified,
|
||||
is `null` which infers no upper-bound limit on the core range.
|
||||
A value of `0` infers exact match required for the specified
|
||||
`amount`.
|
||||
* (optional) `memory` defines scheduling behvior for the amount of
|
||||
memory required for each task within the task group. This property
|
||||
is mutually exclusive of the `vm_size` constraint.
|
||||
* (optional) `amount` is the amount of memory required for each
|
||||
task in the task group. This value should be a string where
|
||||
the amount is followed directly by a suffix `b`, `k`, `m`, `g`,
|
||||
or `t`. Azure Batch schedules to scheduling slots and
|
||||
does not schedule to memory. This limitation leads to
|
||||
the next constraint. The default, if not specified, is
|
||||
`null` or no restriction.
|
||||
* (optional) `schedulable_variance` is the maximum
|
||||
"over-provisioned" memory capacity allowed for each task which
|
||||
constructs a valid compute node memory range target of
|
||||
[amount, amount * (1 + schedulable_variance)]. For example,
|
||||
if `amount` is set to `4096m` and this property is set to `1`
|
||||
(which is equivalent to 100% of the specified `amount`) then
|
||||
the job's tasks will only target pools with compute nodes with
|
||||
a range of memory [4096m, 8192m], inclusive. The default value,
|
||||
if not specified, is `null` which infers no upper-bound limit
|
||||
on the memory range. A value of `0` infers exact match
|
||||
required for the specified `amount`. It is not recommended to
|
||||
set this value to `0` for a `memory` constraint.
|
||||
* (optional) `exclusive` specifies if each task within the task group
|
||||
must not be co-scheduled with other running tasks on compute nodes.
|
||||
Effectively this excludes pools as scheduling targets with the
|
||||
setting `max_tasks_per_node` greater than `1`.
|
||||
* (optional) `gpu` specifies if tasks within the task group should
|
||||
be scheduled on a compute node that has a GPU. Note that specifying
|
||||
this property as `true` does not implicitly create tasks that
|
||||
utilize nvidia-docker. You must, instead, specify the `gpu`
|
||||
property as `true` at the job or task-level.
|
||||
* (optional) `infiniband` specifies if tasks within the task group
|
||||
should be scheduled on a compute node that has a RDMA/IB. Note that
|
||||
specifying this property as `true` does not implicitly create tasks
|
||||
that will enable RDMA/IB settings with Intel MPI. You must, instead,
|
||||
specify the `infiniband` property as `true` at the job or task-level.
|
||||
|
||||
The required `tasks` property is an array of tasks to add to the job:
|
||||
|
||||
|
@ -610,7 +778,7 @@ Singularity image to use for this task
|
|||
* (optional) `depends_on` is an array of task ids for which this container
|
||||
invocation (task) depends on and must run to successful completion prior
|
||||
to this task executing. Note that when a `task_factory` is specified, all
|
||||
tasks generated by the task factory depend on the listed depenedent tasks.
|
||||
tasks generated by the task factory depend on the listed dependent tasks.
|
||||
You cannot specify another task factory to depend on, only discrete tasks.
|
||||
* (optional) `depends_on_range` is an array with exactly two integral
|
||||
elements containing a task `id` range for which this task is dependent
|
||||
|
|
|
@ -0,0 +1,194 @@
|
|||
# Batch Shipyard Federation Configuration
|
||||
This page contains in-depth details on how to configure the federation
|
||||
configuration file for Batch Shipyard.
|
||||
|
||||
## Schema
|
||||
The federation schema is as follows:
|
||||
|
||||
```yaml
|
||||
federation:
|
||||
storage_account_settings: mystorageaccount
|
||||
location: <Azure region, e.g., eastus>
|
||||
resource_group: my-federation-proxy-rg
|
||||
hostname_prefix: fed
|
||||
ssh:
|
||||
username: shipyard
|
||||
ssh_public_key: /path/to/rsa/publickey.pub
|
||||
ssh_public_key_data: ssh-rsa ...
|
||||
ssh_private_key: /path/to/rsa/privatekey
|
||||
generated_file_export_path: null
|
||||
public_ip:
|
||||
enabled: true
|
||||
static: false
|
||||
virtual_network:
|
||||
name: myvnet
|
||||
resource_group: my-vnet-resource-group
|
||||
existing_ok: false
|
||||
address_space: 10.0.0.0/16
|
||||
subnet:
|
||||
name: my-federation-proxy-subnet
|
||||
address_prefix: 10.0.0.0/24
|
||||
network_security:
|
||||
ssh:
|
||||
- '*'
|
||||
custom_inbound_rules:
|
||||
myrule:
|
||||
destination_port_range: 5000-5001
|
||||
protocol: '*'
|
||||
source_address_prefix:
|
||||
- 1.2.3.4
|
||||
- 5.6.7.0/24
|
||||
vm_size: STANDARD_D2_V2
|
||||
accelerated_networking: false
|
||||
proxy_options:
|
||||
polling_interval:
|
||||
federations: 15
|
||||
jobs: 5
|
||||
logging:
|
||||
persistence: true
|
||||
level: debug
|
||||
filename: fedproxy.log
|
||||
scheduling:
|
||||
after_success:
|
||||
blackout_interval: 15
|
||||
evaluate_autoscale: true
|
||||
```
|
||||
|
||||
The `federation` property has the following members:
|
||||
|
||||
* (required) `storage_account_settings` is the storage account link to store
|
||||
all federation metadata. Any `fed` command that must store metadata or
|
||||
actions uses this storage account. The federation proxy created with this
|
||||
configuration file will also utilize this storage account.
|
||||
* (required) `location` is the Azure region name for the resources, e.g.,
|
||||
`eastus` or `northeurope`.
|
||||
* (required) `resource_group` this is the resource group to use for the
|
||||
federation proxy.
|
||||
* (required) `hostname_prefix` is the DNS label prefix to apply to each
|
||||
virtual machine and resource allocated for the federation proxy. It should
|
||||
be unique.
|
||||
* (required) `ssh` is the SSH admin user to create on the machine. This is not
|
||||
optional in this configuration as it is in the pool specification. If you are
|
||||
running Batch Shipyard on Windows, please refer to
|
||||
[these instructions](85-batch-shipyard-ssh-docker-tunnel.md#ssh-keygen)
|
||||
on how to generate an SSH keypair for use with Batch Shipyard.
|
||||
* (required) `username` is the admin user to create on all virtual machines
|
||||
* (optional) `ssh_public_key` is the path to a pre-existing ssh public
|
||||
key to use. If this is not specified, an RSA public/private key pair will
|
||||
be generated for use in your current working directory (with a
|
||||
non-colliding name for auto-generated SSH keys for compute pools, i.e.,
|
||||
`id_rsa_shipyard_remotefs`). On Windows only, if this is option is not
|
||||
specified, the SSH keys are not auto-generated (unless `ssh-keygen.exe`
|
||||
can be invoked in the current working directory or is in `%PATH%`).
|
||||
This option cannot be specified with `ssh_public_key_data`.
|
||||
* (optional) `ssh_public_key_data` is the raw RSA public key data in
|
||||
OpenSSH format, e.g., a string starting with `ssh-rsa ...`. Only one
|
||||
key may be specified. This option cannot be specified with
|
||||
`ssh_public_key`.
|
||||
* (optional) `ssh_private_key` is the path to an existing SSH private key
|
||||
to use against either `ssh_public_key` or `ssh_public_key_data` for
|
||||
connecting to storage nodes and performing operations that require SSH
|
||||
such as cluster resize and detail status. This option should only be
|
||||
specified if either `ssh_public_key` or `ssh_public_key_data` are
|
||||
specified.
|
||||
* (optional) `generated_file_export_path` is an optional path to specify
|
||||
for where to create the RSA public/private key pair.
|
||||
* (optional) `public_ip` are public IP properties for the virtual machine.
|
||||
* (optional) `enabled` designates if public IPs should be assigned. The
|
||||
default is `true`. Note that if public IP is disabled, then you must
|
||||
create an alternate means for accessing the federation proxy virtual
|
||||
machine through a "jumpbox" on the virtual network. If this property
|
||||
is set to `false` (disabled), then any action requiring SSH, or the
|
||||
SSH command itself, will occur against the private IP address of the
|
||||
virtual machine.
|
||||
* (optional) `static` is to specify if static public IPs should be assigned
|
||||
to each virtual machine allocated. The default is `false` which
|
||||
results in dynamic public IP addresses. A "static" FQDN will be provided
|
||||
per virtual machine, regardless of this setting if public IPs are
|
||||
enabled.
|
||||
* (required) `virtual_network` is the virtual network to use for the
|
||||
federation proxy.
|
||||
* (required) `name` is the virtual network name
|
||||
* (optional) `resource_group` is the resource group for the virtual
|
||||
network. If this is not specified, the resource group name falls back
|
||||
to the resource group specified in the federation proxy.
|
||||
* (optional) `existing_ok` allows use of a pre-existing virtual network.
|
||||
The default is `false`.
|
||||
* (required if creating, optional otherwise) `address_space` is the
|
||||
allowed address space for the virtual network.
|
||||
* (required) `subnet` specifies the subnet properties.
|
||||
* (required) `name` is the subnet name.
|
||||
* (required) `address_prefix` is the subnet address prefix to use for
|
||||
allocation of the federation proxy virtual machine to.
|
||||
* (required) `network_security` defines the network security rules to apply
|
||||
to the federation proxy virtual machine.
|
||||
* (required) `ssh` is the rule for which address prefixes to allow for
|
||||
connecting to sshd port 22 on the virtual machine. In the example, `"*"`
|
||||
allows any IP address to connect. This is an array property which allows
|
||||
multiple address prefixes to be specified.
|
||||
* (optional) `grafana` rule allows grafana HTTPS (443) server port to be
|
||||
exposed to the specified address prefix. Multiple address prefixes
|
||||
can be specified.
|
||||
* (optional) `prometheus` rule allows the Prometheus server port to be
|
||||
exposed to the specified address prefix. Multiple address prefixes
|
||||
can be specified.
|
||||
* (optional) `custom_inbound_rules` are custom inbound rules for other
|
||||
services that you need to expose.
|
||||
* (required) `<rule name>` is the name of the rule; the example uses
|
||||
`myrule`. Each rule name should be unique.
|
||||
* (required) `destination_port_range` is the ports on each virtual
|
||||
machine that will be exposed. This can be a single port and
|
||||
should be a string.
|
||||
* (required) `source_address_prefix` is an array of address
|
||||
prefixes to allow.
|
||||
* (required) `protocol` is the protocol to allow. Valid values are
|
||||
`tcp`, `udp` and `*` (which means any protocol).
|
||||
* (required) `vm_size` is the virtual machine instance size to use.
|
||||
* (optional) `accelerated_networking` enables or disables
|
||||
[accelerated networking](https://docs.microsoft.com/azure/virtual-network/create-vm-accelerated-networking-cli).
|
||||
The default is `false` if not specified.
|
||||
* (optional) `proxy_options` are the federation proxy specific properties
|
||||
* (optional) `polling_interval` specifies different polling interval
|
||||
lengths
|
||||
* (optional) `federations` specifies the amount of time in seconds
|
||||
between checking for federation updates. The default, if not
|
||||
specified, is `15`.
|
||||
* (optional) `jobs` specifies the amount of time in seconds between.
|
||||
checking for federation action queued messages. The default, if not
|
||||
specified, is `5`.
|
||||
* (optional) `logging` specifies various logging options
|
||||
* (optional) `persistence` specifies if logs should be persisted to
|
||||
Azure File storage. The default, if not specified, is `true`.
|
||||
* (optional) `level` specifies the level to log including all "higher"
|
||||
levels. The default, if not specified, is `debug`.
|
||||
* (optional) `filename` is a log filename schema where the `level` is
|
||||
injected as part of the filename. At most two files will be
|
||||
created initially, which is a file containing the specified `level`
|
||||
and the `error` level. The default, if not specified,
|
||||
is `fedproxy.log`. If the `level` specified is `debug`, then
|
||||
the log files `fedproxy-debug.log` and `fedproxy-error.log` will
|
||||
be created. Log files are automatically rotated after 32MiB of
|
||||
data has been written.
|
||||
* (optional) `scheduling` specifies federation proxy wide scheduling
|
||||
options to use while processing actions.
|
||||
* (optional) `after_success` apply to actions which have been
|
||||
successfully scheduled.
|
||||
* (optional) `blackout_interval` specifies the scheduling blackout
|
||||
interval to apply to the target pool in seconds. The default,
|
||||
if not specified, is `15`.
|
||||
* (optional) `evaluate_autoscale` specifies if the autoscale
|
||||
formula should be immediately applied to the target pool after
|
||||
a task group has been successfully scheduled. This option only
|
||||
applies to autoscale-enabled pools. The default, if not
|
||||
specified, is `true`.
|
||||
|
||||
## Federations with Batch Shipyard Guide
|
||||
Please see the [full guide](68-batch-shipyard-federation.md) for
|
||||
relevant terminology and information on how this feature works in Batch
|
||||
Shipyard.
|
||||
|
||||
## Full template
|
||||
A full template of a federation configuration file can be found
|
||||
[here](https://github.com/Azure/batch-shipyard/tree/master/config_templates).
|
||||
Note that these templates cannot be used as-is and must be modified to fit
|
||||
your scenario.
|
|
@ -116,13 +116,26 @@ These options must be specified after the command and sub-command. These are:
|
|||
of commands support this option. Note many of the supported commands are
|
||||
returning raw JSON body results from the Batch API server, thus the output
|
||||
may change/break if the underlying service version changes. It is important
|
||||
to pin the Batch Shipyard release to a specific version if using this feature.
|
||||
The following commands support this option:
|
||||
to pin the Batch Shipyard release to a specific version if using this feature
|
||||
and perform upgrade testing/validation for your scenario and workflow between
|
||||
releases. The following commands support this option:
|
||||
* `account info`
|
||||
* `account quota`
|
||||
* `cert list`
|
||||
* `fed create`
|
||||
* `fed destroy`
|
||||
* `fed list`
|
||||
* `fed jobs add`
|
||||
* `fed jobs del`
|
||||
* `fed jobs list`
|
||||
* `fed jobs term`
|
||||
* `fed jobs zap`
|
||||
* `fed pool add`
|
||||
* `fed pool remove`
|
||||
* `fed proxy status`
|
||||
* `jobs list`
|
||||
* `jobs tasks list`
|
||||
* `monitor status`
|
||||
* `pool autoscale evaluate`
|
||||
* `pool autoscale lastexec`
|
||||
* `pool images list`
|
||||
|
@ -354,6 +367,123 @@ specified Azure storage container.
|
|||
* `--nodeid` is the node id to upload from
|
||||
* `--wait` will wait until the operation completes
|
||||
|
||||
## `fed` Command
|
||||
The `fed` command has the following sub-commands:
|
||||
```
|
||||
create Create a federation
|
||||
destroy Destroy a federation
|
||||
jobs Federation jobs actions
|
||||
list List all federations
|
||||
pool Federation pool actions
|
||||
proxy Federation proxy actions
|
||||
```
|
||||
|
||||
The `fed jobs` sub-command has the following sub-sub-commands:
|
||||
```
|
||||
add Add jobs to a federation
|
||||
del Delete a job or job schedule in a federation
|
||||
list List jobs or job schedules in a federation
|
||||
term Terminate a job or job schedule in a...
|
||||
zap Zap a queued unique id from a federation
|
||||
```
|
||||
|
||||
The `fed pool` sub-command has the following sub-sub-commands:
|
||||
```
|
||||
add Add a pool to a federation
|
||||
remove Remove a pool from a federation
|
||||
```
|
||||
|
||||
The `fed proxy` sub-command has the following sub-sub-commands:
|
||||
```
|
||||
create Create a federation proxy
|
||||
destroy Destroy a federation proxy
|
||||
ssh Interactively login via SSH to federation...
|
||||
start Starts a previously suspended federation...
|
||||
status Query status of a federation proxy
|
||||
suspend Suspend a federation proxy
|
||||
```
|
||||
|
||||
* `create` will create a federation
|
||||
* `FEDERATION_ID` is the federation id name
|
||||
* `--force` force creates the federation even if a federation with a
|
||||
same id exists.
|
||||
* `--no-unique-job-ids` creates a federation without unique job id
|
||||
enforcement.
|
||||
* `destroy` will destroy a previously created federation
|
||||
* `FEDERATION_ID` is the federation id name
|
||||
* `jobs add` submits jobs/task groups or job schedules to a federation
|
||||
* `FEDERATION_ID` is the federation id name
|
||||
* `jobs del` submits an action to delete jobs or job schedules from a
|
||||
federation
|
||||
* `FEDERATION_ID` is the federation id name
|
||||
* `--all-jobs` deletes all jobs in the federation
|
||||
* `--all-jobschedules` deletes all job schedules in the federation
|
||||
* `--job-id` deletes a specific job id. This can be specified multiple
|
||||
times.
|
||||
* `--job-schedule-id` deletes a specific job schedule id. This can be
|
||||
specified multiple times.
|
||||
* `jobs list` lists jobs or locates a job or job schedule
|
||||
* `FEDERATION_ID` is the federation id name
|
||||
* `--job-id` locates a specific job id
|
||||
* `--job-schedule-id` deletes a specific job schedule id
|
||||
* `jobs term` submits an action to terminate jobs or job schedules from a
|
||||
federation
|
||||
* `FEDERATION_ID` is the federation id name
|
||||
* `--all-jobs` deletes all jobs in the federation
|
||||
* `--all-jobschedules` deletes all job schedules in the federation
|
||||
* `--force` forces submission of a termination action for a job even
|
||||
if it doesn't exist
|
||||
* `--job-id` deletes a specific job id. This can be specified multiple
|
||||
times.
|
||||
* `--job-schedule-id` deletes a specific job schedule id. This can be
|
||||
specified multiple times.
|
||||
* `jobs zap` removes a unique id action from a federation
|
||||
* `FEDERATION_ID` is the federation id name
|
||||
* `--unique-id` is the unique id associated with the action to zap
|
||||
* `list` will list federations
|
||||
* `--federation-id` will limit the list to the specified federation id
|
||||
* `pool add` will add a pool to a federation
|
||||
* `FEDERATION_ID` is the federation id name
|
||||
* `--batch-service-url` is the batch service url of the pool id to add
|
||||
instead of read from the credentials configuration
|
||||
* `--pool-id` is the pool id to add instead of the pool id read from the
|
||||
pool configuration
|
||||
* `pool remove`
|
||||
* `FEDERATION_ID` is the federation id name
|
||||
* `--all` remove all pools from the federation
|
||||
* `--batch-service-url` is the batch service url of the pool id to remove
|
||||
instead of read from the credentials configuration
|
||||
* `--pool-id` is the pool id to remove instead of the pool id read from the
|
||||
pool configuration
|
||||
* `proxy create` will create the federation proxy
|
||||
* `proxy destroy` will destroy the federation proxy
|
||||
* `--delete-resource-group` will delete the entire resource group that
|
||||
contains the federation proxy. Please take care when using this
|
||||
option as any resource in the resoure group is deleted which may be
|
||||
other resources that are not Batch Shipyard related.
|
||||
* `--delete-virtual-network` will delete the virtual network and all of
|
||||
its subnets
|
||||
* `--generate-from-prefix` will attempt to generate all resource names
|
||||
using conventions used. This is helpful when there was an issue with
|
||||
monitoring creation/deletion and the original virtual machine resources
|
||||
cannot be enumerated. Note that OS disks cannot be deleted with this
|
||||
option. Please use an alternate means (i.e., the Azure Portal) to
|
||||
delete disks.
|
||||
* `--no-wait` does not wait for deletion completion. It is not recommended
|
||||
to use this parameter.
|
||||
* `proxy ssh` will interactive log into the federation proxy via SSH
|
||||
* `COMMAND` is an optional argument to specify the command to run. If your
|
||||
command has switches, preface `COMMAND` with double dash as per POSIX
|
||||
convention, e.g., `pool ssh -- sudo docker ps -a`.
|
||||
* `--tty` allocates a pseudo-terminal
|
||||
* `proxy start` will start a previously suspended federation proxy
|
||||
* `--no-wait` does not wait for the restart to complete. It is not
|
||||
recommended to use this parameter.
|
||||
* `proxy status` will query status of a federation proxy
|
||||
* `proxy suspend` suspends a federation proxy
|
||||
* `--no-wait` does not wait for the suspension to complete. It is not
|
||||
recommended to use this parameter.
|
||||
|
||||
## `fs` Command
|
||||
The `fs` command has the following sub-commands which work on two different
|
||||
parts of a remote filesystem:
|
||||
|
@ -600,6 +730,7 @@ The `monitor` command has the following sub-commands:
|
|||
remove Remove a resource from monitoring
|
||||
ssh Interactively login via SSH to monitoring...
|
||||
start Starts a previously suspended monitoring...
|
||||
status Query status of a monitoring resource
|
||||
suspend Suspend a monitoring resource
|
||||
```
|
||||
|
||||
|
@ -627,7 +758,7 @@ The `monitor` command has the following sub-commands:
|
|||
* `--all` will remove all resources that are currently monitored
|
||||
* `--poolid` will remove the specified Batch pool to monitor
|
||||
* `--remote-fs` will remove the specified RemoteFS cluster to monitor
|
||||
* `ssh` will interactively log into a compute node via SSH.
|
||||
* `ssh` will interactively log into the monitoring resource via SSH.
|
||||
* `COMMAND` is an optional argument to specify the command to run. If your
|
||||
command has switches, preface `COMMAND` with double dash as per POSIX
|
||||
convention, e.g., `pool ssh -- sudo docker ps -a`.
|
||||
|
@ -635,6 +766,7 @@ The `monitor` command has the following sub-commands:
|
|||
* `start` will start a previously suspended monitoring VM
|
||||
* `--no-wait` does not wait for the restart to complete. It is not
|
||||
recommended to use this parameter.
|
||||
* `status` will query status of a monitoring VM
|
||||
* `suspend` suspends a monitoring VM
|
||||
* `--no-wait` does not wait for the suspension to complete. It is not
|
||||
recommended to use this parameter.
|
||||
|
|
|
@ -28,6 +28,17 @@ role permission or a
|
|||
|
||||
* `Microsoft.Network/virtualNetworks/subnets/join/action`
|
||||
|
||||
### Public IP Quota
|
||||
For pools that are not internode communication enabled, more than 1 public IP
|
||||
and load balancer may be created for the pool. These public IPs are allocated
|
||||
in the subscription that has allocated the virtual network. Please ensure
|
||||
that proper Public IP quota has been granted for the subscription of the
|
||||
virtual network.
|
||||
|
||||
Note that enabling internode communication is not recommended unless
|
||||
running MPI (multinstance) jobs as this will restrict the upper-bound
|
||||
scalability of the pool.
|
||||
|
||||
## `virtual_network` Pool configuration
|
||||
To deploy Batch compute nodes into a subnet within a Virtual Network that
|
||||
you specify, you will need to define the `virtual_network` property in the
|
||||
|
@ -69,7 +80,7 @@ nodes can exceed the specified targets.
|
|||
If you are redirecting Internet-bound traffic from the subnet back to
|
||||
on-premises, then you may have to add
|
||||
[user-defined routes](https://docs.microsoft.com/azure/virtual-network/virtual-networks-udr-overview)
|
||||
to that subnet. Please follow the instructions at this
|
||||
to that subnet. Please follow the instructions found in this
|
||||
[document](https://docs.microsoft.com/azure/batch/batch-virtual-network#user-defined-routes-for-forced-tunneling).
|
||||
|
||||
## Network Security
|
||||
|
|
|
@ -0,0 +1,385 @@
|
|||
# Federations with Batch Shipyard
|
||||
The focus of this article is to explain the federation concept in Batch
|
||||
Shipyard and effectively deploying your workload across multiple pools that
|
||||
may span multiple Azure regions across the world.
|
||||
|
||||
## Overview
|
||||
In Azure Batch, each pool within a Batch account is considered a logical
|
||||
boundary for related work. Thus jobs and job schedules can only target
|
||||
a single Batch pool for execution. However, it may be desirable to have
|
||||
workloads that span multiple pools as there may be a need for a heterogenous
|
||||
mix of compute nodes types, or it may be desirable to be able to manage
|
||||
a distribution of pools, logically, together. Moreover, there should be
|
||||
no restriction for such a logical grouping to be limited to a single
|
||||
Batch account, or even a single region unless data residency, data hydration
|
||||
sensitivity or other requirements prohibit such collections.
|
||||
|
||||
To enable multi-pool collections across any number of Batch accounts and
|
||||
regions, Batch Shipyard defines the concept of a federation. Federations are
|
||||
collections of pools that can be provisioned as entirely different
|
||||
configurations logically organized as a single resource. This enables
|
||||
scenarios such as hybrid VM composition and load balancing workloads by
|
||||
routing jobs to regions where specific capabilities may be available and
|
||||
necessary. Federations also enable rich constraint matching while maintaining
|
||||
important cloud-native features of core Azure Batch such as autoscaling
|
||||
capabilities of the underlying pools.
|
||||
|
||||
## Major Features
|
||||
* Full suite of federation, federated pool and federated job management
|
||||
through the CLI
|
||||
* Simple management of federations through user-defined IDs
|
||||
* Multi-region support within a single federation
|
||||
* Ability to dynamically add and remove pools from federations on-demand
|
||||
* Support for multiple federations simultaneously using a single federation
|
||||
proxy
|
||||
* Fully automated federation proxy deployment and management including
|
||||
on-demand suspend and restart
|
||||
* Automatic persisted federation proxy logging to Azure File Storage for
|
||||
diagnostics
|
||||
* Support for job recurrences (job schedules)
|
||||
* FIFO ordering of actions within a job or job schedule
|
||||
* Leverages Azure MSI to eliminate credential passing and storage in a backing
|
||||
store
|
||||
* Federation proxies can be run in HA mode
|
||||
|
||||
## Mental Model
|
||||
The following picture describes how commands issued through Batch Shipyard
|
||||
affect metadata stores and processing by the federation proxy to ultimately
|
||||
issue Batch service API calls for job scheduling.
|
||||
|
||||
### Terminology
|
||||
Below are some helpful terms that are referenced throughout this guide that
|
||||
are not common to core Azure Batch. It is recommended to review this list
|
||||
before examining the mental model picture.
|
||||
|
||||
* Federation: a collection of pools that are logically grouped together.
|
||||
* Action: a directive for the federation proxy to process. Actions are
|
||||
enqueued on a federation queue.
|
||||
* Federation proxy: a server which processes actions.
|
||||
* Federation queue: contains enqueued actions. Actions across multiple jobs
|
||||
or job schedules are not guaranteed to be processed in FIFO order. Actions
|
||||
within a single job or job schedule are guaranteed to be processed in FIFO
|
||||
order.
|
||||
* Task group: a set of tasks within a job associated with an action.
|
||||
* Constraint: a condition placed on a job to be applied by the federation
|
||||
proxy when selecting a target pool to schedule to.
|
||||
* Scheduling slot: Each compute node can contain up to max tasks per node
|
||||
scheduling slots as specified during pool allocation.
|
||||
* Available scheduling slots: the number of compute nodes within a pool that
|
||||
can run a task. The formula representing this metric is:
|
||||
`(nodes in idle state + nodes in running state) * max_tasks_per_node`.
|
||||
|
||||
```
|
||||
+-------------+
|
||||
| Azure Files |
|
||||
+------+------+
|
||||
^
|
||||
+----------------------------+ | logging
|
||||
| Federation Metadata in | master |
|
||||
| Azure Storage Blob & Table | election +------------+--------------+
|
||||
| | <-----------+ Federation Proxy |
|
||||
| +----------+ +----------+ | | +------------+ +--------+ |
|
||||
| | Job | | Pool | | | |Dynamic | |Task | |
|
||||
+----------+ | | Metadata | | Metadata | | | |Resource | |Re-write| |
|
||||
| | push | +----------+ +----------+ | fetch | |Conditioning| |Engine | |
|
||||
| Batch +----------> | +-----------> | +------------+ +--------+ |
|
||||
| Shipyard | +----------------------------+ | +-----------+ +---------+ |
|
||||
| "fed" | | |Constraint | |Service | |
|
||||
| commands | enqueue action +------------------+ dequeue action | |Matching | |Proxy & | |
|
||||
| +---------------> | +----------------> | |Engine | |MSI Auth | |
|
||||
+----------+ | Federation Queue | | +-----------+ +---------+ |
|
||||
| | +---------------------------+
|
||||
+------------------+ |
|
||||
| schedule
|
||||
|
|
||||
+-----------------------------------------------------------v-------+
|
||||
| Batch Federation |
|
||||
| +-------------------+ +-------------------+ +-------------------+ |
|
||||
| | Region A | | Region B | | Region C | |
|
||||
| | +------+ +------+ | | +------+ +------+ | | +------+ +------+ | |
|
||||
| | |Pool 0| |Pool 1| | | |Pool 3| |Pool 4| | | |Pool 7| |Pool 8| | |
|
||||
| | +------+ +------+ | | +------+ +------+ | | +------+ +------+ | |
|
||||
| | +------+ | | +------+ +------+ | | | |
|
||||
| | |Pool 2| | | |Pool 5| |Pool 6| | | | |
|
||||
| | +------+ | | +------+ +------+ | | | |
|
||||
| +-------------------+ +-------------------+ +-------------------+ |
|
||||
+-------------------------------------------------------------------+
|
||||
```
|
||||
|
||||
## Walkthrough
|
||||
The following is a brief walkthrough of configuring a Batch Shipyard
|
||||
federation and simple usage commands for creating a federation proxy and
|
||||
submitting actions against federations.
|
||||
|
||||
### Configuration
|
||||
The configuration for a Batch Shipyard federation is generally composed of
|
||||
two major parts: the federation proxy and the normal global config, pool, and
|
||||
jobs configuration.
|
||||
|
||||
#### Federation Proxy
|
||||
The federation proxy configuration is defined by a federation configuration
|
||||
file.
|
||||
|
||||
```yaml
|
||||
federation:
|
||||
storage_account_settings: # storage account link name where all
|
||||
# federation metadata is stored
|
||||
# ... other settings
|
||||
proxy_options:
|
||||
polling_interval:
|
||||
federations: # interval in seconds
|
||||
actions: # interval in seconds
|
||||
logging:
|
||||
persistence: # automatically persist logs in real-time to Azure File Storage
|
||||
level: # logging level, defaults to "debug"
|
||||
filename: # filename schema
|
||||
scheduling:
|
||||
after_success:
|
||||
blackout_interval: # scheduling blackout interval for target pool
|
||||
# after success in seconds
|
||||
evaluate_autoscale: # immediately evaluate autoscale after scheduling
|
||||
# success if target pool was autoscale enabled
|
||||
```
|
||||
|
||||
Please refer to the full
|
||||
[federation proxy configuration documentation](17-batch-shipyard-configuration-federation.md)
|
||||
for more detailed explanations of each option, including other options not
|
||||
shown here.
|
||||
|
||||
#### Global Configuration
|
||||
Special care should be provided for the global configuration while
|
||||
provisioning pools that will take part of a federation. Any task that requires
|
||||
a login for a container registry or a shared data volume must have
|
||||
such configuration applied to all pools within the federation.
|
||||
|
||||
If it is known beforehand that a set of container images will be required
|
||||
for all task groups submitted to the federation, then they should be
|
||||
specified for all pools that will be part of the federation under
|
||||
`global_resources`:`docker_images`. Any images that require logins, should
|
||||
be defined in the credentials configuration.
|
||||
|
||||
If task container images will not be known beforehand, then it is imperative
|
||||
that the `global_resources`:`additional_registries` contains a list of all
|
||||
container images that reference private registries. The corresponding login
|
||||
information for these registries should be present in the credentials
|
||||
configuration, including private Docker Hub logins.
|
||||
|
||||
Any tasks within task groups referencing `shared_data_volumes` should have
|
||||
pools allocated with the proper `shared_data_volumes` before joining the
|
||||
federation. Special care must be taken here to ensure that any pools that
|
||||
are in different Batch accounts or regions conform to the same naming
|
||||
scheme used by tasks in task groups.
|
||||
|
||||
#### Federated Job Constraints
|
||||
Note that none of the `federation_constraints` properties are required. They
|
||||
are provided to allow for specified user requirements and optimizations on
|
||||
job/task group placement within the federation.
|
||||
|
||||
```yaml
|
||||
job_specifications:
|
||||
- id: # job id
|
||||
# ... other settings
|
||||
federation_constraints:
|
||||
pool:
|
||||
autoscale:
|
||||
allow: # allow job to be scheduled on an autoscale pool
|
||||
exclusive: # exclusively schedule job on an autoscale pool
|
||||
low_priority_nodes:
|
||||
allow: # allow job to be best-effort scheduled on a pool with low priority nodes
|
||||
exclusive: # best-effort schedule job on a pool with exclusively low priority nodes
|
||||
native: # job must be scheduled on a native container pool
|
||||
windows: # job requires a windows pool
|
||||
location: # job should be routed to a particular Azure region, must be a proper ARM name
|
||||
container_registries:
|
||||
private_docker_hub: # any task in task group with Docker Hub references
|
||||
# refer to a private Docker repository
|
||||
public:
|
||||
- # list of public registries that don't require a login for referenced
|
||||
# container images for all tasks in task group
|
||||
max_active_task_backlog: # limit scheduling a job with queued backlog of tasks
|
||||
ratio: # maximum backlog ratio allowed represented as (active tasks / schedulable slots).
|
||||
autoscale_exempt: # if autoscale pools are exempt from this ratio requirement if there
|
||||
# are no schedulable slots and their allocation state is steady
|
||||
custom_image_arm_id: # job must schedule on a pool with the specified custom image ARM image id
|
||||
virtual_network_arm_id: # job must schedule on a pool with the specified virtual network ARM subnet id
|
||||
compute_node:
|
||||
vm_size: # job must match the named Azure Batch supported Azure VM SKU size exactly
|
||||
cores:
|
||||
amount: # job requires at least this many cores
|
||||
schedulable_variance: # maximum "over-provisioned" core capacity allowed
|
||||
memory:
|
||||
amount: # job requires at least this much memory (allowable suffixes: b, k, m, g, t)
|
||||
schedulable_variance: # maximum "over-provisioned" memory capacity allowed
|
||||
exclusive: # tasks in the task group must run exclusively on the compute node
|
||||
# and cannot potentially be co-scheduled with other running tasks
|
||||
gpu: # job must be scheduled on a compute node with a GPU,
|
||||
# job/task-level requirements override this option
|
||||
infiniband: # job must be scheduled on a compute node with RDMA/IB,
|
||||
# job/task-level requirements override this option
|
||||
tasks:
|
||||
# ... other settings
|
||||
```
|
||||
|
||||
It is strongly recommended to view the
|
||||
[jobs documentation](14-batch-shipyard-configuration-jobs.md) which contains
|
||||
more extensive explanations for each `federation_constraints` property and
|
||||
for other general job and task options not shown here.
|
||||
|
||||
Important notes on job configuration behavioral modifications due to
|
||||
`federation_constraints`:
|
||||
|
||||
* Specifying task dependencies under `depends_on` will result in task ids
|
||||
being modified with a postfix containing a subset of the unique id
|
||||
for federations that do not require unique job ids.
|
||||
* Container registry credential information should be provided at provisioning
|
||||
time to all pools that may potentially execute that container on-behalf of
|
||||
the federation. Please see the `container_registries` constraint for more
|
||||
information.
|
||||
* Be mindful of any `shared_data_volumes` or job-level `input_data` that are
|
||||
specified in the job or corresponding tasks, and if the job should be subject
|
||||
to constraints such as `location` or `virtual_network_arm_id`.
|
||||
|
||||
#### Federated Pool Configuration
|
||||
Pools that comprise the federation should be provisioned with through placed
|
||||
into certain options that will affect scheduling of task groups.
|
||||
|
||||
* `native` under `vm_configuration`:`platform_image` will have a large impact
|
||||
on task group placement routing as task groups can only be scheduled on pools
|
||||
provisioned as their respective configuration of native or non-native. In
|
||||
general, it is recommended to use `native` container pools. Please see
|
||||
[this FAQ item](https://github.com/Azure/batch-shipyard/blob/master/docs/97-faq.md#what-is-native-under-pool-platform_image-and-custom_image)
|
||||
for more information on native vs non-native container pools.
|
||||
* `arm_image_id` under `vm_configuration`:`custom_image` will allow
|
||||
routing of task groups with `custom_image_arm_id` constraints.
|
||||
* `vm_size` will be impacted by `compute_node` job constraints.
|
||||
* `max_tasks_per_node` will impact available scheduling slots and `exclusive`
|
||||
`compute_node` constraint.
|
||||
* `autoscale` changes behavior of scheduling across various constraints.
|
||||
* `inter_node_communication` enabled pools will allow tasks that contain
|
||||
multi-instance tasks.
|
||||
* `virtual_network` proeprties will allow routing of task groups with
|
||||
`virtual_network_arm_id` constraints.
|
||||
|
||||
### Limitations
|
||||
This is a non-exhaustive list of potential limitations while using
|
||||
the federation feature in Batch Shipyard.
|
||||
|
||||
* All task dependencies must be self-contained within the task group.
|
||||
* `depends_on_range` based task dependencies are not allowed, currently.
|
||||
* `input_data:azure_batch` has restrictions. At the job-level it is not
|
||||
allowed and at the task-level it must be self-contained within the task group.
|
||||
No validation is performed at the task-level to ensure self-containment
|
||||
of input data from other Batch tasks within the task group.
|
||||
* A maximum of 14625 actions can be actively queued per unique job id.
|
||||
Actions processed by the federation proxy do not count towards this limit.
|
||||
This limit only applies to federations that allow non-unique job ids for
|
||||
job submissions (i.e., `fed jobs add`).
|
||||
* Low priority/dedicated compute node constraints are best-effort. If a task
|
||||
group is scheduled to a pool with dedicated-only nodes due to a specified
|
||||
constraint, but the pool later resizes with low priority nodes, portions of
|
||||
the task group may get scheduled to these nodes.
|
||||
* Constraints restricting low priority or dedicated execution may be
|
||||
subject to the autoscale formula supplied. It is assumed that an autoscale
|
||||
formula will scale up/down both low priority and dedicated nodes.
|
||||
* Each pool in a federation should only be associated with one and only one
|
||||
federation. Adding a pool to multiple federations simultaneously will result
|
||||
in undefined behavior.
|
||||
* Singularity containers are not fully supported in federations.
|
||||
|
||||
### Quotas
|
||||
Ensure that you have sufficient active job/job schedule quota for each
|
||||
Batch account. You may also consider increasing your pool quota if you
|
||||
intend on having many pools within a Batch account that comprise a
|
||||
federation.
|
||||
|
||||
Please note that *all* quotas (except for the number of Batch accounts
|
||||
per region per subscription) apply to each individual Batch account
|
||||
separately. User subscription based Batch accounts share the underlying
|
||||
subscription regional core quotas.
|
||||
|
||||
### Usage
|
||||
The following will describe many of the federation commands available in
|
||||
the CLI for managing the federation proxy, federations and action submission.
|
||||
Federation-related commands are grouped under the `fed` sub-command.
|
||||
|
||||
Note that most federation commands have the `--raw` option available which
|
||||
allows callers to consume the result of a command invocation in JSON
|
||||
format.
|
||||
|
||||
Please refer to the [usage documentation](20-batch-shipyard-usage.md) for
|
||||
further explanation of each federation command and other options not
|
||||
documented here.
|
||||
|
||||
#### Federation Setup
|
||||
The following list shows a typical set of steps to setup a federation. Please
|
||||
ensure that you've reviewed the prior sections and any relevant configuration
|
||||
documentation.
|
||||
|
||||
1. Construct the `federation.yaml` configuration file.
|
||||
2. Deploy pools that will comprise the federation. These can be
|
||||
autoscale-enabled pools.
|
||||
3. Deploy the federation proxy: `fed proxy create`
|
||||
4. Verify proxy: `fed proxy status`
|
||||
5. Create a federation:
|
||||
* With unique job id requirement: `fed create <fed-id>`
|
||||
* Without unique job id requirement: `fed create <fed-id> --no-unique-job-ids`
|
||||
6. Add pools to the federation:
|
||||
* With a pool configuration file: `fed pool add <fed-id>`
|
||||
* Without a pool configuration file: `fed pool add <fed-id> --pool-id <pool-id> --batch-service-url <batch-service-url>`
|
||||
7. Verify federation: `fed list`
|
||||
|
||||
A federation with a unique job id requirement means that all jobs
|
||||
submitted to the federation must have unique job ids. Job submissions which
|
||||
collide with a pre-existing job with the same job id in the federation will
|
||||
be actively rejected. This mode is recommended when federations are shared
|
||||
resources amongst a team. A federation without a unique job id requirement
|
||||
allows jobs to be submitted even if a job with the same id exists in the
|
||||
federation. The federation proxy will attempt to dynamically condition the
|
||||
action such that it can actively co-locate task groups among similarly named
|
||||
jobs. Because this requires specific unique id tracking to disambiguate task
|
||||
groups within the same job on potentially the same target pool, tracking
|
||||
task group submissions can become confusing with a team sharing a federation.
|
||||
This non-unique job id mode is only recommended for a federation with a
|
||||
single user.
|
||||
|
||||
Conversly, teardown of a federation would generally follow these steps:
|
||||
|
||||
1. Ensure all federation jobs have been deleted.
|
||||
2. Destroy the federation proxy: `fed proxy destroy`
|
||||
3. Destroy the federation: `fed destroy <fed-id>`
|
||||
|
||||
If you do not need to destroy the federation, but would like to minimze
|
||||
the cost of a federation proxy, you can instead suspend the proxy and
|
||||
re-start it at a later time through the `fed proxy suspend` and
|
||||
`fed proxy start` commands. In that case you would not destroy the
|
||||
federation metadata with the `fed destroy <fed-id>` command.
|
||||
|
||||
#### Job Lifecycle
|
||||
Federation jobs have the following commands available:
|
||||
|
||||
* Add jobs: `fed jobs add <fed-id>`
|
||||
* List jobs: `fed jobs list <fed-id>`
|
||||
* Terminate jobs: `fed jobs term <fed-id> --job-id <job-id>`
|
||||
* Delete jobs: `fed jobs del <fed-id> --job-id <job-id>`
|
||||
|
||||
When adding jobs, the specified jobs configuration and pool configuration
|
||||
(e.g., pool.yaml) are consumed. If specific `federation_constraints`
|
||||
overrides are not specified, then the federation job is created with
|
||||
settings as read from the pool configuration file. It is important
|
||||
to define job `federation_constraints` if necessary.
|
||||
|
||||
To inspect federation jobs and associated task groups, you can typically
|
||||
follow this pattern:
|
||||
|
||||
1. Locate the job: `fed jobs list <fed-id> --job-id <job-id>`
|
||||
2. Use Batch Explorer or Azure Portal to graphically manage. Or you can
|
||||
directly use Batch Shipyard commands if you have the correct Batch
|
||||
credentials populated targeting the region with the pool listed in the job
|
||||
location.
|
||||
|
||||
Sometimes a job/task group is submitted which can "block" other actions for
|
||||
the same specified job if an improper constraint or other incorrect
|
||||
configuration is specified (this can be particularly acute for non-unique job
|
||||
id federations). In this case, it is necessary to remove such problematic
|
||||
actions to unblock processing. This can be done with the command:
|
||||
`fed jobs zap <fed-id> --unique-id <uid>`.
|
|
@ -277,7 +277,7 @@ Please see this
|
|||
regarding when to choose `native` container support pools.
|
||||
|
||||
### Jobs Configuration Changes
|
||||
#### `docker_image` requires instead of `image`
|
||||
#### `docker_image` required instead of `image`
|
||||
In the tasks array, `docker_image` is now required over `image` for
|
||||
disambiguation.
|
||||
|
||||
|
|
|
@ -0,0 +1,25 @@
|
|||
# Dockerfile for Azure/batch-shipyard (Federation)
|
||||
|
||||
FROM alpine:3.7
|
||||
MAINTAINER Fred Park <https://github.com/Azure/batch-shipyard>
|
||||
|
||||
# copy in files
|
||||
COPY federation.py requirements.txt /opt/batch-shipyard/
|
||||
|
||||
# add base packages and python dependencies
|
||||
RUN apk update \
|
||||
&& apk add --update --no-cache \
|
||||
musl build-base python3 python3-dev libressl-dev libffi-dev \
|
||||
ca-certificates libressl cifs-utils bash \
|
||||
&& pip3 install --no-cache-dir --upgrade pip \
|
||||
&& pip3 install --no-cache-dir --upgrade -r /opt/batch-shipyard/requirements.txt \
|
||||
&& apk del --purge \
|
||||
build-base python3-dev libressl-dev libffi-dev \
|
||||
&& rm /var/cache/apk/* \
|
||||
&& rm -f /opt/batch-shipyard/requirements.txt
|
||||
|
||||
# pre-compile files
|
||||
RUN python3 -m compileall -f /opt/batch-shipyard
|
||||
|
||||
# set entrypoint
|
||||
ENTRYPOINT ["python3", "/opt/batch-shipyard/federation.py"]
|
|
@ -0,0 +1,13 @@
|
|||
version: "3.5"
|
||||
|
||||
services:
|
||||
federation:
|
||||
image: alfpark/batch-shipyard:{BATCH_SHIPYARD_VERSION}-federation
|
||||
container_name: federation
|
||||
restart: unless-stopped
|
||||
privileged: true
|
||||
volumes:
|
||||
- /var/batch-shipyard:/var/batch-shipyard
|
||||
command:
|
||||
- --conf
|
||||
- /var/batch-shipyard/federation.json
|
Разница между файлами не показана из-за своего большого размера
Загрузить разницу
|
@ -0,0 +1,10 @@
|
|||
azure-batch==4.1.3
|
||||
azure-cosmosdb-table==1.0.4
|
||||
azure-mgmt-compute==4.0.1
|
||||
azure-mgmt-resource==2.0.0
|
||||
azure-mgmt-storage==2.0.0rc4
|
||||
azure-storage-blob==1.3.1
|
||||
azure-storage-queue==1.3.0
|
||||
msrestazure==0.4.34
|
||||
python-dateutil==2.7.3
|
||||
requests==2.19.1
|
|
@ -618,6 +618,7 @@ def main() -> None:
|
|||
with open(args.conf, 'rb') as f:
|
||||
config = json.load(f)
|
||||
logger.debug('loaded config: {}'.format(config))
|
||||
del args
|
||||
_BATCH_SHIPYARD_VERSION = config['batch_shipyard_version']
|
||||
# convert cloud type
|
||||
cloud = convert_cloud_type(config['aad_cloud'])
|
||||
|
|
|
@ -1,8 +1,8 @@
|
|||
azure-batch==4.1.3
|
||||
azure-cosmosdb-table==1.0.4
|
||||
azure-mgmt-compute==3.1.0rc3
|
||||
azure-mgmt-network==2.0.0rc3
|
||||
azure-mgmt-resource==2.0.0rc2
|
||||
azure-mgmt-compute==4.0.1
|
||||
azure-mgmt-network==2.0.0
|
||||
azure-mgmt-resource==2.0.0
|
||||
azure-mgmt-storage==2.0.0rc4
|
||||
msrestazure==0.4.34
|
||||
requests==2.19.1
|
||||
|
|
|
@ -19,6 +19,7 @@ pages:
|
|||
- Jobs: 14-batch-shipyard-configuration-jobs.md
|
||||
- RemoteFS: 15-batch-shipyard-configuration-fs.md
|
||||
- Monitoring: 16-batch-shipyard-configuration-monitor.md
|
||||
- Federation: 17-batch-shipyard-configuration-federation.md
|
||||
- CLI Commands and Usage: 20-batch-shipyard-usage.md
|
||||
- Platform Image support: 25-batch-shipyard-platform-image-support.md
|
||||
- In-Depth Feature Guides:
|
||||
|
@ -29,6 +30,7 @@ pages:
|
|||
- Virtual Networks: 64-batch-shipyard-byovnet.md
|
||||
- Remote Filesystems: 65-batch-shipyard-remote-fs.md
|
||||
- Resource Monitoring: 66-batch-shipyard-resource-monitoring.md
|
||||
- Federations: 68-batch-shipyard-federation.md
|
||||
- Data Movement: 70-batch-shipyard-data-movement.md
|
||||
- Azure KeyVault for Credential Management: 74-batch-shipyard-azure-keyvault.md
|
||||
- Credential Encryption: 75-batch-shipyard-credential-encryption.md
|
||||
|
|
|
@ -4,12 +4,13 @@ azure-cosmosdb-table==1.0.4
|
|||
azure-keyvault==1.0.0
|
||||
azure-mgmt-authorization==0.50.0
|
||||
azure-mgmt-batch==5.0.1
|
||||
azure-mgmt-compute==3.1.0rc3
|
||||
azure-mgmt-network==2.0.0rc3
|
||||
azure-mgmt-resource==2.0.0rc2
|
||||
azure-mgmt-compute==4.0.1
|
||||
azure-mgmt-network==2.0.0
|
||||
azure-mgmt-resource==2.0.0
|
||||
azure-mgmt-storage==2.0.0rc4
|
||||
azure-storage-blob==1.3.1
|
||||
azure-storage-file==1.3.1
|
||||
azure-storage-queue==1.3.0
|
||||
blobxfer==1.3.1
|
||||
click==6.7
|
||||
future==0.16.0
|
||||
|
@ -20,5 +21,5 @@ msrestazure==0.4.34
|
|||
pathlib2==2.3.2; python_version < '3.5'
|
||||
python-dateutil==2.7.3
|
||||
requests==2.19.1
|
||||
ruamel.yaml==0.15.44
|
||||
ruamel.yaml==0.15.46
|
||||
scandir==1.7; python_version < '3.5'
|
||||
|
|
|
@ -0,0 +1,124 @@
|
|||
desc: Federation Configuration Schema
|
||||
|
||||
type: map
|
||||
mapping:
|
||||
federation:
|
||||
type: map
|
||||
mapping:
|
||||
storage_account_settings:
|
||||
type: str
|
||||
required: true
|
||||
location:
|
||||
type: str
|
||||
required: true
|
||||
resource_group:
|
||||
type: str
|
||||
required: true
|
||||
hostname_prefix:
|
||||
type: str
|
||||
required: true
|
||||
ssh:
|
||||
type: map
|
||||
required: true
|
||||
mapping:
|
||||
username:
|
||||
type: str
|
||||
required: true
|
||||
ssh_public_key:
|
||||
type: str
|
||||
ssh_public_key_data:
|
||||
type: str
|
||||
ssh_private_key:
|
||||
type: str
|
||||
generated_file_export_path:
|
||||
type: str
|
||||
public_ip:
|
||||
type: map
|
||||
mapping:
|
||||
enabled:
|
||||
type: bool
|
||||
static:
|
||||
type: bool
|
||||
virtual_network:
|
||||
type: map
|
||||
required: true
|
||||
mapping:
|
||||
name:
|
||||
type: str
|
||||
required: true
|
||||
resource_group:
|
||||
type: str
|
||||
existing_ok:
|
||||
type: bool
|
||||
address_space:
|
||||
type: str
|
||||
subnet:
|
||||
type: map
|
||||
mapping:
|
||||
name:
|
||||
type: str
|
||||
required: true
|
||||
address_prefix:
|
||||
type: str
|
||||
required: true
|
||||
network_security:
|
||||
type: map
|
||||
required: true
|
||||
mapping:
|
||||
ssh:
|
||||
type: seq
|
||||
required: true
|
||||
sequence:
|
||||
- type: str
|
||||
custom_inbound_rules:
|
||||
type: map
|
||||
mapping:
|
||||
regex;([a-zA-Z0-9]+):
|
||||
type: map
|
||||
mapping:
|
||||
destination_port_range:
|
||||
type: str
|
||||
required: true
|
||||
protocol:
|
||||
type: str
|
||||
enum: ['*', 'tcp', 'udp']
|
||||
source_address_prefix:
|
||||
type: seq
|
||||
required: true
|
||||
sequence:
|
||||
- type: str
|
||||
vm_size:
|
||||
type: str
|
||||
required: true
|
||||
accelerated_networking:
|
||||
type: bool
|
||||
proxy_options:
|
||||
type: map
|
||||
mapping:
|
||||
polling_interval:
|
||||
type: map
|
||||
mapping:
|
||||
federations:
|
||||
type: int
|
||||
actions:
|
||||
type: int
|
||||
logging:
|
||||
type: map
|
||||
mapping:
|
||||
persistence:
|
||||
type: bool
|
||||
level:
|
||||
type: str
|
||||
enum: ['debug', 'info', 'warning', 'error', 'critical']
|
||||
filename:
|
||||
type: str
|
||||
scheduling:
|
||||
type: map
|
||||
mapping:
|
||||
after_success:
|
||||
type: map
|
||||
mapping:
|
||||
blackout_interval:
|
||||
type: int
|
||||
evaluate_autoscale:
|
||||
type: bool
|
|
@ -155,6 +155,85 @@ mapping:
|
|||
default_working_dir:
|
||||
type: str
|
||||
enum: ['batch', 'container']
|
||||
federation_constraints:
|
||||
type: map
|
||||
mapping:
|
||||
pool:
|
||||
type: map
|
||||
mapping:
|
||||
autoscale:
|
||||
type: map
|
||||
mapping:
|
||||
allow:
|
||||
type: bool
|
||||
required: true
|
||||
exclusive:
|
||||
type: bool
|
||||
required: true
|
||||
low_priority_nodes:
|
||||
type: map
|
||||
mapping:
|
||||
allow:
|
||||
type: bool
|
||||
required: true
|
||||
exclusive:
|
||||
type: bool
|
||||
required: true
|
||||
native:
|
||||
type: bool
|
||||
windows:
|
||||
type: bool
|
||||
location:
|
||||
type: str
|
||||
container_registries:
|
||||
type: map
|
||||
mapping:
|
||||
private_docker_hub:
|
||||
type: bool
|
||||
public:
|
||||
type: seq
|
||||
sequence:
|
||||
- type: str
|
||||
max_active_task_backlog:
|
||||
type: map
|
||||
mapping:
|
||||
ratio:
|
||||
type: float
|
||||
required: true
|
||||
autoscale_exempt:
|
||||
type: bool
|
||||
required: true
|
||||
custom_image_arm_id:
|
||||
type: str
|
||||
virtual_network_arm_id:
|
||||
type: str
|
||||
compute_node:
|
||||
type: map
|
||||
mapping:
|
||||
vm_size:
|
||||
type: str
|
||||
cores:
|
||||
type: map
|
||||
mapping:
|
||||
amount:
|
||||
type: int
|
||||
required: true
|
||||
schedulable_variance:
|
||||
type: float
|
||||
memory:
|
||||
type: map
|
||||
mapping:
|
||||
amount:
|
||||
type: str
|
||||
required: true
|
||||
schedulable_variance:
|
||||
type: float
|
||||
exclusive:
|
||||
type: bool
|
||||
gpu:
|
||||
type: bool
|
||||
infiniband:
|
||||
type: bool
|
||||
tasks:
|
||||
type: seq
|
||||
sequence:
|
||||
|
|
|
@ -0,0 +1,392 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
# shellcheck disable=SC1091
|
||||
|
||||
set -e
|
||||
set -o pipefail
|
||||
|
||||
# version consts
|
||||
DOCKER_CE_VERSION_DEBIAN=18.03.1
|
||||
|
||||
# consts
|
||||
DOCKER_CE_PACKAGE_DEBIAN="docker-ce=${DOCKER_CE_VERSION_DEBIAN}~ce~3-0~"
|
||||
SHIPYARD_VAR_DIR=/var/batch-shipyard
|
||||
SHIPYARD_CONF_FILE=${SHIPYARD_VAR_DIR}/federation.json
|
||||
|
||||
log() {
|
||||
local level=$1
|
||||
shift
|
||||
echo "$(date -u -Ins) - $level - $*"
|
||||
}
|
||||
|
||||
# dump uname immediately
|
||||
uname -ar
|
||||
|
||||
# try to get /etc/lsb-release
|
||||
if [ -e /etc/lsb-release ]; then
|
||||
. /etc/lsb-release
|
||||
else
|
||||
if [ -e /etc/os-release ]; then
|
||||
. /etc/os-release
|
||||
DISTRIB_ID=$ID
|
||||
DISTRIB_RELEASE=$VERSION_ID
|
||||
fi
|
||||
fi
|
||||
if [ -z ${DISTRIB_ID+x} ] || [ -z ${DISTRIB_RELEASE+x} ]; then
|
||||
log ERROR "Unknown DISTRIB_ID or DISTRIB_RELEASE."
|
||||
exit 1
|
||||
fi
|
||||
DISTRIB_ID=${DISTRIB_ID,,}
|
||||
DISTRIB_RELEASE=${DISTRIB_RELEASE,,}
|
||||
|
||||
# set distribution specific vars
|
||||
PACKAGER=
|
||||
if [ "$DISTRIB_ID" == "ubuntu" ]; then
|
||||
PACKAGER=apt
|
||||
elif [ "$DISTRIB_ID" == "debian" ]; then
|
||||
PACKAGER=apt
|
||||
elif [[ $DISTRIB_ID == centos* ]] || [ "$DISTRIB_ID" == "rhel" ]; then
|
||||
PACKAGER=yum
|
||||
else
|
||||
PACKAGER=zypper
|
||||
fi
|
||||
if [ "$PACKAGER" == "apt" ]; then
|
||||
export DEBIAN_FRONTEND=noninteractive
|
||||
fi
|
||||
|
||||
# globals
|
||||
aad_cloud=
|
||||
log_level=
|
||||
log_persist=
|
||||
log_filename=
|
||||
post_success_blackout=
|
||||
post_success_autoscale=
|
||||
prefix=
|
||||
refresh_interval_actions=5
|
||||
refresh_interval_federation=30
|
||||
storage_account=
|
||||
storage_rg=
|
||||
shipyardversion=
|
||||
|
||||
# process command line options
|
||||
while getopts "h?a:l:p:r:s:v:" opt; do
|
||||
case "$opt" in
|
||||
h|\?)
|
||||
echo "shipyard_federation_bootstrap.sh parameters"
|
||||
echo ""
|
||||
echo "-a [aad cloud type] AAD cloud type for MSI"
|
||||
echo "-l [persistence:level:filename] logging options"
|
||||
echo "-p [blackout interval:evaluate autoscale] post-success scheduling options"
|
||||
echo "-r [federation:actions] refresh intervals"
|
||||
echo "-s [storage account:resource group:prefix] storage config"
|
||||
echo "-v [version] batch-shipyard version"
|
||||
echo ""
|
||||
exit 1
|
||||
;;
|
||||
a)
|
||||
aad_cloud=${OPTARG,,}
|
||||
;;
|
||||
p)
|
||||
IFS=':' read -ra p <<< "${OPTARG}"
|
||||
post_success_blackout=${p[0]}
|
||||
post_success_autoscale=${p[1],,}
|
||||
;;
|
||||
l)
|
||||
IFS=':' read -ra lo <<< "${OPTARG}"
|
||||
log_persist=${lo[0],,}
|
||||
log_level=${lo[1],,}
|
||||
log_filename=${lo[2]}
|
||||
;;
|
||||
r)
|
||||
IFS=':' read -ra ri <<< "${OPTARG,,}"
|
||||
refresh_interval_federation=${ri[0]}
|
||||
refresh_interval_actions=${ri[1]}
|
||||
;;
|
||||
s)
|
||||
IFS=':' read -ra ss <<< "${OPTARG,,}"
|
||||
storage_account=${ss[0]}
|
||||
storage_rg=${ss[1]}
|
||||
prefix=${ss[2]}
|
||||
;;
|
||||
v)
|
||||
shipyardversion=$OPTARG
|
||||
;;
|
||||
esac
|
||||
done
|
||||
shift $((OPTIND-1))
|
||||
[ "$1" = "--" ] && shift
|
||||
# check required params
|
||||
if [ -z "$aad_cloud" ]; then
|
||||
log ERROR "AAD cloud type not specified"
|
||||
exit 1
|
||||
fi
|
||||
|
||||
check_for_buggy_ntfs_mount() {
|
||||
# Check to ensure sdb1 mount is not mounted as ntfs
|
||||
set +e
|
||||
mount | grep /dev/sdb1 | grep fuseblk
|
||||
local rc=$?
|
||||
set -e
|
||||
if [ $rc -eq 0 ]; then
|
||||
log ERROR "/dev/sdb1 temp disk is mounted as fuseblk/ntfs"
|
||||
exit 1
|
||||
fi
|
||||
}
|
||||
|
||||
download_file_as() {
|
||||
log INFO "Downloading: $1 as $2"
|
||||
local retries=10
|
||||
set +e
|
||||
while [ $retries -gt 0 ]; do
|
||||
if curl -fSsL "$1" -o "$2"; then
|
||||
break
|
||||
fi
|
||||
retries=$((retries-1))
|
||||
if [ $retries -eq 0 ]; then
|
||||
log ERROR "Could not download: $1"
|
||||
exit 1
|
||||
fi
|
||||
sleep 1
|
||||
done
|
||||
set -e
|
||||
}
|
||||
|
||||
add_repo() {
|
||||
local url=$1
|
||||
set +e
|
||||
local retries=120
|
||||
local rc
|
||||
while [ $retries -gt 0 ]; do
|
||||
if [ "$PACKAGER" == "apt" ]; then
|
||||
curl -fSsL "$url" | apt-key add -
|
||||
rc=$?
|
||||
elif [ "$PACKAGER" == "yum" ]; then
|
||||
yum-config-manager --add-repo "$url"
|
||||
rc=$?
|
||||
elif [ "$PACKAGER" == "zypper" ]; then
|
||||
zypper addrepo "$url"
|
||||
rc=$?
|
||||
fi
|
||||
if [ $rc -eq 0 ]; then
|
||||
break
|
||||
fi
|
||||
retries=$((retries-1))
|
||||
if [ $retries -eq 0 ]; then
|
||||
log ERROR "Could not add repo: $url"
|
||||
exit 1
|
||||
fi
|
||||
sleep 1
|
||||
done
|
||||
set -e
|
||||
}
|
||||
|
||||
refresh_package_index() {
|
||||
set +e
|
||||
local retries=120
|
||||
local rc
|
||||
while [ $retries -gt 0 ]; do
|
||||
if [ "$PACKAGER" == "apt" ]; then
|
||||
apt-get update
|
||||
rc=$?
|
||||
elif [ "$PACKAGER" == "yum" ]; then
|
||||
yum makecache -y fast
|
||||
rc=$?
|
||||
elif [ "$PACKAGER" == "zypper" ]; then
|
||||
zypper -n --gpg-auto-import-keys ref
|
||||
rc=$?
|
||||
fi
|
||||
if [ $rc -eq 0 ]; then
|
||||
break
|
||||
fi
|
||||
retries=$((retries-1))
|
||||
if [ $retries -eq 0 ]; then
|
||||
log ERROR "Could not update package index"
|
||||
exit 1
|
||||
fi
|
||||
sleep 1
|
||||
done
|
||||
set -e
|
||||
}
|
||||
|
||||
install_packages() {
|
||||
set +e
|
||||
local retries=120
|
||||
local rc
|
||||
while [ $retries -gt 0 ]; do
|
||||
if [ "$PACKAGER" == "apt" ]; then
|
||||
apt-get install -y -q -o Dpkg::Options::="--force-confnew" --no-install-recommends "$@"
|
||||
rc=$?
|
||||
elif [ "$PACKAGER" == "yum" ]; then
|
||||
yum install -y "$@"
|
||||
rc=$?
|
||||
elif [ "$PACKAGER" == "zypper" ]; then
|
||||
zypper -n in "$@"
|
||||
rc=$?
|
||||
fi
|
||||
if [ $rc -eq 0 ]; then
|
||||
break
|
||||
fi
|
||||
retries=$((retries-1))
|
||||
if [ $retries -eq 0 ]; then
|
||||
log ERROR "Could not install packages ($PACKAGER): $*"
|
||||
exit 1
|
||||
fi
|
||||
sleep 1
|
||||
done
|
||||
set -e
|
||||
}
|
||||
|
||||
create_batch_shipyard_federation_config() {
|
||||
mkdir -p ${SHIPYARD_VAR_DIR}
|
||||
chmod 755 ${SHIPYARD_VAR_DIR}
|
||||
cat > ${SHIPYARD_CONF_FILE} << EOF
|
||||
{
|
||||
"aad_cloud": "$aad_cloud",
|
||||
"storage": {
|
||||
"account": "$storage_account",
|
||||
"resource_group": "$storage_rg",
|
||||
"entity_prefix": "$prefix"
|
||||
},
|
||||
"batch_shipyard": {
|
||||
"var_path": "$SHIPYARD_VAR_DIR",
|
||||
"version": "$shipyardversion"
|
||||
},
|
||||
"logging": {
|
||||
"persistence": $log_persist,
|
||||
"level": "$log_level",
|
||||
"filename": "$log_filename"
|
||||
},
|
||||
"refresh_intervals": {
|
||||
"federations": $refresh_interval_federation,
|
||||
"actions": $refresh_interval_actions
|
||||
},
|
||||
"scheduling": {
|
||||
"after_success": {
|
||||
"blackout_interval": $post_success_blackout,
|
||||
"evaluate_autoscale": $post_success_autoscale
|
||||
}
|
||||
}
|
||||
}
|
||||
EOF
|
||||
log INFO "Batch Shipyard federation config created"
|
||||
}
|
||||
|
||||
install_docker_host_engine() {
|
||||
log DEBUG "Installing Docker Host Engine"
|
||||
# set vars
|
||||
if [ "$PACKAGER" == "apt" ]; then
|
||||
local repo=https://download.docker.com/linux/"${DISTRIB_ID}"
|
||||
local gpgkey="${repo}"/gpg
|
||||
local dockerversion="${DOCKER_CE_PACKAGE_DEBIAN}${DISTRIB_ID}"
|
||||
local prereq_pkgs="apt-transport-https ca-certificates curl gnupg2 software-properties-common"
|
||||
elif [ "$PACKAGER" == "yum" ]; then
|
||||
local repo=https://download.docker.com/linux/centos/docker-ce.repo
|
||||
local dockerversion="${DOCKER_CE_PACKAGE_CENTOS}"
|
||||
local prereq_pkgs="yum-utils device-mapper-persistent-data lvm2"
|
||||
elif [ "$PACKAGER" == "zypper" ]; then
|
||||
if [[ "$DISTRIB_RELEASE" == 12-sp3* ]]; then
|
||||
local repodir=SLE_12_SP3
|
||||
fi
|
||||
local repo="http://download.opensuse.org/repositories/Virtualization:containers/${repodir}/Virtualization:containers.repo"
|
||||
local dockerversion="${DOCKER_CE_PACKAGE_SLES}"
|
||||
fi
|
||||
# refresh package index
|
||||
refresh_package_index
|
||||
# install required software first
|
||||
# shellcheck disable=SC2086
|
||||
install_packages $prereq_pkgs
|
||||
if [ "$PACKAGER" == "apt" ]; then
|
||||
# add gpgkey for repo
|
||||
add_repo "$gpgkey"
|
||||
# add repo
|
||||
add-apt-repository "deb [arch=amd64] $repo $(lsb_release -cs) stable"
|
||||
else
|
||||
add_repo "$repo"
|
||||
fi
|
||||
# refresh index
|
||||
refresh_package_index
|
||||
# install docker engine
|
||||
install_packages "$dockerversion"
|
||||
systemctl start docker.service
|
||||
systemctl enable docker.service
|
||||
systemctl --no-pager status docker.service
|
||||
docker info
|
||||
log INFO "Docker Host Engine installed"
|
||||
# install docker-compose
|
||||
install_packages python3-pip python3-distutils apache2-utils
|
||||
pip3 install --upgrade setuptools wheel
|
||||
pip3 install docker-compose
|
||||
log INFO "Docker-compose installed"
|
||||
}
|
||||
|
||||
setup_docker_compose_systemd() {
|
||||
# create systemd area for docker compose
|
||||
mkdir -p /etc/docker/compose/batch-shipyard-federation
|
||||
chmod 644 docker-compose.yml
|
||||
cp docker-compose.yml /etc/docker/compose/batch-shipyard-federation/
|
||||
# substitute batch shipyard version
|
||||
sed -i "s/{BATCH_SHIPYARD_VERSION}/$shipyardversion/g" /etc/docker/compose/batch-shipyard-federation/docker-compose.yml
|
||||
# create systemd unit file
|
||||
cat << EOF > /etc/systemd/system/docker-compose@.service
|
||||
[Unit]
|
||||
Description=%i service with docker compose
|
||||
Requires=docker.service
|
||||
After=docker.service
|
||||
|
||||
[Service]
|
||||
Restart=always
|
||||
|
||||
WorkingDirectory=/etc/docker/compose/%i
|
||||
|
||||
# Remove old containers, images and volumes
|
||||
ExecStartPre=/usr/local/bin/docker-compose down -v
|
||||
ExecStartPre=/usr/local/bin/docker-compose rm -fv
|
||||
ExecStartPre=-/bin/bash -c 'docker volume ls -qf "name=%i_" | xargs -r docker volume rm'
|
||||
ExecStartPre=-/bin/bash -c 'docker network ls -qf "name=%i_" | xargs -r docker network rm'
|
||||
ExecStartPre=-/bin/bash -c 'docker ps -aqf "name=%i_*" | xargs -r docker rm'
|
||||
|
||||
# Compose up
|
||||
ExecStart=/usr/local/bin/docker-compose up
|
||||
|
||||
# Compose down, remove containers and volumes
|
||||
ExecStop=/usr/local/bin/docker-compose down -v
|
||||
|
||||
[Install]
|
||||
WantedBy=multi-user.target
|
||||
EOF
|
||||
log INFO "systemd unit files for docker compose installed"
|
||||
}
|
||||
|
||||
log INFO "Bootstrap start"
|
||||
echo "Configuration:"
|
||||
echo "--------------"
|
||||
echo "OS Distribution: $DISTRIB_ID $DISTRIB_RELEASE"
|
||||
echo "Batch Shipyard version: $shipyardversion"
|
||||
echo "AAD cloud: $aad_cloud"
|
||||
echo "Storage: $storage_account:$storage_rg:$prefix"
|
||||
echo "Refresh intervals: feds=$refresh_interval_federation actions=$refresh_interval_actions"
|
||||
echo "Logging: persist=$log_persist level=$log_level filename=$log_filename"
|
||||
echo ""
|
||||
|
||||
# check sdb1 mount
|
||||
check_for_buggy_ntfs_mount
|
||||
|
||||
# set sudoers to not require tty
|
||||
sed -i 's/^Defaults[ ]*requiretty/# Defaults requiretty/g' /etc/sudoers
|
||||
|
||||
# write batch shipyard config
|
||||
create_batch_shipyard_federation_config
|
||||
|
||||
# install docker host engine and docker compose
|
||||
install_docker_host_engine
|
||||
|
||||
# setup docker compose on startup
|
||||
setup_docker_compose_systemd
|
||||
|
||||
# start and enable services
|
||||
systemctl daemon-reload
|
||||
systemctl start docker-compose@batch-shipyard-federation
|
||||
systemctl enable docker-compose@batch-shipyard-federation
|
||||
systemctl --no-pager status docker-compose@batch-shipyard-federation
|
||||
|
||||
log INFO "Bootstrap completed"
|
|
@ -79,7 +79,7 @@ while getopts "h?a:d:flp:s:v:" opt; do
|
|||
echo "-d [fqdn] fully qualified domain name"
|
||||
echo "-f use staging LE ACME CSR (fake)"
|
||||
echo "-l enable lets encrypt"
|
||||
echo "-p [polling interval] pollint interval"
|
||||
echo "-p [polling interval] polling interval"
|
||||
echo "-s [storage account:table name:resource group] monitoring table"
|
||||
echo "-v [version] batch-shipyard version"
|
||||
echo ""
|
||||
|
|
447
shipyard.py
447
shipyard.py
|
@ -66,11 +66,13 @@ class CliContext(object):
|
|||
self.conf_jobs = None
|
||||
self.conf_fs = None
|
||||
self.conf_monitor = None
|
||||
self.conf_federation = None
|
||||
# clients
|
||||
self.batch_mgmt_client = None
|
||||
self.batch_client = None
|
||||
self.blob_client = None
|
||||
self.table_client = None
|
||||
self.queue_client = None
|
||||
self.keyvault_client = None
|
||||
self.auth_client = None
|
||||
self.resource_client = None
|
||||
|
@ -150,7 +152,8 @@ class CliContext(object):
|
|||
self._init_keyvault_client()
|
||||
self._init_config(
|
||||
skip_global_config=False, skip_pool_config=True,
|
||||
skip_monitor_config=True, fs_storage=True)
|
||||
skip_monitor_config=True, skip_federation_config=True,
|
||||
fs_storage=True)
|
||||
self._ensure_credentials_section('storage')
|
||||
_, self.resource_client, self.compute_client, self.network_client, \
|
||||
self.storage_mgmt_client, _, _ = \
|
||||
|
@ -158,7 +161,7 @@ class CliContext(object):
|
|||
# inject storage account keys if via aad
|
||||
convoy.fleet.fetch_storage_account_keys_from_aad(
|
||||
self.storage_mgmt_client, self.config, fs_storage=True)
|
||||
self.blob_client, _ = convoy.clients.create_storage_clients()
|
||||
self.blob_client, _, _ = convoy.clients.create_storage_clients()
|
||||
self._cleanup_after_initialize()
|
||||
|
||||
def initialize_for_monitor(self):
|
||||
|
@ -173,7 +176,8 @@ class CliContext(object):
|
|||
self._init_keyvault_client()
|
||||
self._init_config(
|
||||
skip_global_config=False, skip_pool_config=True,
|
||||
skip_monitor_config=False, fs_storage=True)
|
||||
skip_monitor_config=False, skip_federation_config=True,
|
||||
fs_storage=True)
|
||||
self._ensure_credentials_section('storage')
|
||||
self._ensure_credentials_section('monitoring')
|
||||
self.auth_client, self.resource_client, self.compute_client, \
|
||||
|
@ -182,7 +186,38 @@ class CliContext(object):
|
|||
# inject storage account keys if via aad
|
||||
convoy.fleet.fetch_storage_account_keys_from_aad(
|
||||
self.storage_mgmt_client, self.config, fs_storage=True)
|
||||
self.blob_client, self.table_client = \
|
||||
self.blob_client, self.table_client, _ = \
|
||||
convoy.clients.create_storage_clients()
|
||||
self._cleanup_after_initialize()
|
||||
|
||||
def initialize_for_federation(self, init_batch=False):
|
||||
# type: (CliContext, bool) -> None
|
||||
"""Initialize context for fed commands
|
||||
:param CliContext self: this
|
||||
:param bool init_batch: initialize batch
|
||||
"""
|
||||
self._read_credentials_config()
|
||||
self._set_global_cli_options()
|
||||
if self.verbose:
|
||||
logger.debug('initializing for fed actions')
|
||||
self._init_keyvault_client()
|
||||
self._init_config(
|
||||
skip_global_config=False, skip_pool_config=not init_batch,
|
||||
skip_monitor_config=True, skip_federation_config=False,
|
||||
fs_storage=not init_batch)
|
||||
self._ensure_credentials_section('storage')
|
||||
self.auth_client, self.resource_client, self.compute_client, \
|
||||
self.network_client, self.storage_mgmt_client, _, \
|
||||
self.batch_client = convoy.clients.create_all_clients(
|
||||
self, batch_clients=init_batch)
|
||||
# inject storage account keys if via aad
|
||||
convoy.fleet.fetch_storage_account_keys_from_aad(
|
||||
self.storage_mgmt_client, self.config, fs_storage=not init_batch)
|
||||
# call populate global settings again to adjust for federation storage
|
||||
sc = convoy.settings.federation_credentials_storage(self.config)
|
||||
convoy.fleet.populate_global_settings(
|
||||
self.config, fs_storage=not init_batch, sc=sc)
|
||||
self.blob_client, self.table_client, self.queue_client = \
|
||||
convoy.clients.create_storage_clients()
|
||||
self._cleanup_after_initialize()
|
||||
|
||||
|
@ -198,7 +233,8 @@ class CliContext(object):
|
|||
self._init_keyvault_client()
|
||||
self._init_config(
|
||||
skip_global_config=True, skip_pool_config=True,
|
||||
skip_monitor_config=True, fs_storage=False)
|
||||
skip_monitor_config=True, skip_federation_config=True,
|
||||
fs_storage=False)
|
||||
# do not perform keyvault credentials section check as all
|
||||
# options can be specified off the cli, validity of the keyvault
|
||||
# client will be checked later
|
||||
|
@ -216,7 +252,8 @@ class CliContext(object):
|
|||
self._init_keyvault_client()
|
||||
self._init_config(
|
||||
skip_global_config=False, skip_pool_config=False,
|
||||
skip_monitor_config=True, fs_storage=False)
|
||||
skip_monitor_config=True, skip_federation_config=True,
|
||||
fs_storage=False)
|
||||
self._ensure_credentials_section('storage')
|
||||
self._ensure_credentials_section('batch')
|
||||
_, self.resource_client, self.compute_client, self.network_client, \
|
||||
|
@ -226,7 +263,7 @@ class CliContext(object):
|
|||
# inject storage account keys if via aad
|
||||
convoy.fleet.fetch_storage_account_keys_from_aad(
|
||||
self.storage_mgmt_client, self.config, fs_storage=False)
|
||||
self.blob_client, self.table_client = \
|
||||
self.blob_client, self.table_client, _ = \
|
||||
convoy.clients.create_storage_clients()
|
||||
self._cleanup_after_initialize()
|
||||
|
||||
|
@ -242,14 +279,15 @@ class CliContext(object):
|
|||
self._init_keyvault_client()
|
||||
self._init_config(
|
||||
skip_global_config=False, skip_pool_config=False,
|
||||
skip_monitor_config=True, fs_storage=False)
|
||||
skip_monitor_config=True, skip_federation_config=True,
|
||||
fs_storage=False)
|
||||
self._ensure_credentials_section('storage')
|
||||
# inject storage account keys if via aad
|
||||
_, _, _, _, self.storage_mgmt_client, _, _ = \
|
||||
convoy.clients.create_all_clients(self)
|
||||
convoy.fleet.fetch_storage_account_keys_from_aad(
|
||||
self.storage_mgmt_client, self.config, fs_storage=False)
|
||||
self.blob_client, self.table_client = \
|
||||
self.blob_client, self.table_client, _ = \
|
||||
convoy.clients.create_storage_clients()
|
||||
self._cleanup_after_initialize()
|
||||
|
||||
|
@ -280,6 +318,7 @@ class CliContext(object):
|
|||
del self.conf_pool
|
||||
del self.conf_jobs
|
||||
del self.conf_monitor
|
||||
del self.conf_federation
|
||||
# free cli options
|
||||
del self.verbose
|
||||
del self.yes
|
||||
|
@ -352,13 +391,15 @@ class CliContext(object):
|
|||
|
||||
def _init_config(
|
||||
self, skip_global_config=False, skip_pool_config=False,
|
||||
skip_monitor_config=True, fs_storage=False):
|
||||
# type: (CliContext, bool, bool, bool, bool) -> None
|
||||
skip_monitor_config=True, skip_federation_config=True,
|
||||
fs_storage=False):
|
||||
# type: (CliContext, bool, bool, bool, bool, bool) -> None
|
||||
"""Initializes configuration of the context
|
||||
:param CliContext self: this
|
||||
:param bool skip_global_config: skip global config
|
||||
:param bool skip_pool_config: skip pool config
|
||||
:param bool skip_monitor_config: skip monitoring config
|
||||
:param bool skip_federation_config: skip federation config
|
||||
:param bool fs_storage: adjust storage settings for fs
|
||||
"""
|
||||
# reset config
|
||||
|
@ -408,6 +449,16 @@ class CliContext(object):
|
|||
self.conf_monitor)
|
||||
convoy.validator.validate_config(
|
||||
convoy.validator.ConfigType.Monitor, self.conf_monitor)
|
||||
# set/validate federation config
|
||||
if not skip_federation_config:
|
||||
self.conf_federation = self._form_conf_path(
|
||||
self.conf_federation, 'federation')
|
||||
if self.conf_federation is None:
|
||||
raise ValueError('federation conf file was not specified')
|
||||
self.conf_federation = CliContext.ensure_pathlib_conf(
|
||||
self.conf_federation)
|
||||
convoy.validator.validate_config(
|
||||
convoy.validator.ConfigType.Federation, self.conf_federation)
|
||||
# fetch credentials from keyvault, if conf file is missing
|
||||
kvcreds = None
|
||||
if self.conf_credentials is None or not self.conf_credentials.exists():
|
||||
|
@ -458,6 +509,8 @@ class CliContext(object):
|
|||
self._read_config_file(self.conf_jobs)
|
||||
if not skip_monitor_config:
|
||||
self._read_config_file(self.conf_monitor)
|
||||
if not skip_federation_config:
|
||||
self._read_config_file(self.conf_federation)
|
||||
# adjust settings
|
||||
convoy.fleet.initialize_globals(convoy.settings.verbose(self.config))
|
||||
if not skip_global_config:
|
||||
|
@ -768,6 +821,19 @@ def _jobs_option(f):
|
|||
callback=callback)(f)
|
||||
|
||||
|
||||
def federation_option(f):
|
||||
def callback(ctx, param, value):
|
||||
clictx = ctx.ensure_object(CliContext)
|
||||
clictx.conf_federation = value
|
||||
return value
|
||||
return click.option(
|
||||
'--federation',
|
||||
expose_value=False,
|
||||
envvar='SHIPYARD_FEDERATION_CONF',
|
||||
help='Federation config file',
|
||||
callback=callback)(f)
|
||||
|
||||
|
||||
def fs_option(f):
|
||||
def callback(ctx, param, value):
|
||||
clictx = ctx.ensure_object(CliContext)
|
||||
|
@ -858,6 +924,12 @@ def monitor_options(f):
|
|||
return f
|
||||
|
||||
|
||||
def federation_options(f):
|
||||
f = federation_option(f)
|
||||
f = _azure_subscription_id_option(f)
|
||||
return f
|
||||
|
||||
|
||||
@click.group(context_settings=_CONTEXT_SETTINGS)
|
||||
@click.version_option(version=convoy.__version__)
|
||||
@click.pass_context
|
||||
|
@ -2322,6 +2394,19 @@ def monitor_start(ctx, no_wait):
|
|||
ctx.compute_client, ctx.config, not no_wait)
|
||||
|
||||
|
||||
@monitor.command('status')
|
||||
@common_options
|
||||
@monitor_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def monitor_status(ctx):
|
||||
"""Query status of a monitoring resource"""
|
||||
ctx.initialize_for_monitor()
|
||||
convoy.fleet.action_monitor_status(
|
||||
ctx.compute_client, ctx.network_client, ctx.config)
|
||||
|
||||
|
||||
@monitor.command('destroy')
|
||||
@click.option(
|
||||
'--delete-resource-group', is_flag=True,
|
||||
|
@ -2349,6 +2434,346 @@ def monitor_destroy(
|
|||
delete_virtual_network, generate_from_prefix, not no_wait)
|
||||
|
||||
|
||||
@cli.group()
|
||||
@pass_cli_context
|
||||
def fed(ctx):
|
||||
"""Federation actions"""
|
||||
pass
|
||||
|
||||
|
||||
@fed.group()
|
||||
@pass_cli_context
|
||||
def proxy(ctx):
|
||||
"""Federation proxy actions"""
|
||||
pass
|
||||
|
||||
|
||||
@proxy.command('create')
|
||||
@common_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_proxy_create(ctx):
|
||||
"""Create a federation proxy"""
|
||||
ctx.initialize_for_federation()
|
||||
convoy.fleet.action_fed_proxy_create(
|
||||
ctx.auth_client, ctx.resource_client, ctx.compute_client,
|
||||
ctx.network_client, ctx.blob_client, ctx.table_client,
|
||||
ctx.queue_client, ctx.config)
|
||||
|
||||
|
||||
@proxy.command('ssh')
|
||||
@click.option(
|
||||
'--tty', is_flag=True, help='Allocate a pseudo-tty')
|
||||
@common_options
|
||||
@federation_options
|
||||
@click.argument('command', nargs=-1)
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_proxy_ssh(ctx, tty, command):
|
||||
"""Interactively login via SSH to federation proxy virtual
|
||||
machine in Azure"""
|
||||
ctx.initialize_for_federation()
|
||||
convoy.fleet.action_fed_proxy_ssh(
|
||||
ctx.compute_client, ctx.network_client, ctx.config, tty, command)
|
||||
|
||||
|
||||
@proxy.command('suspend')
|
||||
@click.option(
|
||||
'--no-wait', is_flag=True, help='Do not wait for suspension to complete')
|
||||
@common_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_proxy_suspend(ctx, no_wait):
|
||||
"""Suspend a federation proxy"""
|
||||
ctx.initialize_for_federation()
|
||||
convoy.fleet.action_fed_proxy_suspend(
|
||||
ctx.compute_client, ctx.config, not no_wait)
|
||||
|
||||
|
||||
@proxy.command('start')
|
||||
@click.option(
|
||||
'--no-wait', is_flag=True, help='Do not wait for restart to complete')
|
||||
@common_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_proxy_start(ctx, no_wait):
|
||||
"""Starts a previously suspended federation proxy"""
|
||||
ctx.initialize_for_federation()
|
||||
convoy.fleet.action_fed_proxy_start(
|
||||
ctx.compute_client, ctx.config, not no_wait)
|
||||
|
||||
|
||||
@proxy.command('status')
|
||||
@common_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_proxy_status(ctx):
|
||||
"""Query status of a federation proxy"""
|
||||
ctx.initialize_for_federation()
|
||||
convoy.fleet.action_fed_proxy_status(
|
||||
ctx.compute_client, ctx.network_client, ctx.config)
|
||||
|
||||
|
||||
@proxy.command('destroy')
|
||||
@click.option(
|
||||
'--delete-resource-group', is_flag=True,
|
||||
help='Delete all resources in the federation resource group')
|
||||
@click.option(
|
||||
'--delete-virtual-network', is_flag=True, help='Delete virtual network')
|
||||
@click.option(
|
||||
'--generate-from-prefix', is_flag=True,
|
||||
help='Generate resources to delete from federation hostname prefix')
|
||||
@click.option(
|
||||
'--no-wait', is_flag=True, help='Do not wait for deletion to complete')
|
||||
@common_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_proxy_destroy(
|
||||
ctx, delete_resource_group, delete_virtual_network,
|
||||
generate_from_prefix, no_wait):
|
||||
"""Destroy a federation proxy"""
|
||||
ctx.initialize_for_federation()
|
||||
convoy.fleet.action_fed_proxy_destroy(
|
||||
ctx.resource_client, ctx.compute_client, ctx.network_client,
|
||||
ctx.blob_client, ctx.table_client, ctx.queue_client, ctx.config,
|
||||
delete_resource_group, delete_virtual_network, generate_from_prefix,
|
||||
not no_wait)
|
||||
|
||||
|
||||
@fed.command('create')
|
||||
@click.argument('federation-id')
|
||||
@click.option(
|
||||
'--force', is_flag=True,
|
||||
help='Force creation of the federation even if it exists')
|
||||
@click.option(
|
||||
'--no-unique-job-ids', is_flag=True,
|
||||
help='Allow non-unique job ids to be submitted')
|
||||
@common_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_create(ctx, federation_id, force, no_unique_job_ids):
|
||||
"""Create a federation"""
|
||||
ctx.initialize_for_federation()
|
||||
convoy.fleet.action_fed_create(
|
||||
ctx.blob_client, ctx.table_client, ctx.queue_client, ctx.config,
|
||||
federation_id, force, not no_unique_job_ids)
|
||||
|
||||
|
||||
@fed.command('list')
|
||||
@click.option(
|
||||
'--federation-id', multiple=True, help='Limit to specified federation id')
|
||||
@common_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_list(ctx, federation_id):
|
||||
"""List all federations"""
|
||||
ctx.initialize_for_federation()
|
||||
convoy.fleet.action_fed_list(
|
||||
ctx.table_client, ctx.config, federation_id)
|
||||
|
||||
|
||||
@fed.command('destroy')
|
||||
@click.argument('federation-id')
|
||||
@common_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_destroy(ctx, federation_id):
|
||||
"""Destroy a federation"""
|
||||
ctx.initialize_for_federation()
|
||||
convoy.fleet.action_fed_destroy(
|
||||
ctx.blob_client, ctx.table_client, ctx.queue_client, ctx.config,
|
||||
federation_id)
|
||||
|
||||
|
||||
@fed.group()
|
||||
@pass_cli_context
|
||||
def pool(ctx):
|
||||
"""Federation pool actions"""
|
||||
pass
|
||||
|
||||
|
||||
@pool.command('add')
|
||||
@click.argument('federation-id')
|
||||
@click.option(
|
||||
'--batch-service-url',
|
||||
help='Associate specified pools with batch service url')
|
||||
@click.option(
|
||||
'--pool-id', multiple=True, help='Add pool to federation')
|
||||
@common_options
|
||||
@batch_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_pool_add(ctx, federation_id, batch_service_url, pool_id):
|
||||
"""Add a pool to a federation"""
|
||||
init_batch = convoy.util.is_none_or_empty(batch_service_url)
|
||||
ctx.initialize_for_federation(init_batch=init_batch)
|
||||
convoy.fleet.action_fed_pool_add(
|
||||
ctx.batch_client, ctx.table_client, ctx.config, federation_id,
|
||||
batch_service_url, pool_id)
|
||||
|
||||
|
||||
@pool.command('remove')
|
||||
@click.argument('federation-id')
|
||||
@click.option(
|
||||
'--all', is_flag=True, help='Remove all pools from federation')
|
||||
@click.option(
|
||||
'--batch-service-url',
|
||||
help='Associate specified pools with batch service url')
|
||||
@click.option(
|
||||
'--pool-id', multiple=True, help='Remove pool from federation')
|
||||
@common_options
|
||||
@batch_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_pool_remove(ctx, federation_id, all, batch_service_url, pool_id):
|
||||
"""Remove a pool from a federation"""
|
||||
init_batch = convoy.util.is_none_or_empty(batch_service_url)
|
||||
ctx.initialize_for_federation(init_batch=init_batch)
|
||||
convoy.fleet.action_fed_pool_remove(
|
||||
ctx.batch_client, ctx.table_client, ctx.config, federation_id, all,
|
||||
batch_service_url, pool_id)
|
||||
|
||||
|
||||
@fed.group()
|
||||
@pass_cli_context
|
||||
def jobs(ctx):
|
||||
"""Federation jobs actions"""
|
||||
pass
|
||||
|
||||
|
||||
@jobs.command('add')
|
||||
@click.argument('federation-id')
|
||||
@common_options
|
||||
@batch_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_jobs_add(ctx, federation_id):
|
||||
"""Add jobs to a federation"""
|
||||
ctx.initialize_for_federation(init_batch=True)
|
||||
convoy.fleet.action_fed_jobs_add(
|
||||
ctx.batch_client, ctx.keyvault_client, ctx.blob_client,
|
||||
ctx.table_client, ctx.queue_client, ctx.config, federation_id)
|
||||
|
||||
|
||||
@jobs.command('list')
|
||||
@click.argument('federation-id')
|
||||
@click.option(
|
||||
'--job-id', help='List the specified job id')
|
||||
@click.option(
|
||||
'--jobschedule-id', help='List the specified job schedule id')
|
||||
@common_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_jobs_list(ctx, federation_id, job_id, jobschedule_id):
|
||||
"""List jobs or job schedules in a federation"""
|
||||
ctx.initialize_for_federation()
|
||||
convoy.fleet.action_fed_jobs_list(
|
||||
ctx.table_client, ctx.config, federation_id, job_id, jobschedule_id)
|
||||
|
||||
|
||||
@jobs.command('term')
|
||||
@click.argument('federation-id')
|
||||
@click.option(
|
||||
'--all-jobs', is_flag=True, help='Terminate all jobs in federation')
|
||||
@click.option(
|
||||
'--all-jobschedules', is_flag=True,
|
||||
help='Terminate all job schedules in federation')
|
||||
@click.option(
|
||||
'--force', is_flag=True,
|
||||
help='Force termination even if jobs do not exist')
|
||||
@click.option(
|
||||
'--job-id', multiple=True, help='Terminate the specified job id')
|
||||
@click.option(
|
||||
'--jobschedule-id', multiple=True,
|
||||
help='Terminate the specified job schedule id')
|
||||
@common_options
|
||||
@batch_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_jobs_term(
|
||||
ctx, federation_id, all_jobs, all_jobschedules, force, job_id,
|
||||
jobschedule_id):
|
||||
"""Terminate a job or job schedule in a federation"""
|
||||
ctx.initialize_for_federation(init_batch=True)
|
||||
convoy.fleet.action_fed_jobs_del_or_term(
|
||||
ctx.blob_client, ctx.table_client, ctx.queue_client, ctx.config,
|
||||
False, federation_id, job_id, jobschedule_id, all_jobs,
|
||||
all_jobschedules, force)
|
||||
|
||||
|
||||
@jobs.command('del')
|
||||
@click.argument('federation-id')
|
||||
@click.option(
|
||||
'--all-jobs', is_flag=True, help='Delete all jobs in federation')
|
||||
@click.option(
|
||||
'--all-jobschedules', is_flag=True,
|
||||
help='Delete all job schedules in federation')
|
||||
@click.option(
|
||||
'--job-id', multiple=True, help='Delete the specified job id')
|
||||
@click.option(
|
||||
'--jobschedule-id', multiple=True,
|
||||
help='Delete the specified job schedule id')
|
||||
@common_options
|
||||
@batch_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_jobs_del(
|
||||
ctx, federation_id, all_jobs, all_jobschedules, job_id,
|
||||
jobschedule_id):
|
||||
"""Delete a job or job schedule in a federation"""
|
||||
ctx.initialize_for_federation(init_batch=True)
|
||||
convoy.fleet.action_fed_jobs_del_or_term(
|
||||
ctx.blob_client, ctx.table_client, ctx.queue_client, ctx.config,
|
||||
True, federation_id, job_id, jobschedule_id, all_jobs,
|
||||
all_jobschedules, False)
|
||||
|
||||
|
||||
@jobs.command('zap')
|
||||
@click.argument('federation-id')
|
||||
@click.option(
|
||||
'--unique-id', help='Zap the specified queued unique id')
|
||||
@common_options
|
||||
@federation_options
|
||||
@keyvault_options
|
||||
@aad_options
|
||||
@pass_cli_context
|
||||
def fed_jobs_zap(
|
||||
ctx, federation_id, unique_id):
|
||||
"""Zap a queued unique id from a federation"""
|
||||
ctx.initialize_for_federation()
|
||||
convoy.fleet.action_fed_jobs_zap(
|
||||
ctx.blob_client, ctx.config, federation_id, unique_id)
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
convoy.util.setup_logger(logger)
|
||||
cli()
|
||||
|
|
Загрузка…
Ссылка в новой задаче