444 lines
		
	
	
		
			16 KiB
		
	
	
	
		
			Go
		
	
	
	
	
	
			
		
		
	
	
			444 lines
		
	
	
		
			16 KiB
		
	
	
	
		
			Go
		
	
	
	
	
	
| // A class that translates the informations held in the graph object
 | |
| // via its lists of components into an argo file, using the a list of
 | |
| // link ID to build the dag
 | |
| 
 | |
| package workflow_builder
 | |
| 
 | |
| import (
 | |
| 	"fmt"
 | |
| 	"oc-monitord/conf"
 | |
| 	. "oc-monitord/models"
 | |
| 	tools2 "oc-monitord/tools"
 | |
| 	"os"
 | |
| 	"strings"
 | |
| 	"time"
 | |
| 
 | |
| 	oclib "cloud.o-forge.io/core/oc-lib"
 | |
| 	"cloud.o-forge.io/core/oc-lib/models/common/enum"
 | |
| 	"cloud.o-forge.io/core/oc-lib/models/resources"
 | |
| 	w "cloud.o-forge.io/core/oc-lib/models/workflow"
 | |
| 	"github.com/nwtgck/go-fakelish"
 | |
| 	"github.com/rs/zerolog"
 | |
| 	"gopkg.in/yaml.v3"
 | |
| )
 | |
| 
 | |
| var logger zerolog.Logger
 | |
| 
 | |
| type ArgoBuilder struct {
 | |
| 	OriginWorkflow 	*w.Workflow
 | |
| 	Workflow       	Workflow
 | |
| 	Services       	[]*Service
 | |
| 	Timeout        	int
 | |
| 	RemotePeers		[]string
 | |
| }
 | |
| 
 | |
| type Workflow struct {
 | |
| 	ApiVersion string `yaml:"apiVersion"`
 | |
| 	Kind       string `yaml:"kind"`
 | |
| 	Metadata   struct {
 | |
| 		Name string `yaml:"name"`
 | |
| 	} `yaml:"metadata"`
 | |
| 	Spec Spec `yaml:"spec,omitempty"`
 | |
| }
 | |
| 
 | |
| func (b *Workflow) getDag() *Dag {
 | |
| 	for _, t := range b.Spec.Templates {
 | |
| 		if t.Name == "dag" {
 | |
| 			return t.Dag
 | |
| 		}
 | |
| 	}
 | |
| 	b.Spec.Templates = append(b.Spec.Templates, Template{Name: "dag", Dag: &Dag{}})
 | |
| 	return b.Spec.Templates[len(b.Spec.Templates)-1].Dag
 | |
| }
 | |
| 
 | |
| type Spec struct {
 | |
| 	Entrypoint string                `yaml:"entrypoint"`
 | |
| 	Arguments  []Parameter           `yaml:"arguments,omitempty"`
 | |
| 	Volumes    []VolumeClaimTemplate `yaml:"volumeClaimTemplates,omitempty"`
 | |
| 	Templates  []Template            `yaml:"templates"`
 | |
| 	Timeout    int                   `yaml:"activeDeadlineSeconds,omitempty"`
 | |
| }
 | |
| 
 | |
| // TODO: found on a processing instance linked to storage
 | |
| // add s3, gcs, azure, etc if needed on a link between processing and storage
 | |
| func (b *ArgoBuilder) CreateDAG(namespace string, write bool) (string, int, []string, []string, error) {
 | |
| 	fmt.Println("Creating DAG", b.OriginWorkflow.Graph.Items)
 | |
| 	// handle services by checking if there is only one processing with hostname and port
 | |
| 	firstItems, lastItems, volumes := b.createTemplates(namespace)
 | |
| 	b.createVolumes(volumes)
 | |
| 
 | |
| 	if b.Timeout > 0 {
 | |
| 		b.Workflow.Spec.Timeout = b.Timeout
 | |
| 	}
 | |
| 	b.Workflow.Spec.Entrypoint = "dag"
 | |
| 	b.Workflow.ApiVersion = "argoproj.io/v1alpha1"
 | |
| 	b.Workflow.Kind = "Workflow"
 | |
| 	if !write {
 | |
| 		return "", len(b.Workflow.getDag().Tasks), firstItems, lastItems, nil
 | |
| 	}
 | |
| 	random_name := fakelish.GenerateFakeWord(5, 8) + "-" + fakelish.GenerateFakeWord(5, 8)
 | |
| 	b.Workflow.Metadata.Name = "oc-monitor-" + random_name
 | |
| 	logger = oclib.GetLogger()
 | |
| 	yamlified, err := yaml.Marshal(b.Workflow)
 | |
| 	if err != nil {
 | |
| 		logger.Error().Msg("Could not transform object to yaml file")
 | |
| 		return "", 0, firstItems, lastItems, err
 | |
| 	}
 | |
| 	// Give a unique name to each argo file with its timestamp DD:MM:YYYY_hhmmss
 | |
| 	current_timestamp := time.Now().Format("02_01_2006_150405")
 | |
| 	file_name := random_name + "_" + current_timestamp + ".yml"
 | |
| 	workflows_dir := "./argo_workflows/"
 | |
| 	err = os.WriteFile(workflows_dir+file_name, []byte(yamlified), 0660)
 | |
| 	if err != nil {
 | |
| 		logger.Error().Msg("Could not write the yaml file")
 | |
| 		return "", 0, firstItems, lastItems, err
 | |
| 	}
 | |
| 	return workflows_dir + file_name, len(b.Workflow.getDag().Tasks), firstItems, lastItems, nil
 | |
| }
 | |
| 
 | |
| func (b *ArgoBuilder) createTemplates(namespace string) ([]string, []string, []VolumeMount) {
 | |
| 	volumes := []VolumeMount{}
 | |
| 	firstItems := []string{}
 | |
| 	lastItems := []string{}
 | |
| 	items := b.OriginWorkflow.GetGraphItems(b.OriginWorkflow.Graph.IsProcessing)
 | |
| 	fmt.Println("Creating templates", len(items))
 | |
| 	for _, item := range b.OriginWorkflow.GetGraphItems(b.OriginWorkflow.Graph.IsProcessing) {
 | |
| 		instance := item.Processing.GetSelectedInstance()
 | |
| 		fmt.Println("Creating template for", item.Processing.GetName(), instance)
 | |
| 		if instance == nil || instance.(*resources.ProcessingInstance).Access == nil && instance.(*resources.ProcessingInstance).Access.Container != nil {
 | |
| 			logger.Error().Msg("Not enough configuration setup, template can't be created : " + item.Processing.GetName())
 | |
| 			return firstItems, lastItems, volumes
 | |
| 		}
 | |
| 		volumes, firstItems, lastItems = b.createArgoTemplates(namespace,
 | |
| 			item.ID, item.Processing, volumes, firstItems, lastItems)
 | |
| 	}
 | |
| 	firstWfTasks := map[string][]string{}
 | |
| 	latestWfTasks := map[string][]string{}
 | |
| 	relatedWfTasks := map[string][]string{}
 | |
| 	for _, wf := range b.OriginWorkflow.Workflows {
 | |
| 		realWorkflow, code, err := w.NewAccessor(nil).LoadOne(wf)
 | |
| 		if code != 200 {
 | |
| 			logger.Error().Msg("Error loading the workflow : " + err.Error())
 | |
| 			continue
 | |
| 		}
 | |
| 		subBuilder := ArgoBuilder{OriginWorkflow: realWorkflow.(*w.Workflow), Timeout: b.Timeout}
 | |
| 		_, _, fi, li, err := subBuilder.CreateDAG(namespace, false)
 | |
| 		if err != nil {
 | |
| 			logger.Error().Msg("Error creating the subworkflow : " + err.Error())
 | |
| 			continue
 | |
| 		}
 | |
| 		firstWfTasks[wf] = fi
 | |
| 		if ok, depsOfIds := subBuilder.isArgoDependancy(wf); ok { // IS BEFORE
 | |
| 			latestWfTasks[wf] = li
 | |
| 			relatedWfTasks[wf] = depsOfIds
 | |
| 		}
 | |
| 		subDag := subBuilder.Workflow.getDag()
 | |
| 		d := b.Workflow.getDag()
 | |
| 		d.Tasks = append(d.Tasks, subDag.Tasks...) // add the tasks of the subworkflow to the main workflow
 | |
| 		b.Workflow.Spec.Templates = append(b.Workflow.Spec.Templates, subBuilder.Workflow.Spec.Templates...)
 | |
| 		b.Workflow.Spec.Volumes = append(b.Workflow.Spec.Volumes, subBuilder.Workflow.Spec.Volumes...)
 | |
| 		b.Workflow.Spec.Arguments = append(b.Workflow.Spec.Arguments, subBuilder.Workflow.Spec.Arguments...)
 | |
| 		b.Services = append(b.Services, subBuilder.Services...)
 | |
| 	}
 | |
| 	for wfID, depsOfIds := range relatedWfTasks {
 | |
| 		for _, dep := range depsOfIds {
 | |
| 			for _, task := range b.Workflow.getDag().Tasks {
 | |
| 				if strings.Contains(task.Name, dep) {
 | |
| 					index := -1
 | |
| 					for i, depp := range task.Dependencies {
 | |
| 						if strings.Contains(depp, wfID) {
 | |
| 							index = i
 | |
| 							break
 | |
| 						}
 | |
| 					}
 | |
| 					if index != -1 {
 | |
| 						task.Dependencies = append(task.Dependencies[:index], task.Dependencies[index+1:]...)
 | |
| 					}
 | |
| 					task.Dependencies = append(task.Dependencies, latestWfTasks[wfID]...)
 | |
| 				}
 | |
| 			}
 | |
| 		}
 | |
| 	}
 | |
| 	for wfID, fi := range firstWfTasks {
 | |
| 		deps := b.getArgoDependencies(wfID)
 | |
| 		if len(deps) > 0 {
 | |
| 			for _, dep := range fi {
 | |
| 				for _, task := range b.Workflow.getDag().Tasks {
 | |
| 					if strings.Contains(task.Name, dep) {
 | |
| 						task.Dependencies = append(task.Dependencies, deps...)
 | |
| 					}
 | |
| 				}
 | |
| 			}
 | |
| 		}
 | |
| 	}
 | |
| 	if b.Services != nil {
 | |
| 		dag := b.Workflow.getDag()
 | |
| 		dag.Tasks = append(dag.Tasks, Task{Name: "workflow-service-pod", Template: "workflow-service-pod"})
 | |
| 		b.addServiceToArgo()
 | |
| 	}
 | |
| 	return firstItems, lastItems, volumes
 | |
| }
 | |
| 
 | |
| func (b *ArgoBuilder) createArgoTemplates(namespace string,
 | |
| 	id string,
 | |
| 	processing *resources.ProcessingResource,
 | |
| 	volumes []VolumeMount,
 | |
| 	firstItems []string,
 | |
| 	lastItems []string) ([]VolumeMount, []string, []string) {
 | |
| 	_, firstItems, lastItems = b.addTaskToArgo(b.Workflow.getDag(), id, processing, firstItems, lastItems)
 | |
| 	template := &Template{Name: getArgoName(processing.GetName(), id)}
 | |
| 	fmt.Println("Creating template for", template.Name)
 | |
| 	isReparted, peerId := b.isProcessingReparted(*processing,id)
 | |
| 	template.CreateContainer(processing, b.Workflow.getDag())
 | |
| 	if isReparted {
 | |
| 		b.RemotePeers = append(b.RemotePeers, peerId)
 | |
| 	}
 | |
| 	// get datacenter from the processing
 | |
| 	if processing.IsService {
 | |
| 		b.CreateService(id, processing)
 | |
| 		template.Metadata.Labels = make(map[string]string)
 | |
| 		template.Metadata.Labels["app"] = "oc-service-" + processing.GetName() // Construct the template for the k8s service and add a link in graph between k8s service and processing
 | |
| 	}
 | |
| 	related := b.OriginWorkflow.GetByRelatedProcessing(id, b.OriginWorkflow.Graph.IsStorage)
 | |
| 	for _, r := range related {
 | |
| 		storage := r.Node.(*resources.StorageResource)
 | |
| 		for _, linkToStorage := range r.Links {
 | |
| 			for _, rw := range linkToStorage.StorageLinkInfos {
 | |
| 				art := Artifact{Path: template.ReplacePerEnv(rw.Source, linkToStorage.Env)}
 | |
| 				if rw.Write {
 | |
| 					art.Name = storage.GetName() + "-" + rw.Destination + "-input-write"
 | |
| 				} else {
 | |
| 					art.Name = storage.GetName() + "-" + rw.Destination + "-input-read"
 | |
| 				}
 | |
| 				if storage.StorageType == enum.S3 {
 | |
| 					art.S3 = &Key{
 | |
| 						Key:      template.ReplacePerEnv(rw.Destination+"/"+rw.FileName, linkToStorage.Env),
 | |
| 						Insecure: true, // temporary
 | |
| 					}
 | |
| 					sel := storage.GetSelectedInstance()
 | |
| 					if sel != nil {
 | |
| 						if sel.(*resources.StorageResourceInstance).Credentials != nil {
 | |
| 							tool, err := tools2.NewService(conf.GetConfig().Mode)
 | |
| 							if err != nil || tool == nil {
 | |
| 								logger.Error().Msg("Could not create the access secret")
 | |
| 							} else {
 | |
| 								id, err := tool.CreateAccessSecret(namespace,
 | |
| 									sel.(*resources.StorageResourceInstance).Credentials.Login,
 | |
| 									sel.(*resources.StorageResourceInstance).Credentials.Pass)
 | |
| 								if err == nil {
 | |
| 									art.S3.AccessKeySecret = &Secret{
 | |
| 										Name: id,
 | |
| 										Key:  "access-key",
 | |
| 									}
 | |
| 									art.S3.SecretKeySecret = &Secret{
 | |
| 										Name: id,
 | |
| 										Key:  "secret-key",
 | |
| 									}
 | |
| 								}
 | |
| 							}
 | |
| 						}
 | |
| 						art.S3.Key = strings.ReplaceAll(art.S3.Key, sel.(*resources.StorageResourceInstance).Source+"/", "")
 | |
| 						art.S3.Key = strings.ReplaceAll(art.S3.Key, sel.(*resources.StorageResourceInstance).Source, "")
 | |
| 						splits := strings.Split(art.S3.EndPoint, "/")
 | |
| 						if len(splits) > 1 {
 | |
| 							art.S3.Bucket = splits[0]
 | |
| 							art.S3.EndPoint = strings.Join(splits[1:], "/")
 | |
| 						} else {
 | |
| 							art.S3.Bucket = splits[0]
 | |
| 						}
 | |
| 					}
 | |
| 				}
 | |
| 				if rw.Write {
 | |
| 					template.Outputs.Artifacts = append(template.Inputs.Artifacts, art)
 | |
| 				} else {
 | |
| 					template.Inputs.Artifacts = append(template.Outputs.Artifacts, art)
 | |
| 				}
 | |
| 			}
 | |
| 		}
 | |
| 		index := 0
 | |
| 		if storage.SelectedInstanceIndex != nil && (*storage.SelectedInstanceIndex) >= 0 {
 | |
| 			index = *storage.SelectedInstanceIndex
 | |
| 		}
 | |
| 		s := storage.Instances[index]
 | |
| 		if s.Local {
 | |
| 			volumes = template.Container.AddVolumeMount(VolumeMount{
 | |
| 				Name:      strings.ReplaceAll(strings.ToLower(storage.GetName()), " ", "-"),
 | |
| 				MountPath: s.Source,
 | |
| 				Storage:   storage,
 | |
| 			}, volumes)
 | |
| 		}
 | |
| 	}
 | |
| 	b.Workflow.Spec.Templates = append(b.Workflow.Spec.Templates, *template)
 | |
| 	return volumes, firstItems, lastItems
 | |
| }
 | |
| 
 | |
| func (b *ArgoBuilder) addTaskToArgo(dag *Dag, graphItemID string, processing *resources.ProcessingResource,
 | |
| 	firstItems []string, lastItems []string) (*Dag, []string, []string) {
 | |
| 	unique_name := getArgoName(processing.GetName(), graphItemID)
 | |
| 	step := Task{Name: unique_name, Template: unique_name}
 | |
| 	instance := processing.GetSelectedInstance()
 | |
| 	if instance != nil {
 | |
| 		for _, value := range instance.(*resources.ProcessingInstance).Env {
 | |
| 			step.Arguments.Parameters = append(step.Arguments.Parameters, Parameter{
 | |
| 				Name:  value.Name,
 | |
| 				Value: value.Value,
 | |
| 			})
 | |
| 		}
 | |
| 		for _, value := range instance.(*resources.ProcessingInstance).Inputs {
 | |
| 			step.Arguments.Parameters = append(step.Arguments.Parameters, Parameter{
 | |
| 				Name:  value.Name,
 | |
| 				Value: value.Value,
 | |
| 			})
 | |
| 		}
 | |
| 		for _, value := range instance.(*resources.ProcessingInstance).Outputs {
 | |
| 			step.Arguments.Parameters = append(step.Arguments.Parameters, Parameter{
 | |
| 				Name:  value.Name,
 | |
| 				Value: value.Value,
 | |
| 			})
 | |
| 		}
 | |
| 	}
 | |
| 	step.Dependencies = b.getArgoDependencies(graphItemID)
 | |
| 	name := ""
 | |
| 	if b.OriginWorkflow.Graph.Items[graphItemID].Processing != nil {
 | |
| 		name = b.OriginWorkflow.Graph.Items[graphItemID].Processing.GetName()
 | |
| 	}
 | |
| 	if b.OriginWorkflow.Graph.Items[graphItemID].Workflow != nil {
 | |
| 		name = b.OriginWorkflow.Graph.Items[graphItemID].Workflow.GetName()
 | |
| 	}
 | |
| 	if len(step.Dependencies) == 0 && name != "" {
 | |
| 		firstItems = append(firstItems, getArgoName(name, graphItemID))
 | |
| 	}
 | |
| 	if ok, _ := b.isArgoDependancy(graphItemID); !ok && name != "" {
 | |
| 		lastItems = append(lastItems, getArgoName(name, graphItemID))
 | |
| 	}
 | |
| 	dag.Tasks = append(dag.Tasks, step)
 | |
| 	return dag, firstItems, lastItems
 | |
| }
 | |
| 
 | |
| func (b *ArgoBuilder) createVolumes(volumes []VolumeMount) { // TODO : one think about remote volume but TG
 | |
| 	for _, volume := range volumes {
 | |
| 		index := 0
 | |
| 		if volume.Storage.SelectedInstanceIndex != nil && (*volume.Storage.SelectedInstanceIndex) >= 0 {
 | |
| 			index = *volume.Storage.SelectedInstanceIndex
 | |
| 		}
 | |
| 		storage := volume.Storage.Instances[index]
 | |
| 		new_volume := VolumeClaimTemplate{}
 | |
| 		new_volume.Metadata.Name = strings.ReplaceAll(strings.ToLower(volume.Name), " ", "-")
 | |
| 		new_volume.Spec.AccessModes = []string{"ReadWriteOnce"}
 | |
| 		new_volume.Spec.Resources.Requests.Storage = fmt.Sprintf("%v", storage.SizeGB) + storage.SizeType.ToArgo()
 | |
| 		b.Workflow.Spec.Volumes = append(b.Workflow.Spec.Volumes, new_volume)
 | |
| 	}
 | |
| }
 | |
| 
 | |
| func (b *ArgoBuilder) isArgoDependancy(id string) (bool, []string) {
 | |
| 	dependancyOfIDs := []string{}
 | |
| 	isDeps := false
 | |
| 	for _, link := range b.OriginWorkflow.Graph.Links {
 | |
| 		if _, ok := b.OriginWorkflow.Graph.Items[link.Destination.ID]; !ok {
 | |
| 			fmt.Println("Could not find the source of the link", link.Destination.ID)
 | |
| 			continue
 | |
| 		}
 | |
| 		source := b.OriginWorkflow.Graph.Items[link.Destination.ID].Processing
 | |
| 		if id == link.Source.ID && source != nil {
 | |
| 			isDeps = true
 | |
| 			dependancyOfIDs = append(dependancyOfIDs, getArgoName(source.GetName(), link.Destination.ID))
 | |
| 		}
 | |
| 		wourceWF := b.OriginWorkflow.Graph.Items[link.Destination.ID].Workflow
 | |
| 		if id == link.Source.ID && wourceWF != nil {
 | |
| 			isDeps = true
 | |
| 			dependancyOfIDs = append(dependancyOfIDs, getArgoName(wourceWF.GetName(), link.Destination.ID))
 | |
| 		}
 | |
| 	}
 | |
| 	return isDeps, dependancyOfIDs
 | |
| }
 | |
| 
 | |
| func (b *ArgoBuilder) getArgoDependencies(id string) (dependencies []string) {
 | |
| 	for _, link := range b.OriginWorkflow.Graph.Links {
 | |
| 		if _, ok := b.OriginWorkflow.Graph.Items[link.Source.ID]; !ok {
 | |
| 			fmt.Println("Could not find the source of the link", link.Source.ID)
 | |
| 			continue
 | |
| 		}
 | |
| 		source := b.OriginWorkflow.Graph.Items[link.Source.ID].Processing
 | |
| 		if id == link.Destination.ID && source != nil {
 | |
| 			dependency_name := getArgoName(source.GetName(), link.Source.ID)
 | |
| 			dependencies = append(dependencies, dependency_name)
 | |
| 			continue
 | |
| 		}
 | |
| 	}
 | |
| 	return
 | |
| }
 | |
| 
 | |
| func getArgoName(raw_name string, component_id string) (formatedName string) {
 | |
| 	formatedName = strings.ReplaceAll(raw_name, " ", "-")
 | |
| 	formatedName += "-" + component_id
 | |
| 	formatedName = strings.ToLower(formatedName)
 | |
| 	return
 | |
| }
 | |
| 
 | |
| // Verify if a processing resource is attached to another Compute than the one hosting
 | |
| // the current Open Cloud instance. If true return the peer ID to contact
 | |
| func (b *ArgoBuilder) isProcessingReparted(processing resources.ProcessingResource, graphID string) (bool,string) {
 | |
| 	computeAttached := b.retrieveProcessingCompute(graphID)
 | |
| 	if computeAttached == nil {
 | |
| 		logger.Error().Msg("No compute was found attached to processing " + processing.Name + " : " + processing.UUID )
 | |
| 		panic(0)
 | |
| 	}
 | |
| 
 | |
| 	
 | |
| 	// Creates an accessor srtictly for Peer Collection 
 | |
| 	req := oclib.NewRequest(oclib.LibDataEnum(oclib.PEER),"","",nil,nil)
 | |
| 	if req == nil {
 | |
| 		fmt.Println("TODO : handle error when trying to create a request on the Peer Collection")
 | |
| 		return false, ""
 | |
| 	} 
 | |
| 
 | |
| 	res := req.LoadOne(computeAttached.CreatorID)
 | |
| 	if res.Err != "" {
 | |
| 		fmt.Print("TODO : handle error when requesting PeerID")
 | |
| 		fmt.Print(res.Err)
 | |
| 		return false, ""
 | |
| 	}
 | |
| 	
 | |
| 	peer := *res.ToPeer()
 | |
| 
 | |
| 	isNotReparted, _ := peer.IsMySelf()
 | |
| 	fmt.Println("Result IsMySelf for ", peer.UUID ," : ", isNotReparted)
 | |
| 	
 | |
| 	return !isNotReparted, peer.UUID
 | |
| }
 | |
| 
 | |
| func (b *ArgoBuilder) retrieveProcessingCompute(graphID string) *resources.ComputeResource {
 | |
| 	for _, link := range b.OriginWorkflow.Graph.Links {
 | |
| 		// If a link contains the id of the processing
 | |
| 		var oppositeId string 
 | |
| 		if link.Source.ID == graphID{
 | |
| 			oppositeId = link.Destination.ID
 | |
| 		} else if(link.Destination.ID == graphID){
 | |
| 			oppositeId = link.Source.ID
 | |
| 		}
 | |
| 		fmt.Println("OppositeId : ", oppositeId)
 | |
| 		if oppositeId != "" {
 | |
| 			dt, res := b.OriginWorkflow.Graph.GetResource(oppositeId)
 | |
| 			if dt == oclib.COMPUTE_RESOURCE {
 | |
| 				return res.(*resources.ComputeResource)
 | |
| 			} else {
 | |
| 				continue
 | |
| 			}
 | |
| 		}
 | |
| 
 | |
| 	}
 | |
| 		
 | |
| 	return nil 
 | |
| }
 | |
| 
 | |
| 
 | |
| // Execute the last actions once the YAML file for the Argo Workflow is created
 | |
| func (b *ArgoBuilder) CompleteBuild(executionsId string) error {
 | |
| 	fmt.Println("DEV :: Completing build")
 | |
| 	for _, peer := range b.RemotePeers {
 | |
| 		fmt.Println("DEV :: Launching Admiralty Setup for ", peer)
 | |
| 		setter := AdmiraltySetter{Id: executionsId}
 | |
| 		setter.InitializeAdmiralty(conf.GetConfig().PeerID,peer)
 | |
| 	}
 | |
| 	return nil
 | |
| } |