Add management support to non-ephemeral hosts

Management support is only available to the ephemeral host. This change
extends support of management features to hosts not labeled as the
ephemeral host.

Change-Id: I0ac917f2633f659e56e67cafc52d2db9d967754f
Signed-off-by: Drew Walters <andrew.walters@att.com>
This commit is contained in:
Drew Walters 2020-04-15 21:01:36 +00:00
parent 9fe36fb4b2
commit 995538829e
21 changed files with 872 additions and 563 deletions

View File

@ -17,6 +17,8 @@ package bootstrap
import ( import (
"github.com/spf13/cobra" "github.com/spf13/cobra"
"opendev.org/airship/airshipctl/pkg/config"
"opendev.org/airship/airshipctl/pkg/document"
"opendev.org/airship/airshipctl/pkg/environment" "opendev.org/airship/airshipctl/pkg/environment"
"opendev.org/airship/airshipctl/pkg/remote" "opendev.org/airship/airshipctl/pkg/remote"
) )
@ -27,12 +29,19 @@ func NewRemoteDirectCommand(rootSettings *environment.AirshipCTLSettings) *cobra
Use: "remotedirect", Use: "remotedirect",
Short: "Bootstrap ephemeral node", Short: "Bootstrap ephemeral node",
RunE: func(cmd *cobra.Command, args []string) error { RunE: func(cmd *cobra.Command, args []string) error {
a, err := remote.NewAdapter(rootSettings) manager, err := remote.NewManager(rootSettings,
config.BootstrapPhase,
remote.ByLabel(document.EphemeralHostSelector))
if err != nil { if err != nil {
return err return err
} }
return a.DoRemoteDirect() if len(manager.Hosts) != 1 {
return remote.NewRemoteDirectErrorf("more than one node defined as the ephemeral node")
}
ephemeralHost := manager.Hosts[0]
return ephemeralHost.DoRemoteDirect(rootSettings)
}, },
} }

View File

@ -1,14 +1,16 @@
// Licensed under the Apache License, Version 2.0 (the "License"); /*
// you may not use this file except in compliance with the License. Licensed under the Apache License, Version 2.0 (the "License");
// You may obtain a copy of the License at you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
// https://www.apache.org/licenses/LICENSE-2.0
// https://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS, Unless required by applicable law or agreed to in writing, software
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// limitations under the License. See the License for the specific language governing permissions and
limitations under the License.
*/
package remote package remote
@ -18,6 +20,11 @@ import (
"opendev.org/airship/airshipctl/pkg/environment" "opendev.org/airship/airshipctl/pkg/environment"
) )
const (
flagPhase = "phase"
flagPhaseDescription = "airshipctl phase that contains the desired baremetal host document(s)"
)
// NewRemoteCommand creates a new command that provides functionality to control remote entities. // NewRemoteCommand creates a new command that provides functionality to control remote entities.
func NewRemoteCommand(rootSettings *environment.AirshipCTLSettings) *cobra.Command { func NewRemoteCommand(rootSettings *environment.AirshipCTLSettings) *cobra.Command {
remoteRootCmd := &cobra.Command{ remoteRootCmd := &cobra.Command{

View File

@ -1,14 +1,16 @@
// Licensed under the Apache License, Version 2.0 (the "License"); /*
// you may not use this file except in compliance with the License. Licensed under the Apache License, Version 2.0 (the "License");
// You may obtain a copy of the License at you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
// https://www.apache.org/licenses/LICENSE-2.0
// https://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS, Unless required by applicable law or agreed to in writing, software
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// limitations under the License. See the License for the specific language governing permissions and
limitations under the License.
*/
package remote package remote
@ -17,31 +19,38 @@ import (
"github.com/spf13/cobra" "github.com/spf13/cobra"
"opendev.org/airship/airshipctl/pkg/config"
"opendev.org/airship/airshipctl/pkg/environment" "opendev.org/airship/airshipctl/pkg/environment"
"opendev.org/airship/airshipctl/pkg/remote" "opendev.org/airship/airshipctl/pkg/remote"
) )
// NewPowerOffCommand provides a command to shutdown a remote host. // NewPowerOffCommand provides a command to shutdown a remote host.
func NewPowerOffCommand(rootSettings *environment.AirshipCTLSettings) *cobra.Command { func NewPowerOffCommand(rootSettings *environment.AirshipCTLSettings) *cobra.Command {
powerOffCmd := &cobra.Command{ var phase string
cmd := &cobra.Command{
Use: "poweroff SYSTEM_ID", Use: "poweroff SYSTEM_ID",
Short: "Shutdown a host", Short: "Shutdown a host",
Args: cobra.ExactArgs(1), Args: cobra.ExactArgs(1),
RunE: func(cmd *cobra.Command, args []string) error { RunE: func(cmd *cobra.Command, args []string) error {
a, err := remote.NewAdapter(rootSettings) m, err := remote.NewManager(rootSettings, phase, remote.ByName(args[0]))
if err != nil { if err != nil {
return err return err
} }
if err := a.OOBClient.SystemPowerOff(a.Context, args[0]); err != nil { for _, host := range m.Hosts {
return err if err := host.SystemPowerOff(host.Context); err != nil {
} return err
}
fmt.Fprintf(cmd.OutOrStdout(), "Remote host %s powered off\n", args[0]) fmt.Fprintf(cmd.OutOrStdout(), "Remote host %s powered off\n", args[0])
}
return nil return nil
}, },
} }
return powerOffCmd flags := cmd.Flags()
flags.StringVar(&phase, flagPhase, config.BootstrapPhase, flagPhaseDescription)
return cmd
} }

View File

@ -1,14 +1,16 @@
// Licensed under the Apache License, Version 2.0 (the "License"); /*
// you may not use this file except in compliance with the License. Licensed under the Apache License, Version 2.0 (the "License");
// You may obtain a copy of the License at you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
// https://www.apache.org/licenses/LICENSE-2.0
// https://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS, Unless required by applicable law or agreed to in writing, software
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// limitations under the License. See the License for the specific language governing permissions and
limitations under the License.
*/
package remote package remote
@ -17,32 +19,40 @@ import (
"github.com/spf13/cobra" "github.com/spf13/cobra"
"opendev.org/airship/airshipctl/pkg/config"
"opendev.org/airship/airshipctl/pkg/environment" "opendev.org/airship/airshipctl/pkg/environment"
"opendev.org/airship/airshipctl/pkg/remote" "opendev.org/airship/airshipctl/pkg/remote"
) )
// NewPowerStatusCommand provides a command to retrieve the power status of a remote host. // NewPowerStatusCommand provides a command to retrieve the power status of a remote host.
func NewPowerStatusCommand(rootSettings *environment.AirshipCTLSettings) *cobra.Command { func NewPowerStatusCommand(rootSettings *environment.AirshipCTLSettings) *cobra.Command {
powerStatusCmd := &cobra.Command{ var phase string
cmd := &cobra.Command{
Use: "powerstatus SYSTEM_ID", Use: "powerstatus SYSTEM_ID",
Short: "Retrieve the power status of a host", Short: "Retrieve the power status of a host",
Args: cobra.ExactArgs(1), Args: cobra.ExactArgs(1),
RunE: func(cmd *cobra.Command, args []string) error { RunE: func(cmd *cobra.Command, args []string) error {
a, err := remote.NewAdapter(rootSettings) m, err := remote.NewManager(rootSettings, phase, remote.ByName(args[0]))
if err != nil { if err != nil {
return err return err
} }
powerStatus, err := a.OOBClient.SystemPowerStatus(a.Context, args[0]) for _, host := range m.Hosts {
if err != nil { powerStatus, err := host.SystemPowerStatus(host.Context)
return err if err != nil {
} return err
}
fmt.Fprintf(cmd.OutOrStdout(), "Remote host %s has power status: %s\n", args[0], powerStatus) fmt.Fprintf(cmd.OutOrStdout(), "Remote host %s has power status: %s\n", args[0],
powerStatus)
}
return nil return nil
}, },
} }
return powerStatusCmd flags := cmd.Flags()
flags.StringVar(&phase, flagPhase, config.BootstrapPhase, flagPhaseDescription)
return cmd
} }

View File

@ -1,14 +1,16 @@
// Licensed under the Apache License, Version 2.0 (the "License"); /*
// you may not use this file except in compliance with the License. Licensed under the Apache License, Version 2.0 (the "License");
// You may obtain a copy of the License at you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
// https://www.apache.org/licenses/LICENSE-2.0
// https://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS, Unless required by applicable law or agreed to in writing, software
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// limitations under the License. See the License for the specific language governing permissions and
limitations under the License.
*/
package remote package remote
@ -17,31 +19,38 @@ import (
"github.com/spf13/cobra" "github.com/spf13/cobra"
"opendev.org/airship/airshipctl/pkg/config"
"opendev.org/airship/airshipctl/pkg/environment" "opendev.org/airship/airshipctl/pkg/environment"
"opendev.org/airship/airshipctl/pkg/remote" "opendev.org/airship/airshipctl/pkg/remote"
) )
// NewRebootCommand provides a command with the capability to reboot hosts. // NewRebootCommand provides a command with the capability to reboot hosts.
func NewRebootCommand(rootSettings *environment.AirshipCTLSettings) *cobra.Command { func NewRebootCommand(rootSettings *environment.AirshipCTLSettings) *cobra.Command {
rebootCmd := &cobra.Command{ var phase string
cmd := &cobra.Command{
Use: "reboot SYSTEM_ID", Use: "reboot SYSTEM_ID",
Short: "Reboot a host", Short: "Reboot a host",
Args: cobra.ExactArgs(1), Args: cobra.ExactArgs(1),
RunE: func(cmd *cobra.Command, args []string) error { RunE: func(cmd *cobra.Command, args []string) error {
a, err := remote.NewAdapter(rootSettings) m, err := remote.NewManager(rootSettings, phase, remote.ByName(args[0]))
if err != nil { if err != nil {
return err return err
} }
if err = a.OOBClient.RebootSystem(a.Context, args[0]); err != nil { for _, host := range m.Hosts {
return err if err := host.RebootSystem(host.Context); err != nil {
} return err
}
fmt.Fprintf(cmd.OutOrStdout(), "Rebooted remote host %s\n", args[0]) fmt.Fprintf(cmd.OutOrStdout(), "Rebooted remote host %s\n", args[0])
}
return nil return nil
}, },
} }
return rebootCmd flags := cmd.Flags()
flags.StringVar(&phase, flagPhase, config.BootstrapPhase, flagPhaseDescription)
return cmd
} }

View File

@ -193,17 +193,6 @@ type Builder struct {
// RemoteDirect configuration options // RemoteDirect configuration options
type RemoteDirect struct { type RemoteDirect struct {
// RemoteType specifies type of epehemeral node managfement (e.g redfish,
// smash e.t.c.)
RemoteType string `json:"remoteType,omitempty"`
// IsoURL specifies url to download ISO image for epehemeral node // IsoURL specifies url to download ISO image for epehemeral node
IsoURL string `json:"isoUrl,omitempty"` IsoURL string `json:"isoUrl,omitempty"`
// Ignore SSL certificate check. This options is useful for remote APIs
// with non-trusted or self-signed SSL certificates
Insecure bool `json:"insecure,omitempty"`
// Allow remotedirect requests to be proxied. This defaults to false
// because in general, most users will want to communicate directly
// with redfish and other bmc urls directly even if the environment
// has a proxy set
UseProxy bool `json:"useproxy,omitempty"`
} }

View File

@ -42,8 +42,7 @@ func NewConfig() *Config {
OutputMetadataFileName: "output-metadata.yaml", OutputMetadataFileName: "output-metadata.yaml",
}, },
RemoteDirect: &RemoteDirect{ RemoteDirect: &RemoteDirect{
RemoteType: AirshipDefaultRemoteType, IsoURL: AirshipDefaultIsoURL,
IsoURL: AirshipDefaultIsoURL,
}, },
}, },
}, },

View File

@ -29,3 +29,25 @@ func NewRemoteDirectErrorf(format string, v ...interface{}) error {
e.Message = fmt.Sprintf(format, v...) e.Message = fmt.Sprintf(format, v...)
return e return e
} }
// ErrUnknownManagementType is an error that indicates the remote type specified in the airshipctl management
// configuration (e.g. redfish, redfish-dell) is not supported.
type ErrUnknownManagementType struct {
aerror.AirshipError
Type string
}
func (e ErrUnknownManagementType) Error() string {
return fmt.Sprintf("unknown management type: %s", e.Type)
}
// ErrMissingBootstrapInfoOption is an error that indicates a bootstrap option is missing in the airshipctl
// bootstrapInfo configuration.
type ErrMissingBootstrapInfoOption struct {
aerror.AirshipError
What string
}
func (e ErrMissingBootstrapInfoOption) Error() string {
return fmt.Sprintf("missing bootstrapInfo option: %s", e.What)
}

191
pkg/remote/management.go Normal file
View File

@ -0,0 +1,191 @@
/*
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
https://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
// Package remote manages baremetal hosts.
package remote
import (
"context"
"opendev.org/airship/airshipctl/pkg/config"
"opendev.org/airship/airshipctl/pkg/document"
"opendev.org/airship/airshipctl/pkg/environment"
"opendev.org/airship/airshipctl/pkg/remote/redfish"
)
// Client is a set of functions that clients created for out-of-band power management and control should implement. The
// functions within client are used by power management commands and remote direct functionality.
type Client interface {
RebootSystem(context.Context) error
SystemPowerOff(context.Context) error
// TODO(drewwalters96): Should this be a string forever? We may want to define our own custom type, as the
// string format will be client dependent when we add new clients.
SystemPowerStatus(context.Context) (string, error)
NodeID() string
SetBootSourceByType(context.Context) error
// TODO(drewwalters96): This function is tightly coupled to Redfish. It should be combined with the
// SetBootSource operation and removed from the client interface.
SetVirtualMedia(context.Context, string) error
}
// Manager orchestrates a grouping of baremetal hosts. When a manager is created using its convenience function, the
// manager contains a list of hosts ready for out-of-band management. Iterate over the Hosts property to invoke actions
// on each host.
type Manager struct {
Config config.ManagementConfiguration
Hosts []baremetalHost
}
// baremetalHost is an airshipctl representation of a baremetal host, defined by a baremetal host document, that embeds
// actions an out-of-band client can perform. Once instantiated, actions can be performed on a baremetal host.
type baremetalHost struct {
Client
Context context.Context
BMCAddress string
username string
password string
}
// HostSelector populates baremetal hosts within a manager when supplied with selection criteria.
type HostSelector func(*Manager, config.ManagementConfiguration, document.Bundle) error
// ByLabel adds all hosts to a manager whose documents match a supplied label selector.
func ByLabel(label string) HostSelector {
return func(a *Manager, mgmtCfg config.ManagementConfiguration, docBundle document.Bundle) error {
selector := document.NewSelector().ByKind(document.BareMetalHostKind).ByLabel(label)
docs, err := docBundle.Select(selector)
if err != nil {
return err
}
if len(docs) == 0 {
return document.ErrDocNotFound{Selector: selector}
}
for _, doc := range docs {
host, err := newBaremetalHost(mgmtCfg, doc, docBundle)
if err != nil {
return err
}
a.Hosts = append(a.Hosts, host)
}
return nil
}
}
// ByName adds the host to a manager whose document meets the specified name.
func ByName(name string) HostSelector {
return func(a *Manager, mgmtCfg config.ManagementConfiguration, docBundle document.Bundle) error {
selector := document.NewSelector().ByKind(document.BareMetalHostKind).ByName(name)
doc, err := docBundle.SelectOne(selector)
if err != nil {
return err
}
host, err := newBaremetalHost(mgmtCfg, doc, docBundle)
if err != nil {
return err
}
a.Hosts = append(a.Hosts, host)
return nil
}
}
// NewManager provides a manager that exposes the capability to perform remote direct functionality and other
// out-of-band management on multiple hosts.
func NewManager(settings *environment.AirshipCTLSettings, phase string, hosts ...HostSelector) (*Manager, error) {
configContext, err := settings.Config.GetCurrentContext()
if err != nil {
return nil, err
}
managementCfg, err := settings.Config.CurrentContextManagementConfig()
if err != nil {
return nil, err
}
clusterType := configContext.ClusterType()
entrypoint, err := settings.Config.CurrentContextEntryPoint(clusterType, phase)
if err != nil {
return nil, err
}
docBundle, err := document.NewBundleByPath(entrypoint)
if err != nil {
return nil, err
}
manager := &Manager{
Config: *managementCfg,
Hosts: []baremetalHost{},
}
// Each function in hosts modifies the list of hosts for the new manager based on selection criteria provided
// by CLI arguments and airshipctl settings.
for _, addHost := range hosts {
if err := addHost(manager, *managementCfg, docBundle); err != nil {
return nil, err
}
}
return manager, nil
}
// newBaremetalHost creates a representation of a baremetal host that is configured to perform management actions by
// invoking its client methods (provided by the remote.Client interface).
func newBaremetalHost(mgmtCfg config.ManagementConfiguration,
hostDoc document.Document,
docBundle document.Bundle) (baremetalHost, error) {
var host baremetalHost
address, err := document.GetBMHBMCAddress(hostDoc)
if err != nil {
return host, err
}
username, password, err := document.GetBMHBMCCredentials(hostDoc, docBundle)
if err != nil {
return host, err
}
// Select the client that corresponds to the management type specified in the airshipctl config.
switch mgmtCfg.Type {
case redfish.ClientType:
ctx, client, err := redfish.NewClient(
address,
mgmtCfg.Insecure,
mgmtCfg.UseProxy,
username,
password)
if err != nil {
return host, err
}
host = baremetalHost{client, ctx, address, username, password}
default:
return host, ErrUnknownManagementType{Type: mgmtCfg.Type}
}
return host, nil
}

View File

@ -0,0 +1,137 @@
/*
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
https://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package remote
import (
"fmt"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"opendev.org/airship/airshipctl/pkg/config"
"opendev.org/airship/airshipctl/pkg/document"
"opendev.org/airship/airshipctl/pkg/environment"
"opendev.org/airship/airshipctl/testutil"
)
type Configuration func(*environment.AirshipCTLSettings)
// initSettings initializes the global airshipctl settings with test data by accepting functions as arguments that
// manipulate configuration sections.
func initSettings(t *testing.T, configs ...Configuration) *environment.AirshipCTLSettings {
t.Helper()
settings := &environment.AirshipCTLSettings{Config: testutil.DummyConfig()}
for _, cfg := range configs {
cfg(settings)
}
return settings
}
// withManagementConfig initializes the management config when used as an argument to initSettings.
func withManagementConfig(cfg *config.ManagementConfiguration) Configuration {
return func(settings *environment.AirshipCTLSettings) {
settings.Config.ManagementConfiguration["dummy_management_config"] = cfg
}
}
// withTestDataPath sets the test data path when used as an argument to initSettings.
func withTestDataPath(path string) Configuration {
return func(settings *environment.AirshipCTLSettings) {
manifest, err := settings.Config.CurrentContextManifest()
if err != nil {
panic(fmt.Sprintf("Unable to initialize management tests. Current Context error %q", err))
}
manifest.TargetPath = fmt.Sprintf("testdata/%s", path)
}
}
func TestNewManagerEphemeralHost(t *testing.T) {
settings := initSettings(t, withTestDataPath("base"))
manager, err := NewManager(settings, config.BootstrapPhase, ByLabel(document.EphemeralHostSelector))
require.NoError(t, err)
require.Equal(t, 1, len(manager.Hosts))
assert.Equal(t, "ephemeral", manager.Hosts[0].NodeID())
}
func TestNewManagerByName(t *testing.T) {
settings := initSettings(t, withTestDataPath("base"))
manager, err := NewManager(settings, config.BootstrapPhase, ByName("master-1"))
require.NoError(t, err)
require.Equal(t, 1, len(manager.Hosts))
assert.Equal(t, "node-master-1", manager.Hosts[0].NodeID())
}
func TestNewManagerMultipleNodes(t *testing.T) {
settings := initSettings(t, withTestDataPath("base"))
manager, err := NewManager(settings, config.BootstrapPhase, ByLabel("airshipit.org/test-node=true"))
require.NoError(t, err)
require.Equal(t, 2, len(manager.Hosts))
assert.Equal(t, "node-master-1", manager.Hosts[0].NodeID())
assert.Equal(t, "node-master-2", manager.Hosts[1].NodeID())
}
func TestNewManagerByNameNoHostFound(t *testing.T) {
settings := initSettings(t, withTestDataPath("base"))
_, err := NewManager(settings, config.BootstrapPhase, ByName("bad-name"))
assert.Error(t, err)
}
func TestNewManagerByLabelNoHostsFound(t *testing.T) {
settings := initSettings(t, withTestDataPath("base"))
_, err := NewManager(settings, config.BootstrapPhase, ByLabel("bad-label=true"))
assert.Error(t, err)
}
func TestNewManagerUnknownRemoteType(t *testing.T) {
badCfg := &config.ManagementConfiguration{Type: "bad-remote-type"}
settings := initSettings(t, withManagementConfig(badCfg), withTestDataPath("base"))
_, err := NewManager(settings, config.BootstrapPhase, ByLabel(document.EphemeralHostSelector))
assert.Error(t, err)
}
func TestNewManagerMissingBMCAddress(t *testing.T) {
settings := initSettings(t, withTestDataPath("emptyurl"))
_, err := NewManager(settings, config.BootstrapPhase, ByLabel(document.EphemeralHostSelector))
assert.Error(t, err)
}
func TestNewManagerMissingCredentials(t *testing.T) {
settings := initSettings(t, withTestDataPath("emptyurl"))
_, err := NewManager(settings, config.BootstrapPhase, ByName("no-creds"))
assert.Error(t, err)
}
func TestNewManagerBadPhase(t *testing.T) {
settings := initSettings(t, withTestDataPath("base"))
_, err := NewManager(settings, "bad-phase", ByLabel(document.EphemeralHostSelector))
assert.Error(t, err)
}

View File

@ -17,7 +17,6 @@ import (
"crypto/tls" "crypto/tls"
"fmt" "fmt"
"net/http" "net/http"
"net/url"
"strings" "strings"
"time" "time"
@ -36,20 +35,18 @@ const (
// Client holds details about a Redfish out-of-band system required for out-of-band management. // Client holds details about a Redfish out-of-band system required for out-of-band management.
type Client struct { type Client struct {
ephemeralNodeID string nodeID string
isoPath string RedfishAPI redfishAPI.RedfishAPI
redfishURL url.URL RedfishCFG *redfishClient.Configuration
RedfishAPI redfishAPI.RedfishAPI
RedfishCFG *redfishClient.Configuration
} }
// EphemeralNodeID retrieves the ephemeral node ID. // NodeID retrieves the ephemeral node ID.
func (c *Client) EphemeralNodeID() string { func (c *Client) NodeID() string {
return c.ephemeralNodeID return c.nodeID
} }
// RebootSystem power cycles a host by sending a shutdown signal followed by a power on signal. // RebootSystem power cycles a host by sending a shutdown signal followed by a power on signal.
func (c *Client) RebootSystem(ctx context.Context, systemID string) error { func (c *Client) RebootSystem(ctx context.Context) error {
waitForPowerState := func(desiredState redfishClient.PowerState) error { waitForPowerState := func(desiredState redfishClient.PowerState) error {
// Check if number of retries is defined in context // Check if number of retries is defined in context
totalRetries, ok := ctx.Value("numRetries").(int) totalRetries, ok := ctx.Value("numRetries").(int)
@ -57,8 +54,8 @@ func (c *Client) RebootSystem(ctx context.Context, systemID string) error {
totalRetries = systemActionRetries totalRetries = systemActionRetries
} }
for retry := 0; retry < totalRetries; retry++ { for retry := 0; retry <= totalRetries; retry++ {
system, httpResp, err := c.RedfishAPI.GetSystem(ctx, systemID) system, httpResp, err := c.RedfishAPI.GetSystem(ctx, c.nodeID)
if err = ScreenRedfishError(httpResp, err); err != nil { if err = ScreenRedfishError(httpResp, err); err != nil {
return err return err
} }
@ -68,7 +65,7 @@ func (c *Client) RebootSystem(ctx context.Context, systemID string) error {
time.Sleep(systemRebootDelay) time.Sleep(systemRebootDelay)
} }
return ErrOperationRetriesExceeded{ return ErrOperationRetriesExceeded{
What: fmt.Sprintf("reboot system %s", c.EphemeralNodeID()), What: fmt.Sprintf("reboot system %s", c.nodeID),
Retries: totalRetries, Retries: totalRetries,
} }
} }
@ -77,7 +74,7 @@ func (c *Client) RebootSystem(ctx context.Context, systemID string) error {
// Send PowerOff request // Send PowerOff request
resetReq.ResetType = redfishClient.RESETTYPE_FORCE_OFF resetReq.ResetType = redfishClient.RESETTYPE_FORCE_OFF
_, httpResp, err := c.RedfishAPI.ResetSystem(ctx, systemID, resetReq) _, httpResp, err := c.RedfishAPI.ResetSystem(ctx, c.nodeID, resetReq)
if err = ScreenRedfishError(httpResp, err); err != nil { if err = ScreenRedfishError(httpResp, err); err != nil {
return err return err
} }
@ -89,7 +86,7 @@ func (c *Client) RebootSystem(ctx context.Context, systemID string) error {
// Send PowerOn request // Send PowerOn request
resetReq.ResetType = redfishClient.RESETTYPE_ON resetReq.ResetType = redfishClient.RESETTYPE_ON
_, httpResp, err = c.RedfishAPI.ResetSystem(ctx, systemID, resetReq) _, httpResp, err = c.RedfishAPI.ResetSystem(ctx, c.nodeID, resetReq)
if err = ScreenRedfishError(httpResp, err); err != nil { if err = ScreenRedfishError(httpResp, err); err != nil {
return err return err
} }
@ -98,18 +95,18 @@ func (c *Client) RebootSystem(ctx context.Context, systemID string) error {
return waitForPowerState(redfishClient.POWERSTATE_ON) return waitForPowerState(redfishClient.POWERSTATE_ON)
} }
// SetEphemeralBootSourceByType sets the boot source of the ephemeral node to one that's compatible with the boot // SetBootSourceByType sets the boot source of the ephemeral node to one that's compatible with the boot
// source type. // source type.
func (c *Client) SetEphemeralBootSourceByType(ctx context.Context) error { func (c *Client) SetBootSourceByType(ctx context.Context) error {
_, vMediaType, err := GetVirtualMediaID(ctx, c.RedfishAPI, c.ephemeralNodeID) _, vMediaType, err := GetVirtualMediaID(ctx, c.RedfishAPI, c.nodeID)
if err != nil { if err != nil {
return err return err
} }
// Retrieve system information, containing available boot sources // Retrieve system information, containing available boot sources
system, _, err := c.RedfishAPI.GetSystem(ctx, c.ephemeralNodeID) system, _, err := c.RedfishAPI.GetSystem(ctx, c.nodeID)
if err != nil { if err != nil {
return ErrRedfishClient{Message: fmt.Sprintf("Get System[%s] failed with err: %v", c.ephemeralNodeID, err)} return ErrRedfishClient{Message: fmt.Sprintf("Get System[%s] failed with err: %v", c.nodeID, err)}
} }
allowableValues := system.Boot.BootSourceOverrideTargetRedfishAllowableValues allowableValues := system.Boot.BootSourceOverrideTargetRedfishAllowableValues
@ -118,12 +115,12 @@ func (c *Client) SetEphemeralBootSourceByType(ctx context.Context) error {
/* set boot source */ /* set boot source */
systemReq := redfishClient.ComputerSystem{} systemReq := redfishClient.ComputerSystem{}
systemReq.Boot.BootSourceOverrideTarget = bootSource systemReq.Boot.BootSourceOverrideTarget = bootSource
_, httpResp, err := c.RedfishAPI.SetSystem(ctx, c.ephemeralNodeID, systemReq) _, httpResp, err := c.RedfishAPI.SetSystem(ctx, c.nodeID, systemReq)
return ScreenRedfishError(httpResp, err) return ScreenRedfishError(httpResp, err)
} }
} }
return ErrRedfishClient{Message: fmt.Sprintf("failed to set system[%s] boot source", c.ephemeralNodeID)} return ErrRedfishClient{Message: fmt.Sprintf("failed to set system[%s] boot source", c.nodeID)}
} }
// SetVirtualMedia injects a virtual media device to an established virtual media ID. This assumes that isoPath is // SetVirtualMedia injects a virtual media device to an established virtual media ID. This assumes that isoPath is
@ -151,16 +148,16 @@ func (c *Client) SetVirtualMedia(ctx context.Context, isoPath string) error {
return ErrOperationRetriesExceeded{What: fmt.Sprintf("eject media %s", vMediaID), Retries: totalRetries} return ErrOperationRetriesExceeded{What: fmt.Sprintf("eject media %s", vMediaID), Retries: totalRetries}
} }
log.Debugf("Ephemeral Node System ID: '%s'", c.ephemeralNodeID) log.Debugf("Setting virtual media for node: '%s'", c.nodeID)
managerID, err := GetManagerID(ctx, c.RedfishAPI, c.ephemeralNodeID) managerID, err := getManagerID(ctx, c.RedfishAPI, c.nodeID)
if err != nil { if err != nil {
return err return err
} }
log.Debugf("Ephemeral node managerID: '%s'", managerID) log.Debugf("Ephemeral node managerID: '%s'", managerID)
vMediaID, _, err := GetVirtualMediaID(ctx, c.RedfishAPI, c.ephemeralNodeID) vMediaID, _, err := GetVirtualMediaID(ctx, c.RedfishAPI, c.nodeID)
if err != nil { if err != nil {
return err return err
} }
@ -194,18 +191,18 @@ func (c *Client) SetVirtualMedia(ctx context.Context, isoPath string) error {
} }
// SystemPowerOff shuts down a host. // SystemPowerOff shuts down a host.
func (c *Client) SystemPowerOff(ctx context.Context, systemID string) error { func (c *Client) SystemPowerOff(ctx context.Context) error {
resetReq := redfishClient.ResetRequestBody{} resetReq := redfishClient.ResetRequestBody{}
resetReq.ResetType = redfishClient.RESETTYPE_FORCE_OFF resetReq.ResetType = redfishClient.RESETTYPE_FORCE_OFF
_, httpResp, err := c.RedfishAPI.ResetSystem(ctx, systemID, resetReq) _, httpResp, err := c.RedfishAPI.ResetSystem(ctx, c.nodeID, resetReq)
return ScreenRedfishError(httpResp, err) return ScreenRedfishError(httpResp, err)
} }
// SystemPowerStatus retrieves the power status of a host as a human-readable string. // SystemPowerStatus retrieves the power status of a host as a human-readable string.
func (c *Client) SystemPowerStatus(ctx context.Context, systemID string) (string, error) { func (c *Client) SystemPowerStatus(ctx context.Context) (string, error) {
computerSystem, httpResp, err := c.RedfishAPI.GetSystem(ctx, systemID) computerSystem, httpResp, err := c.RedfishAPI.GetSystem(ctx, c.nodeID)
if err = ScreenRedfishError(httpResp, err); err != nil { if err = ScreenRedfishError(httpResp, err); err != nil {
return "", err return "", err
} }
@ -214,9 +211,7 @@ func (c *Client) SystemPowerStatus(ctx context.Context, systemID string) (string
} }
// NewClient returns a client with the capability to make Redfish requests. // NewClient returns a client with the capability to make Redfish requests.
func NewClient(ephemeralNodeID string, func NewClient(redfishURL string,
isoPath string,
redfishURL string,
insecure bool, insecure bool,
useProxy bool, useProxy bool,
username string, username string,
@ -236,13 +231,13 @@ func NewClient(ephemeralNodeID string,
return ctx, nil, ErrRedfishMissingConfig{What: "Redfish URL"} return ctx, nil, ErrRedfishMissingConfig{What: "Redfish URL"}
} }
parsedURL, err := url.Parse(redfishURL) basePath, err := getBasePath(redfishURL)
if err != nil { if err != nil {
return ctx, nil, err return ctx, nil, err
} }
cfg := &redfishClient.Configuration{ cfg := &redfishClient.Configuration{
BasePath: redfishURL, BasePath: basePath,
DefaultHeader: make(map[string]string), DefaultHeader: make(map[string]string),
UserAgent: headerUserAgent, UserAgent: headerUserAgent,
} }
@ -268,12 +263,16 @@ func NewClient(ephemeralNodeID string,
Transport: transport, Transport: transport,
} }
// Retrieve system ID from end of Redfish URL
systemID := GetResourceIDFromURL(redfishURL)
if len(systemID) == 0 {
return ctx, nil, ErrRedfishMissingConfig{What: "management URL system ID"}
}
c := &Client{ c := &Client{
ephemeralNodeID: ephemeralNodeID, nodeID: systemID,
isoPath: isoPath, RedfishAPI: redfishClient.NewAPIClient(cfg).DefaultApi,
redfishURL: *parsedURL, RedfishCFG: cfg,
RedfishAPI: redfishClient.NewAPIClient(cfg).DefaultApi,
RedfishCFG: cfg,
} }
return ctx, c, nil return ctx, c, nil

View File

@ -29,18 +29,33 @@ import (
) )
const ( const (
ephemeralNodeID = "ephemeral-node-id" nodeID = "System.Embedded.1"
isoPath = "https://localhost:8080/debian.iso" isoPath = "https://localhost:8080/debian.iso"
redfishURL = "https://localhost:1234" redfishURL = "redfish+https://localhost:2224/Systems/System.Embedded.1"
) )
func TestNewClient(t *testing.T) { func TestNewClient(t *testing.T) {
_, _, err := NewClient(ephemeralNodeID, isoPath, redfishURL, false, false, "", "") _, _, err := NewClient(redfishURL, false, false, "", "")
assert.NoError(t, err)
}
func TestNewClientMissingSystemID(t *testing.T) {
badURL := "redfish+https://localhost:2224"
_, _, err := NewClient(badURL, false, false, "", "")
_, ok := err.(ErrRedfishMissingConfig)
assert.True(t, ok)
}
func TestNewClientNoRedfishMarking(t *testing.T) {
url := "https://localhost:2224/Systems/System.Embedded.1"
_, _, err := NewClient(url, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
} }
func TestNewClientAuth(t *testing.T) { func TestNewClientAuth(t *testing.T) {
ctx, _, err := NewClient(ephemeralNodeID, isoPath, redfishURL, false, false, "username", "password") ctx, _, err := NewClient(redfishURL, false, false, "username", "password")
assert.NoError(t, err) assert.NoError(t, err)
cAuth := ctx.Value(redfishClient.ContextBasicAuth) cAuth := ctx.Value(redfishClient.ContextBasicAuth)
@ -50,7 +65,7 @@ func TestNewClientAuth(t *testing.T) {
func TestNewClientEmptyRedfishURL(t *testing.T) { func TestNewClientEmptyRedfishURL(t *testing.T) {
// Redfish URL cannot be empty when creating a client. // Redfish URL cannot be empty when creating a client.
_, _, err := NewClient(ephemeralNodeID, isoPath, "", false, false, "", "") _, _, err := NewClient("", false, false, "", "")
assert.Error(t, err) assert.Error(t, err)
} }
@ -58,29 +73,31 @@ func TestRebootSystem(t *testing.T) {
m := &redfishMocks.RedfishAPI{} m := &redfishMocks.RedfishAPI{}
defer m.AssertExpectations(t) defer m.AssertExpectations(t)
ctx, client, err := NewClient(ephemeralNodeID, isoPath, redfishURL, false, false, "", "") ctx, client, err := NewClient(redfishURL, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
client.nodeID = nodeID
// Mock redfish shutdown and status requests // Mock redfish shutdown and status requests
resetReq := redfishClient.ResetRequestBody{} resetReq := redfishClient.ResetRequestBody{}
resetReq.ResetType = redfishClient.RESETTYPE_FORCE_OFF resetReq.ResetType = redfishClient.RESETTYPE_FORCE_OFF
httpResp := &http.Response{StatusCode: 200} httpResp := &http.Response{StatusCode: 200}
m.On("ResetSystem", ctx, ephemeralNodeID, resetReq).Times(1).Return(redfishClient.RedfishError{}, httpResp, nil) m.On("ResetSystem", ctx, client.nodeID, resetReq).Times(1).Return(redfishClient.RedfishError{}, httpResp, nil)
m.On("GetSystem", ctx, ephemeralNodeID).Times(1).Return( m.On("GetSystem", ctx, client.nodeID).Times(1).Return(
redfishClient.ComputerSystem{PowerState: redfishClient.POWERSTATE_OFF}, httpResp, nil) redfishClient.ComputerSystem{PowerState: redfishClient.POWERSTATE_OFF}, httpResp, nil)
// Mock redfish startup and status requests // Mock redfish startup and status requests
resetReq.ResetType = redfishClient.RESETTYPE_ON resetReq.ResetType = redfishClient.RESETTYPE_ON
m.On("ResetSystem", ctx, ephemeralNodeID, resetReq).Times(1).Return(redfishClient.RedfishError{}, httpResp, nil) m.On("ResetSystem", ctx, client.nodeID, resetReq).Times(1).Return(redfishClient.RedfishError{}, httpResp, nil)
m.On("GetSystem", ctx, ephemeralNodeID).Times(1). m.On("GetSystem", ctx, client.nodeID).Times(1).
Return(redfishClient.ComputerSystem{PowerState: redfishClient.POWERSTATE_ON}, httpResp, nil) Return(redfishClient.ComputerSystem{PowerState: redfishClient.POWERSTATE_ON}, httpResp, nil)
// Replace normal API client with mocked API client // Replace normal API client with mocked API client
client.RedfishAPI = m client.RedfishAPI = m
err = client.RebootSystem(ctx, ephemeralNodeID) err = client.RebootSystem(ctx)
assert.NoError(t, err) assert.NoError(t, err)
} }
@ -88,20 +105,22 @@ func TestRebootSystemShutdownError(t *testing.T) {
m := &redfishMocks.RedfishAPI{} m := &redfishMocks.RedfishAPI{}
defer m.AssertExpectations(t) defer m.AssertExpectations(t)
ctx, client, err := NewClient(ephemeralNodeID, isoPath, redfishURL, false, false, "", "") ctx, client, err := NewClient(redfishURL, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
client.nodeID = nodeID
resetReq := redfishClient.ResetRequestBody{} resetReq := redfishClient.ResetRequestBody{}
resetReq.ResetType = redfishClient.RESETTYPE_FORCE_OFF resetReq.ResetType = redfishClient.RESETTYPE_FORCE_OFF
// Mock redfish shutdown request for failure // Mock redfish shutdown request for failure
m.On("ResetSystem", ctx, ephemeralNodeID, resetReq).Times(1).Return(redfishClient.RedfishError{}, m.On("ResetSystem", ctx, client.nodeID, resetReq).Times(1).Return(redfishClient.RedfishError{},
&http.Response{StatusCode: 401}, redfishClient.GenericOpenAPIError{}) &http.Response{StatusCode: 401}, redfishClient.GenericOpenAPIError{})
// Replace normal API client with mocked API client // Replace normal API client with mocked API client
client.RedfishAPI = m client.RedfishAPI = m
err = client.RebootSystem(ctx, ephemeralNodeID) err = client.RebootSystem(ctx)
_, ok := err.(ErrRedfishClient) _, ok := err.(ErrRedfishClient)
assert.True(t, ok) assert.True(t, ok)
} }
@ -110,18 +129,19 @@ func TestRebootSystemStartupError(t *testing.T) {
m := &redfishMocks.RedfishAPI{} m := &redfishMocks.RedfishAPI{}
defer m.AssertExpectations(t) defer m.AssertExpectations(t)
ctx, client, err := NewClient(ephemeralNodeID, isoPath, redfishURL, false, false, "", "") ctx, client, err := NewClient(redfishURL, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
client.nodeID = nodeID
resetReq := redfishClient.ResetRequestBody{} resetReq := redfishClient.ResetRequestBody{}
resetReq.ResetType = redfishClient.RESETTYPE_FORCE_OFF resetReq.ResetType = redfishClient.RESETTYPE_FORCE_OFF
// Mock redfish shutdown request // Mock redfish shutdown request
systemID := ephemeralNodeID m.On("ResetSystem", ctx, client.nodeID, resetReq).Times(1).Return(redfishClient.RedfishError{},
m.On("ResetSystem", ctx, systemID, resetReq).Times(1).Return(redfishClient.RedfishError{},
&http.Response{StatusCode: 200}, nil) &http.Response{StatusCode: 200}, nil)
m.On("GetSystem", ctx, systemID).Times(1).Return( m.On("GetSystem", ctx, client.nodeID).Times(1).Return(
redfishClient.ComputerSystem{PowerState: redfishClient.POWERSTATE_OFF}, redfishClient.ComputerSystem{PowerState: redfishClient.POWERSTATE_OFF},
&http.Response{StatusCode: 200}, nil) &http.Response{StatusCode: 200}, nil)
@ -129,13 +149,13 @@ func TestRebootSystemStartupError(t *testing.T) {
resetOnReq.ResetType = redfishClient.RESETTYPE_ON resetOnReq.ResetType = redfishClient.RESETTYPE_ON
// Mock redfish startup request for failure // Mock redfish startup request for failure
m.On("ResetSystem", ctx, systemID, resetOnReq).Times(1).Return(redfishClient.RedfishError{}, m.On("ResetSystem", ctx, client.nodeID, resetOnReq).Times(1).Return(redfishClient.RedfishError{},
&http.Response{StatusCode: 401}, redfishClient.GenericOpenAPIError{}) &http.Response{StatusCode: 401}, redfishClient.GenericOpenAPIError{})
// Replace normal API client with mocked API client // Replace normal API client with mocked API client
client.RedfishAPI = m client.RedfishAPI = m
err = client.RebootSystem(ctx, systemID) err = client.RebootSystem(ctx)
_, ok := err.(ErrRedfishClient) _, ok := err.(ErrRedfishClient)
assert.True(t, ok) assert.True(t, ok)
} }
@ -144,76 +164,83 @@ func TestRebootSystemTimeout(t *testing.T) {
m := &redfishMocks.RedfishAPI{} m := &redfishMocks.RedfishAPI{}
defer m.AssertExpectations(t) defer m.AssertExpectations(t)
_, client, err := NewClient(ephemeralNodeID, isoPath, redfishURL, false, false, "", "") _, client, err := NewClient(redfishURL, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
client.nodeID = nodeID
ctx := context.WithValue(context.Background(), "numRetries", 1) ctx := context.WithValue(context.Background(), "numRetries", 1)
resetReq := redfishClient.ResetRequestBody{} resetReq := redfishClient.ResetRequestBody{}
resetReq.ResetType = redfishClient.RESETTYPE_FORCE_OFF resetReq.ResetType = redfishClient.RESETTYPE_FORCE_OFF
systemID := ephemeralNodeID m.On("ResetSystem", ctx, client.nodeID, resetReq).
m.On("ResetSystem", ctx, systemID, resetReq).
Times(1). Times(1).
Return(redfishClient.RedfishError{}, &http.Response{StatusCode: 200}, nil) Return(redfishClient.RedfishError{}, &http.Response{StatusCode: 200}, nil)
m.On("GetSystem", ctx, systemID). m.On("GetSystem", ctx, client.nodeID).
Return(redfishClient.ComputerSystem{}, &http.Response{StatusCode: 200}, nil) Return(redfishClient.ComputerSystem{}, &http.Response{StatusCode: 200}, nil)
// Replace normal API client with mocked API client // Replace normal API client with mocked API client
client.RedfishAPI = m client.RedfishAPI = m
err = client.RebootSystem(ctx, systemID) err = client.RebootSystem(ctx)
assert.Equal(t, ErrOperationRetriesExceeded{What: "reboot system ephemeral-node-id", Retries: 1}, err) assert.Equal(t, ErrOperationRetriesExceeded{What: "reboot system System.Embedded.1", Retries: 1}, err)
} }
func TestSetEphemeralBootSourceByTypeGetSystemError(t *testing.T) { func TestSetBootSourceByTypeGetSystemError(t *testing.T) {
m := &redfishMocks.RedfishAPI{} m := &redfishMocks.RedfishAPI{}
defer m.AssertExpectations(t) defer m.AssertExpectations(t)
ctx, client, err := NewClient("invalid-server", isoPath, redfishURL, false, false, "", "") ctx, client, err := NewClient(redfishURL, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
client.nodeID = nodeID
// Mock redfish get system request // Mock redfish get system request
m.On("GetSystem", ctx, client.ephemeralNodeID).Times(1).Return(redfishClient.ComputerSystem{}, m.On("GetSystem", ctx, client.NodeID()).Times(1).Return(redfishClient.ComputerSystem{},
&http.Response{StatusCode: 500}, redfishClient.GenericOpenAPIError{}) &http.Response{StatusCode: 500}, redfishClient.GenericOpenAPIError{})
// Replace normal API client with mocked API client // Replace normal API client with mocked API client
client.RedfishAPI = m client.RedfishAPI = m
err = client.SetEphemeralBootSourceByType(ctx) err = client.SetBootSourceByType(ctx)
assert.Error(t, err) assert.Error(t, err)
} }
func TestSetEphemeralBootSourceByTypeSetSystemError(t *testing.T) { func TestSetBootSourceByTypeSetSystemError(t *testing.T) {
m := &redfishMocks.RedfishAPI{} m := &redfishMocks.RedfishAPI{}
defer m.AssertExpectations(t) defer m.AssertExpectations(t)
ctx, client, err := NewClient("invalid-server", isoPath, redfishURL, false, false, "", "") ctx, client, err := NewClient(redfishURL, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
client.nodeID = nodeID
httpResp := &http.Response{StatusCode: 200} httpResp := &http.Response{StatusCode: 200}
m.On("GetSystem", ctx, client.ephemeralNodeID).Return(testutil.GetTestSystem(), httpResp, nil) m.On("GetSystem", ctx, client.nodeID).Return(testutil.GetTestSystem(), httpResp, nil)
m.On("ListManagerVirtualMedia", ctx, testutil.ManagerID).Times(1). m.On("ListManagerVirtualMedia", ctx, testutil.ManagerID).Times(1).
Return(testutil.GetMediaCollection([]string{"Cd"}), httpResp, nil) Return(testutil.GetMediaCollection([]string{"Cd"}), httpResp, nil)
m.On("GetManagerVirtualMedia", ctx, testutil.ManagerID, "Cd").Times(1). m.On("GetManagerVirtualMedia", ctx, testutil.ManagerID, "Cd").Times(1).
Return(testutil.GetVirtualMedia([]string{"CD"}), httpResp, nil) Return(testutil.GetVirtualMedia([]string{"CD"}), httpResp, nil)
m.On("SetSystem", ctx, client.ephemeralNodeID, mock.Anything).Times(1).Return( m.On("SetSystem", ctx, client.nodeID, mock.Anything).Times(1).Return(
redfishClient.ComputerSystem{}, &http.Response{StatusCode: 401}, redfishClient.GenericOpenAPIError{}) redfishClient.ComputerSystem{}, &http.Response{StatusCode: 401}, redfishClient.GenericOpenAPIError{})
// Replace normal API client with mocked API client // Replace normal API client with mocked API client
client.RedfishAPI = m client.RedfishAPI = m
err = client.SetEphemeralBootSourceByType(ctx) err = client.SetBootSourceByType(ctx)
assert.Error(t, err) assert.Error(t, err)
} }
func TestSetEphemeralBootSourceByTypeBootSourceUnavailable(t *testing.T) { func TestSetBootSourceByTypeBootSourceUnavailable(t *testing.T) {
m := &redfishMocks.RedfishAPI{} m := &redfishMocks.RedfishAPI{}
defer m.AssertExpectations(t) defer m.AssertExpectations(t)
ctx, client, err := NewClient("invalid-server", isoPath, redfishURL, false, false, "", "") ctx, client, err := NewClient(redfishURL, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
client.nodeID = nodeID
invalidSystem := testutil.GetTestSystem() invalidSystem := testutil.GetTestSystem()
invalidSystem.Boot.BootSourceOverrideTargetRedfishAllowableValues = []redfishClient.BootSource{ invalidSystem.Boot.BootSourceOverrideTargetRedfishAllowableValues = []redfishClient.BootSource{
redfishClient.BOOTSOURCE_HDD, redfishClient.BOOTSOURCE_HDD,
@ -221,7 +248,7 @@ func TestSetEphemeralBootSourceByTypeBootSourceUnavailable(t *testing.T) {
} }
httpResp := &http.Response{StatusCode: 200} httpResp := &http.Response{StatusCode: 200}
m.On("GetSystem", ctx, client.ephemeralNodeID).Return(invalidSystem, httpResp, nil) m.On("GetSystem", ctx, client.nodeID).Return(invalidSystem, httpResp, nil)
m.On("ListManagerVirtualMedia", ctx, testutil.ManagerID).Times(1). m.On("ListManagerVirtualMedia", ctx, testutil.ManagerID).Times(1).
Return(testutil.GetMediaCollection([]string{"Cd"}), httpResp, nil) Return(testutil.GetMediaCollection([]string{"Cd"}), httpResp, nil)
m.On("GetManagerVirtualMedia", ctx, testutil.ManagerID, "Cd").Times(1). m.On("GetManagerVirtualMedia", ctx, testutil.ManagerID, "Cd").Times(1).
@ -230,7 +257,7 @@ func TestSetEphemeralBootSourceByTypeBootSourceUnavailable(t *testing.T) {
// Replace normal API client with mocked API client // Replace normal API client with mocked API client
client.RedfishAPI = m client.RedfishAPI = m
err = client.SetEphemeralBootSourceByType(ctx) err = client.SetBootSourceByType(ctx)
_, ok := err.(ErrRedfishClient) _, ok := err.(ErrRedfishClient)
assert.True(t, ok) assert.True(t, ok)
} }
@ -239,10 +266,11 @@ func TestSetVirtualMediaEjectVirtualMedia(t *testing.T) {
m := &redfishMocks.RedfishAPI{} m := &redfishMocks.RedfishAPI{}
defer m.AssertExpectations(t) defer m.AssertExpectations(t)
systemID := ephemeralNodeID _, client, err := NewClient(redfishURL, false, false, "", "")
_, client, err := NewClient(systemID, isoPath, redfishURL, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
client.nodeID = nodeID
// Normal retries are 30. Limit them here for test time. // Normal retries are 30. Limit them here for test time.
ctx := context.WithValue(context.Background(), "numRetries", 1) ctx := context.WithValue(context.Background(), "numRetries", 1)
@ -252,7 +280,7 @@ func TestSetVirtualMediaEjectVirtualMedia(t *testing.T) {
testMedia.Inserted = &inserted testMedia.Inserted = &inserted
httpResp := &http.Response{StatusCode: 200} httpResp := &http.Response{StatusCode: 200}
m.On("GetSystem", ctx, client.ephemeralNodeID).Return(testutil.GetTestSystem(), httpResp, nil) m.On("GetSystem", ctx, client.nodeID).Return(testutil.GetTestSystem(), httpResp, nil)
m.On("ListManagerVirtualMedia", ctx, testutil.ManagerID).Times(1). m.On("ListManagerVirtualMedia", ctx, testutil.ManagerID).Times(1).
Return(testutil.GetMediaCollection([]string{"Cd"}), httpResp, nil) Return(testutil.GetMediaCollection([]string{"Cd"}), httpResp, nil)
m.On("GetManagerVirtualMedia", ctx, testutil.ManagerID, "Cd").Times(1). m.On("GetManagerVirtualMedia", ctx, testutil.ManagerID, "Cd").Times(1).
@ -271,7 +299,7 @@ func TestSetVirtualMediaEjectVirtualMedia(t *testing.T) {
// Replace normal API client with mocked API client // Replace normal API client with mocked API client
client.RedfishAPI = m client.RedfishAPI = m
err = client.SetVirtualMedia(ctx, client.isoPath) err = client.SetVirtualMedia(ctx, isoPath)
assert.NoError(t, err) assert.NoError(t, err)
} }
@ -279,17 +307,19 @@ func TestSetVirtualMediaGetSystemError(t *testing.T) {
m := &redfishMocks.RedfishAPI{} m := &redfishMocks.RedfishAPI{}
defer m.AssertExpectations(t) defer m.AssertExpectations(t)
ctx, client, err := NewClient("invalid-server", isoPath, redfishURL, false, false, "", "") ctx, client, err := NewClient(redfishURL, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
client.nodeID = nodeID
// Mock redfish get system request // Mock redfish get system request
m.On("GetSystem", ctx, client.ephemeralNodeID).Times(1).Return(redfishClient.ComputerSystem{}, m.On("GetSystem", ctx, client.nodeID).Times(1).Return(redfishClient.ComputerSystem{},
nil, redfishClient.GenericOpenAPIError{}) nil, redfishClient.GenericOpenAPIError{})
// Replace normal API client with mocked API client // Replace normal API client with mocked API client
client.RedfishAPI = m client.RedfishAPI = m
err = client.SetVirtualMedia(ctx, client.isoPath) err = client.SetVirtualMedia(ctx, isoPath)
assert.Error(t, err) assert.Error(t, err)
} }
@ -297,10 +327,11 @@ func TestSetVirtualMediaEjectVirtualMediaRetriesExceeded(t *testing.T) {
m := &redfishMocks.RedfishAPI{} m := &redfishMocks.RedfishAPI{}
defer m.AssertExpectations(t) defer m.AssertExpectations(t)
systemID := ephemeralNodeID _, client, err := NewClient(redfishURL, false, false, "", "")
_, client, err := NewClient(systemID, isoPath, redfishURL, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
client.nodeID = nodeID
ctx := context.WithValue(context.Background(), "numRetries", 1) ctx := context.WithValue(context.Background(), "numRetries", 1)
// Mark test media as inserted // Mark test media as inserted
@ -309,7 +340,7 @@ func TestSetVirtualMediaEjectVirtualMediaRetriesExceeded(t *testing.T) {
testMedia.Inserted = &inserted testMedia.Inserted = &inserted
httpResp := &http.Response{StatusCode: 200} httpResp := &http.Response{StatusCode: 200}
m.On("GetSystem", ctx, client.ephemeralNodeID).Return(testutil.GetTestSystem(), httpResp, nil) m.On("GetSystem", ctx, client.nodeID).Return(testutil.GetTestSystem(), httpResp, nil)
m.On("ListManagerVirtualMedia", ctx, testutil.ManagerID).Times(1). m.On("ListManagerVirtualMedia", ctx, testutil.ManagerID).Times(1).
Return(testutil.GetMediaCollection([]string{"Cd"}), httpResp, nil) Return(testutil.GetMediaCollection([]string{"Cd"}), httpResp, nil)
m.On("GetManagerVirtualMedia", ctx, testutil.ManagerID, "Cd").Times(1). m.On("GetManagerVirtualMedia", ctx, testutil.ManagerID, "Cd").Times(1).
@ -328,7 +359,7 @@ func TestSetVirtualMediaEjectVirtualMediaRetriesExceeded(t *testing.T) {
// Replace normal API client with mocked API client // Replace normal API client with mocked API client
client.RedfishAPI = m client.RedfishAPI = m
err = client.SetVirtualMedia(ctx, client.isoPath) err = client.SetVirtualMedia(ctx, isoPath)
_, ok := err.(ErrOperationRetriesExceeded) _, ok := err.(ErrOperationRetriesExceeded)
assert.True(t, ok) assert.True(t, ok)
} }
@ -337,12 +368,13 @@ func TestSetVirtualMediaInsertVirtualMediaError(t *testing.T) {
m := &redfishMocks.RedfishAPI{} m := &redfishMocks.RedfishAPI{}
defer m.AssertExpectations(t) defer m.AssertExpectations(t)
systemID := ephemeralNodeID ctx, client, err := NewClient(redfishURL, false, false, "", "")
ctx, client, err := NewClient(systemID, isoPath, redfishURL, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
client.nodeID = nodeID
httpResp := &http.Response{StatusCode: 200} httpResp := &http.Response{StatusCode: 200}
m.On("GetSystem", ctx, client.ephemeralNodeID).Return(testutil.GetTestSystem(), httpResp, nil) m.On("GetSystem", ctx, client.nodeID).Return(testutil.GetTestSystem(), httpResp, nil)
m.On("ListManagerVirtualMedia", ctx, testutil.ManagerID).Times(1). m.On("ListManagerVirtualMedia", ctx, testutil.ManagerID).Times(1).
Return(testutil.GetMediaCollection([]string{"Cd"}), httpResp, nil) Return(testutil.GetMediaCollection([]string{"Cd"}), httpResp, nil)
m.On("GetManagerVirtualMedia", ctx, testutil.ManagerID, "Cd").Times(1). m.On("GetManagerVirtualMedia", ctx, testutil.ManagerID, "Cd").Times(1).
@ -355,7 +387,7 @@ func TestSetVirtualMediaInsertVirtualMediaError(t *testing.T) {
// Replace normal API client with mocked API client // Replace normal API client with mocked API client
client.RedfishAPI = m client.RedfishAPI = m
err = client.SetVirtualMedia(ctx, client.isoPath) err = client.SetVirtualMedia(ctx, isoPath)
_, ok := err.(ErrRedfishClient) _, ok := err.(ErrRedfishClient)
assert.True(t, ok) assert.True(t, ok)
} }

View File

@ -27,7 +27,7 @@ import (
) )
const ( const (
URLSchemeSeparator = "+" redfishURLSchemeSeparator = "+"
) )
// DecodeRawError decodes a raw Redfish HTTP response and retrieves the extended information and available resolutions // DecodeRawError decodes a raw Redfish HTTP response and retrieves the extended information and available resolutions
@ -220,3 +220,27 @@ func ScreenRedfishError(httpResp *http.Response, clientErr error) error {
return finalError return finalError
} }
func getManagerID(ctx context.Context, api redfishAPI.RedfishAPI, systemID string) (string, error) {
system, _, err := api.GetSystem(ctx, systemID)
if err != nil {
return "", err
}
return GetResourceIDFromURL(system.Links.ManagedBy[0].OdataId), nil
}
func getBasePath(redfishURL string) (string, error) {
parsedURL, err := url.Parse(redfishURL)
if err != nil {
return "", ErrRedfishClient{Message: fmt.Sprintf("Redfish URL malformed %s", err.Error())}
}
baseURL := fmt.Sprintf("%s://%s", parsedURL.Scheme, parsedURL.Host)
schemeSplit := strings.Split(parsedURL.Scheme, redfishURLSchemeSeparator)
if len(schemeSplit) > 1 {
baseURL = fmt.Sprintf("%s://%s", schemeSplit[len(schemeSplit)-1], parsedURL.Host)
}
return baseURL, nil
}

View File

@ -69,9 +69,9 @@ type iDRACAPIExtendedInfo struct {
Resolution string `json:"Resolution,omitempty"` Resolution string `json:"Resolution,omitempty"`
} }
// SetEphemeralBootSourceByType sets the boot source of the ephemeral node to a virtual CD, "VCD-DVD". // SetBootSourceByType sets the boot source of the ephemeral node to a virtual CD, "VCD-DVD".
func (c *Client) SetEphemeralBootSourceByType(ctx context.Context) error { func (c *Client) SetBootSourceByType(ctx context.Context) error {
managerID, err := redfish.GetManagerID(ctx, c.RedfishAPI, c.EphemeralNodeID()) managerID, err := redfish.GetManagerID(ctx, c.RedfishAPI, c.NodeID())
if err != nil { if err != nil {
return err return err
} }
@ -120,15 +120,12 @@ func (c *Client) SetEphemeralBootSourceByType(ctx context.Context) error {
} }
// NewClient returns a client with the capability to make Redfish requests. // NewClient returns a client with the capability to make Redfish requests.
func NewClient(ephemeralNodeID string, func NewClient(redfishURL string,
isoPath string,
redfishURL string,
insecure bool, insecure bool,
useProxy bool, useProxy bool,
username string, username string,
password string) (context.Context, *Client, error) { password string) (context.Context, *Client, error) {
ctx, genericClient, err := redfish.NewClient( ctx, genericClient, err := redfish.NewClient(redfishURL, insecure, useProxy, username, password)
ephemeralNodeID, isoPath, redfishURL, insecure, useProxy, username, password)
if err != nil { if err != nil {
return ctx, nil, err return ctx, nil, err
} }

View File

@ -1,14 +1,16 @@
// Licensed under the Apache License, Version 2.0 (the "License"); /*
// you may not use this file except in compliance with the License. Licensed under the Apache License, Version 2.0 (the "License");
// You may obtain a copy of the License at you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
// https://www.apache.org/licenses/LICENSE-2.0
// https://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS, Unless required by applicable law or agreed to in writing, software
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// limitations under the License. See the License for the specific language governing permissions and
limitations under the License.
*/
package dell package dell
@ -23,30 +25,28 @@ import (
) )
const ( const (
ephemeralNodeID = "System.Embedded.1" redfishURL = "redfish+https://localhost/Systems/System.Embedded.1"
isoPath = "https://localhost:8080/debian.iso"
redfishURL = "redfish+https://localhost/Systems/System.Embedded.1"
) )
func TestNewClient(t *testing.T) { func TestNewClient(t *testing.T) {
_, _, err := NewClient(ephemeralNodeID, isoPath, redfishURL, false, false, "username", "password") _, _, err := NewClient(redfishURL, false, false, "username", "password")
assert.NoError(t, err) assert.NoError(t, err)
} }
func TestSetEphemeralBootSourceByTypeGetSystemError(t *testing.T) { func TestSetBootSourceByTypeGetSystemError(t *testing.T) {
m := &redfishMocks.RedfishAPI{} m := &redfishMocks.RedfishAPI{}
defer m.AssertExpectations(t) defer m.AssertExpectations(t)
ctx, client, err := NewClient("invalid-server", isoPath, redfishURL, false, false, "", "") ctx, client, err := NewClient(redfishURL, false, false, "", "")
assert.NoError(t, err) assert.NoError(t, err)
// Mock redfish get system request // Mock redfish get system request
m.On("GetSystem", ctx, client.EphemeralNodeID()).Times(1).Return(redfishClient.ComputerSystem{}, m.On("GetSystem", ctx, client.NodeID()).Times(1).Return(redfishClient.ComputerSystem{},
&http.Response{StatusCode: 500}, redfishClient.GenericOpenAPIError{}) &http.Response{StatusCode: 500}, redfishClient.GenericOpenAPIError{})
// Replace normal API client with mocked API client // Replace normal API client with mocked API client
client.RedfishAPI = m client.RedfishAPI = m
err = client.SetEphemeralBootSourceByType(ctx) err = client.SetBootSourceByType(ctx)
assert.Error(t, err) assert.Error(t, err)
} }

View File

@ -1,186 +1,63 @@
// Licensed under the Apache License, Version 2.0 (the "License"); /*
// you may not use this file except in compliance with the License. Licensed under the Apache License, Version 2.0 (the "License");
// You may obtain a copy of the License at you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
// https://www.apache.org/licenses/LICENSE-2.0
// https://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS, Unless required by applicable law or agreed to in writing, software
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// limitations under the License. See the License for the specific language governing permissions and
limitations under the License.
*/
package remote package remote
import ( import (
"context"
"fmt"
"net/url"
"strings"
"opendev.org/airship/airshipctl/pkg/config" "opendev.org/airship/airshipctl/pkg/config"
"opendev.org/airship/airshipctl/pkg/document"
"opendev.org/airship/airshipctl/pkg/environment" "opendev.org/airship/airshipctl/pkg/environment"
alog "opendev.org/airship/airshipctl/pkg/log" "opendev.org/airship/airshipctl/pkg/log"
"opendev.org/airship/airshipctl/pkg/remote/redfish"
redfishDell "opendev.org/airship/airshipctl/pkg/remote/redfish/vendors/dell"
) )
// Adapter bridges the gap between out-of-band clients. It can hold any type of OOB client, e.g. Redfish. // DoRemoteDirect bootstraps the ephemeral node.
type Adapter struct { func (b baremetalHost) DoRemoteDirect(settings *environment.AirshipCTLSettings) error {
OOBClient Client
Context context.Context
remoteConfig *config.RemoteDirect
remoteURL string
username string
password string
}
// configureClient retrieves a client for remoteDirect requests based on the RemoteType in the Airship config file.
func (a *Adapter) configureClient(remoteURL string) error {
switch a.remoteConfig.RemoteType {
case redfish.ClientType, redfishDell.ClientType:
rfURL, err := url.Parse(remoteURL)
if err != nil {
return err
}
baseURL := fmt.Sprintf("%s://%s", rfURL.Scheme, rfURL.Host)
schemeSplit := strings.Split(rfURL.Scheme, redfish.URLSchemeSeparator)
if len(schemeSplit) > 1 {
baseURL = fmt.Sprintf("%s://%s", schemeSplit[len(schemeSplit)-1], rfURL.Host)
}
urlPath := strings.Split(rfURL.Path, "/")
nodeID := urlPath[len(urlPath)-1]
if nodeID == "" {
return redfish.ErrRedfishMissingConfig{
What: "redfish ephemeral node id empty",
}
}
if a.remoteConfig.IsoURL == "" {
return redfish.ErrRedfishMissingConfig{
What: "redfish ephemeral node iso Path empty",
}
}
if a.remoteConfig.RemoteType == redfishDell.ClientType {
alog.Debug("Remote type: Redfish for Integrated Dell Remote Access Controller (iDrac) systems")
a.Context, a.OOBClient, err = redfishDell.NewClient(
nodeID,
a.remoteConfig.IsoURL,
baseURL,
a.remoteConfig.Insecure,
a.remoteConfig.UseProxy,
a.username,
a.password)
} else {
alog.Debug("Remote type: Redfish")
a.Context, a.OOBClient, err = redfish.NewClient(
nodeID,
a.remoteConfig.IsoURL,
baseURL,
a.remoteConfig.Insecure,
a.remoteConfig.UseProxy,
a.username,
a.password)
}
if err != nil {
return err
}
default:
return NewRemoteDirectErrorf("invalid remote type")
}
return nil
}
// initializeAdapter retrieves the remote direct configuration defined in the Airship configuration file.
func (a *Adapter) initializeAdapter(settings *environment.AirshipCTLSettings) error {
cfg := settings.Config cfg := settings.Config
bootstrapSettings, err := cfg.CurrentContextBootstrapInfo() bootstrapSettings, err := cfg.CurrentContextBootstrapInfo()
if err != nil { if err != nil {
return err return err
} }
a.remoteConfig = bootstrapSettings.RemoteDirect remoteConfig := bootstrapSettings.RemoteDirect
if a.remoteConfig == nil { if remoteConfig == nil {
return config.ErrMissingConfig{What: "RemoteDirect options not defined in bootstrap config"} return config.ErrMissingConfig{What: "RemoteDirect options not defined in bootstrap config"}
} }
bundlePath, err := cfg.CurrentContextEntryPoint(config.Ephemeral, config.BootstrapPhase) log.Debugf("Using ephemeral node %s with BMCAddress %s", b.NodeID(), b.BMCAddress)
// Perform remote direct operations
if remoteConfig.IsoURL == "" {
return ErrMissingBootstrapInfoOption{What: "isoURL"}
}
err = b.SetVirtualMedia(b.Context, remoteConfig.IsoURL)
if err != nil { if err != nil {
return err return err
} }
docBundle, err := document.NewBundleByPath(bundlePath) log.Debugf("Successfully loaded virtual media: %q", remoteConfig.IsoURL)
err = b.SetBootSourceByType(b.Context)
if err != nil { if err != nil {
return err return err
} }
selector := document.NewEphemeralBMHSelector() err = b.RebootSystem(b.Context)
doc, err := docBundle.SelectOne(selector)
if err != nil { if err != nil {
return err return err
} }
a.remoteURL, err = document.GetBMHBMCAddress(doc) log.Debugf("Restarted ephemeral host %s", b.NodeID())
if err != nil {
return err
}
a.username, a.password, err = document.GetBMHBMCCredentials(doc, docBundle)
if err != nil {
return err
}
return nil return nil
} }
// DoRemoteDirect executes remote direct based on remote type.
func (a *Adapter) DoRemoteDirect() error {
alog.Debugf("Using Remote Endpoint: %q", a.remoteURL)
/* Load ISO in manager's virtual media */
err := a.OOBClient.SetVirtualMedia(a.Context, a.remoteConfig.IsoURL)
if err != nil {
return err
}
alog.Debugf("Successfully loaded virtual media: %q", a.remoteConfig.IsoURL)
/* Set system's bootsource to selected media */
err = a.OOBClient.SetEphemeralBootSourceByType(a.Context)
if err != nil {
return err
}
/* Reboot system */
err = a.OOBClient.RebootSystem(a.Context, a.OOBClient.EphemeralNodeID())
if err != nil {
return err
}
alog.Debug("Restarted ephemeral host")
return nil
}
// NewAdapter provides an adapter that exposes the capability to perform remote direct functionality with any
// out-of-band client.
func NewAdapter(settings *environment.AirshipCTLSettings) (*Adapter, error) {
a := &Adapter{}
a.Context = context.Background()
err := a.initializeAdapter(settings)
if err != nil {
return a, err
}
if err := a.configureClient(a.remoteURL); err != nil {
return a, err
}
return a, nil
}

View File

@ -15,205 +15,192 @@
package remote package remote
import ( import (
"fmt"
"testing" "testing"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"opendev.org/airship/airshipctl/pkg/config" "opendev.org/airship/airshipctl/pkg/config"
"opendev.org/airship/airshipctl/pkg/environment" "opendev.org/airship/airshipctl/pkg/environment"
"opendev.org/airship/airshipctl/pkg/remote/redfish" "opendev.org/airship/airshipctl/pkg/remote/redfish"
"opendev.org/airship/airshipctl/testutil"
"opendev.org/airship/airshipctl/testutil/redfishutils" "opendev.org/airship/airshipctl/testutil/redfishutils"
) )
const ( const (
systemID = "server-100" systemID = "System.Embedded.1"
isoURL = "https://localhost:8080/ubuntu.iso" isoURL = "https://localhost:8080/ubuntu.iso"
redfishURL = "https://redfish.local" redfishURL = "redfish+https://localhost:2344/Systems/System.Embedded.1"
username = "admin"
password = "password"
) )
func initSettings(t *testing.T, rd *config.RemoteDirect, testdata string) *environment.AirshipCTLSettings { // withRemoteDirectConfig initializes the remote direct settings when used as an argument to "initSettings".
t.Helper() func withRemoteDirectConfig(cfg *config.RemoteDirect) Configuration {
settings := &environment.AirshipCTLSettings{Config: testutil.DummyConfig()} return func(settings *environment.AirshipCTLSettings) {
bi, err := settings.Config.CurrentContextBootstrapInfo() bootstrapInfo, err := settings.Config.CurrentContextBootstrapInfo()
require.NoError(t, err) if err != nil {
bi.RemoteDirect = rd panic(fmt.Sprintf("Unable to initialize remote direct tests. Current Context error %q", err))
cm, err := settings.Config.CurrentContextManifest() }
require.NoError(t, err)
cm.TargetPath = "testdata/" + testdata bootstrapInfo.RemoteDirect = cfg
return settings }
} }
func TestUnknownRemoteType(t *testing.T) { func TestDoRemoteDirectMissingConfigOpts(t *testing.T) {
s := initSettings( ctx, rMock, err := redfishutils.NewClient(redfishURL, false, false, username, password)
t, assert.NoError(t, err)
&config.RemoteDirect{
RemoteType: "new-remote",
IsoURL: "/test.iso",
},
"base",
)
_, err := NewAdapter(s) ephemeralHost := baremetalHost{
_, ok := err.(*GenericError) rMock,
assert.True(t, ok) ctx,
redfishURL,
username,
password,
}
settings := initSettings(t, withRemoteDirectConfig(nil), withTestDataPath("base"))
err = ephemeralHost.DoRemoteDirect(settings)
assert.Error(t, err)
} }
func TestRedfishRemoteDirectWithEmptyURL(t *testing.T) { func TestDoRemoteDirectMissingISOURL(t *testing.T) {
s := initSettings( ctx, rMock, err := redfishutils.NewClient(redfishURL, false, false, username, password)
t, assert.NoError(t, err)
&config.RemoteDirect{
RemoteType: "redfish",
IsoURL: "/test.iso",
},
"emptyurl",
)
_, err := NewAdapter(s) rMock.On("NodeID").Times(1).Return(systemID)
_, ok := err.(redfish.ErrRedfishMissingConfig)
assert.True(t, ok)
}
func TestRedfishRemoteDirectWithEmptyIsoPath(t *testing.T) { ephemeralHost := baremetalHost{
s := initSettings( rMock,
t, ctx,
&config.RemoteDirect{ redfishURL,
RemoteType: "redfish", username,
IsoURL: "", password,
}, }
"base",
)
_, err := NewAdapter(s) cfg := &config.RemoteDirect{}
_, ok := err.(redfish.ErrRedfishMissingConfig)
assert.True(t, ok)
}
func TestBootstrapRemoteDirectMissingConfigOpts(t *testing.T) { settings := initSettings(t, withRemoteDirectConfig(cfg), withTestDataPath("base"))
s := initSettings( err = ephemeralHost.DoRemoteDirect(settings)
t, assert.Error(t, err)
nil,
"base",
)
_, err := NewAdapter(s)
_, ok := err.(config.ErrMissingConfig)
assert.True(t, ok)
} }
func TestDoRemoteDirectRedfish(t *testing.T) { func TestDoRemoteDirectRedfish(t *testing.T) {
cfg := &config.RemoteDirect{ ctx, rMock, err := redfishutils.NewClient(redfishURL, false, false, username, password)
RemoteType: redfish.ClientType, assert.NoError(t, err)
IsoURL: isoURL,
rMock.On("NodeID").Times(1).Return(systemID)
rMock.On("SetVirtualMedia", ctx, isoURL).Times(1).Return(nil)
rMock.On("SetBootSourceByType", ctx).Times(1).Return(nil)
rMock.On("NodeID").Times(1).Return(systemID)
rMock.On("RebootSystem", ctx).Times(1).Return(nil)
ephemeralHost := baremetalHost{
rMock,
ctx,
redfishURL,
username,
password,
} }
// Initialize a remote direct adapter cfg := &config.RemoteDirect{
settings := initSettings(t, cfg, "base") IsoURL: isoURL,
a, err := NewAdapter(settings) }
assert.NoError(t, err)
ctx, rMock, err := redfishutils.NewClient(systemID, isoURL, redfishURL, "admin", "password") settings := initSettings(t, withRemoteDirectConfig(cfg), withTestDataPath("base"))
assert.NoError(t, err) err = ephemeralHost.DoRemoteDirect(settings)
rMock.On("SetVirtualMedia", a.Context, isoURL).Times(1).Return(nil)
rMock.On("SetEphemeralBootSourceByType", a.Context).Times(1).Return(nil)
rMock.On("EphemeralNodeID").Times(1).Return(systemID)
rMock.On("RebootSystem", a.Context, systemID).Times(1).Return(nil)
// Swap the redfish client initialized by the remote direct adapter with the above mocked client
a.Context = ctx
a.OOBClient = rMock
err = a.DoRemoteDirect()
assert.NoError(t, err) assert.NoError(t, err)
} }
func TestDoRemoteDirectRedfishVirtualMediaError(t *testing.T) { func TestDoRemoteDirectRedfishVirtualMediaError(t *testing.T) {
cfg := &config.RemoteDirect{ ctx, rMock, err := redfishutils.NewClient(redfishURL, false, false, username, password)
RemoteType: redfish.ClientType,
IsoURL: isoURL,
}
// Initialize a remote direct adapter
settings := initSettings(t, cfg, "base")
a, err := NewAdapter(settings)
assert.NoError(t, err)
ctx, rMock, err := redfishutils.NewClient(systemID, isoURL, redfishURL, "admin", "password")
assert.NoError(t, err) assert.NoError(t, err)
expectedErr := redfish.ErrRedfishClient{Message: "Unable to set virtual media."} expectedErr := redfish.ErrRedfishClient{Message: "Unable to set virtual media."}
rMock.On("SetVirtualMedia", a.Context, isoURL).Times(1).Return(expectedErr)
rMock.On("SetEphemeralBootSourceByType", a.Context).Times(1).Return(nil)
rMock.On("EphemeralNodeID").Times(1).Return(systemID)
rMock.On("RebootSystem", a.Context, systemID).Times(1).Return(nil)
// Swap the redfish client initialized by the remote direct adapter with the above mocked client rMock.On("NodeID").Times(1).Return(systemID)
a.Context = ctx rMock.On("SetVirtualMedia", ctx, isoURL).Times(1).Return(expectedErr)
a.OOBClient = rMock rMock.On("SetBootSourceByType", ctx).Times(1).Return(nil)
rMock.On("NodeID").Times(1).Return(systemID)
rMock.On("RebootSystem", ctx).Times(1).Return(nil)
err = a.DoRemoteDirect() ephemeralHost := baremetalHost{
rMock,
ctx,
redfishURL,
username,
password,
}
cfg := &config.RemoteDirect{
IsoURL: isoURL,
}
settings := initSettings(t, withRemoteDirectConfig(cfg), withTestDataPath("base"))
err = ephemeralHost.DoRemoteDirect(settings)
_, ok := err.(redfish.ErrRedfishClient) _, ok := err.(redfish.ErrRedfishClient)
assert.True(t, ok) assert.True(t, ok)
} }
func TestDoRemoteDirectRedfishBootSourceError(t *testing.T) { func TestDoRemoteDirectRedfishBootSourceError(t *testing.T) {
cfg := &config.RemoteDirect{ ctx, rMock, err := redfishutils.NewClient(redfishURL, false, false, username, password)
RemoteType: redfish.ClientType,
IsoURL: isoURL,
}
// Initialize a remote direct adapter
settings := initSettings(t, cfg, "base")
a, err := NewAdapter(settings)
assert.NoError(t, err) assert.NoError(t, err)
ctx, rMock, err := redfishutils.NewClient(systemID, isoURL, redfishURL, "admin", "password") rMock.On("NodeID").Times(1).Return(systemID)
assert.NoError(t, err) rMock.On("SetVirtualMedia", ctx, isoURL).Times(1).Return(nil)
rMock.On("SetVirtualMedia", a.Context, isoURL).Times(1).Return(nil)
expectedErr := redfish.ErrRedfishClient{Message: "Unable to set boot source."} expectedErr := redfish.ErrRedfishClient{Message: "Unable to set boot source."}
rMock.On("SetEphemeralBootSourceByType", a.Context).Times(1).Return(expectedErr) rMock.On("SetBootSourceByType", ctx).Times(1).Return(expectedErr)
rMock.On("EphemeralNodeID").Times(1).Return(systemID) rMock.On("NodeID").Times(1).Return(systemID)
rMock.On("RebootSystem", a.Context, systemID).Times(1).Return(nil) rMock.On("RebootSystem", ctx).Times(1).Return(nil)
// Swap the redfish client initialized by the remote direct adapter with the above mocked client ephemeralHost := baremetalHost{
a.Context = ctx rMock,
a.OOBClient = rMock ctx,
redfishURL,
username,
password,
}
err = a.DoRemoteDirect() cfg := &config.RemoteDirect{
IsoURL: isoURL,
}
settings := initSettings(t, withRemoteDirectConfig(cfg), withTestDataPath("base"))
err = ephemeralHost.DoRemoteDirect(settings)
_, ok := err.(redfish.ErrRedfishClient) _, ok := err.(redfish.ErrRedfishClient)
assert.True(t, ok) assert.True(t, ok)
} }
func TestDoRemoteDirectRedfishRebootError(t *testing.T) { func TestDoRemoteDirectRedfishRebootError(t *testing.T) {
cfg := &config.RemoteDirect{ ctx, rMock, err := redfishutils.NewClient(redfishURL, false, false, username, password)
RemoteType: redfish.ClientType,
IsoURL: isoURL,
}
// Initialize a remote direct adapter
settings := initSettings(t, cfg, "base")
a, err := NewAdapter(settings)
assert.NoError(t, err) assert.NoError(t, err)
ctx, rMock, err := redfishutils.NewClient(systemID, isoURL, redfishURL, "admin", "password") rMock.On("NodeID").Times(1).Return(systemID)
assert.NoError(t, err) rMock.On("SetVirtualMedia", ctx, isoURL).Times(1).Return(nil)
rMock.On("SetVirtualMedia", ctx, isoURL).Times(1).Return(nil)
rMock.On("SetVirtualMedia", a.Context, isoURL).Times(1).Return(nil) rMock.On("SetBootSourceByType", ctx).Times(1).Return(nil)
rMock.On("SetEphemeralBootSourceByType", a.Context).Times(1).Return(nil) rMock.On("NodeID").Times(1).Return(systemID)
rMock.On("EphemeralNodeID").Times(1).Return(systemID)
expectedErr := redfish.ErrRedfishClient{Message: "Unable to set boot source."} expectedErr := redfish.ErrRedfishClient{Message: "Unable to set boot source."}
rMock.On("RebootSystem", a.Context, systemID).Times(1).Return(expectedErr) rMock.On("RebootSystem", ctx).Times(1).Return(expectedErr)
// Swap the redfish client initialized by the remote direct adapter with the above mocked client ephemeralHost := baremetalHost{
a.Context = ctx rMock,
a.OOBClient = rMock ctx,
redfishURL,
username,
password,
}
err = a.DoRemoteDirect() cfg := &config.RemoteDirect{
IsoURL: isoURL,
}
settings := initSettings(t, withRemoteDirectConfig(cfg), withTestDataPath("base"))
err = ephemeralHost.DoRemoteDirect(settings)
_, ok := err.(redfish.ErrRedfishClient) _, ok := err.(redfish.ErrRedfishClient)
assert.True(t, ok) assert.True(t, ok)
} }

View File

@ -9,7 +9,7 @@ spec:
online: true online: true
bootMACAddress: 00:3b:8b:0c:ec:8b bootMACAddress: 00:3b:8b:0c:ec:8b
bmc: bmc:
address: redfish+http://nolocalhost:8888/redfish/v1/Systems/test-node address: redfish+http://nolocalhost:8888/redfish/v1/Systems/ephemeral
credentialsName: master-0-bmc-secret credentialsName: master-0-bmc-secret
--- ---
apiVersion: v1 apiVersion: v1
@ -23,3 +23,52 @@ data:
username: YWRtaW4= username: YWRtaW4=
password: cGFzc3dvcmQ= password: cGFzc3dvcmQ=
... ...
---
apiVersion: metal3.io/v1alpha1
kind: BareMetalHost
metadata:
labels:
airshipit.org/test-node: "true"
name: master-1
spec:
online: true
bootMACAddress: 00:3b:8b:0c:ec:8b
bmc:
address: redfish+http://nolocalhost:8888/redfish/v1/Systems/node-master-1
credentialsName: master-1-bmc-secret
---
apiVersion: metal3.io/v1alpha1
kind: BareMetalHost
metadata:
labels:
airshipit.org/test-node: "true"
name: master-2
spec:
online: true
bootMACAddress: 00:3b:8b:0c:ec:8b
bmc:
address: redfish+http://nolocalhost:8888/redfish/v1/Systems/node-master-2
credentialsName: master-1-bmc-secret
---
apiVersion: v1
kind: Secret
metadata:
labels:
airshipit.org/ephemeral-node: "true"
name: master-1-bmc-secret
type: Opaque
data:
username: YWRtaW4=
password: cGFzc3dvcmQ=
...
---
apiVersion: metal3.io/v1alpha1
kind: BareMetalHost
metadata:
name: no-creds
spec:
online: true
bootMACAddress: 00:3b:8b:0c:ec:8b
bmc:
address: redfish+http://nolocalhost:8888/redfish/v1/Systems/test-node
...

View File

@ -1,39 +0,0 @@
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// https://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package remote
import (
"context"
)
// Client is a set of functions that clients created for out-of-band power management and control should implement. The
// functions within client are used by power management commands and remote direct functionality.
type Client interface {
RebootSystem(context.Context, string) error
SystemPowerOff(context.Context, string) error
// TODO(drewwalters96): Should this be a string forever? We may want to define our own custom type, as the
// string format will be client dependent when we add new clients.
SystemPowerStatus(context.Context, string) (string, error)
EphemeralNodeID() string
// TODO(drewwalters96): This function may be too tightly coupled to remoteDirect operations. This could probably
// be combined with SetVirtualMedia.
SetEphemeralBootSourceByType(context.Context) error
// TODO(drewwalters96): This function is tightly coupled to Redfish. It should be combined with the
// SetBootSource operation and removed from the client interface.
SetVirtualMedia(context.Context, string) error
}

View File

@ -14,12 +14,19 @@ bootstrapInfo:
userDataFileName: user-data userDataFileName: user-data
networkConfigFileName: network-config networkConfigFileName: network-config
outputMetadataFileName: output-metadata.yaml outputMetadataFileName: output-metadata.yaml
managementConfiguration:
dummy_management_config:
type: {{ remote_type }}
insecure: {{ remote_insecure }}
useproxy: {{ remote_proxy }}
clusters: clusters:
dummycluster: dummycluster:
clusterType: clusterType:
ephemeral: ephemeral:
bootstrapInfo: dummy_bootstrap_config bootstrapInfo: dummy_bootstrap_config
clusterKubeconf: dummycluster_ephemeral clusterKubeconf: dummycluster_ephemeral
managementConfiguration: dummy_management_config
contexts: contexts:
dummy_cluster: dummy_cluster:
contextKubeconf: dummy_cluster contextKubeconf: dummy_cluster

View File

@ -14,7 +14,6 @@ package redfishutils
import ( import (
"context" "context"
"net/url"
"github.com/stretchr/testify/mock" "github.com/stretchr/testify/mock"
redfishClient "opendev.org/airship/go-redfish/client" redfishClient "opendev.org/airship/go-redfish/client"
@ -25,20 +24,18 @@ import (
// MockClient is a fake Redfish client for unit testing. // MockClient is a fake Redfish client for unit testing.
type MockClient struct { type MockClient struct {
mock.Mock mock.Mock
ephemeralNodeID string nodeID string
isoPath string
redfishURL url.URL
} }
// EphemeralNodeID provides a stubbed method that can be mocked to test functions that use the Redfish client without // NodeID provides a stubbed method that can be mocked to test functions that use the Redfish client without
// making any Redfish API calls or requiring the appropriate Redfish client settings. // making any Redfish API calls or requiring the appropriate Redfish client settings.
// //
// Example usage: // Example usage:
// client := redfishutils.NewClient() // client := redfishutils.NewClient()
// client.On("GetEphemeralNodeID").Return(<return values>) // client.On("NodeID").Return(<return values>)
// //
// err := client.GetEphemeralNodeID(<args>) // err := client.NodeID()
func (m *MockClient) EphemeralNodeID() string { func (m *MockClient) NodeID() string {
args := m.Called() args := m.Called()
return args.String(0) return args.String(0)
} }
@ -51,20 +48,20 @@ func (m *MockClient) EphemeralNodeID() string {
// client.On("RebootSystem").Return(<return values>) // client.On("RebootSystem").Return(<return values>)
// //
// err := client.RebootSystem(<args>) // err := client.RebootSystem(<args>)
func (m *MockClient) RebootSystem(ctx context.Context, systemID string) error { func (m *MockClient) RebootSystem(ctx context.Context) error {
args := m.Called(ctx, systemID) args := m.Called(ctx)
return args.Error(0) return args.Error(0)
} }
// SetEphemeralBootSourceByType provides a stubbed method that can be mocked to test functions that use the // SetBootSourceByType provides a stubbed method that can be mocked to test functions that use the
// Redfish client without making any Redfish API calls or requiring the appropriate Redfish client settings. // Redfish client without making any Redfish API calls or requiring the appropriate Redfish client settings.
// //
// Example usage: // Example usage:
// client := redfishutils.NewClient() // client := redfishutils.NewClient()
// client.On("SetEphemeralBootSourceByType").Return(<return values>) // client.On("SetBootSourceByType").Return(<return values>)
// //
// err := client.setEphemeralBootSourceByType(<args>) // err := client.SetBootSourceByType(<args>)
func (m *MockClient) SetEphemeralBootSourceByType(ctx context.Context) error { func (m *MockClient) SetBootSourceByType(ctx context.Context) error {
args := m.Called(ctx) args := m.Called(ctx)
return args.Error(0) return args.Error(0)
} }
@ -90,8 +87,8 @@ func (m *MockClient) SetVirtualMedia(ctx context.Context, isoPath string) error
// client.On("SystemPowerOff").Return(<return values>) // client.On("SystemPowerOff").Return(<return values>)
// //
// err := client.SystemPowerOff(<args>) // err := client.SystemPowerOff(<args>)
func (m *MockClient) SystemPowerOff(ctx context.Context, systemID string) error { func (m *MockClient) SystemPowerOff(ctx context.Context) error {
args := m.Called(ctx, systemID) args := m.Called(ctx)
return args.Error(0) return args.Error(0)
} }
@ -103,14 +100,14 @@ func (m *MockClient) SystemPowerOff(ctx context.Context, systemID string) error
// client.On("SystemPowerStatus").Return(<return values>) // client.On("SystemPowerStatus").Return(<return values>)
// //
// err := client.SystemPowerStatus(<args>) // err := client.SystemPowerStatus(<args>)
func (m *MockClient) SystemPowerStatus(ctx context.Context, systemID string) (string, error) { func (m *MockClient) SystemPowerStatus(ctx context.Context) (string, error) {
args := m.Called(ctx, systemID) args := m.Called(ctx)
return args.String(0), args.Error(1) return args.String(0), args.Error(1)
} }
// NewClient returns a mocked Redfish client in order to test functions that use the Redfish client without making any // NewClient returns a mocked Redfish client in order to test functions that use the Redfish client without making any
// Redfish API calls. // Redfish API calls.
func NewClient(ephemeralNodeID string, isoPath string, redfishURL string, username string, func NewClient(redfishURL string, insecure bool, useProxy bool, username string,
password string) (context.Context, *MockClient, error) { password string) (context.Context, *MockClient, error) {
var ctx context.Context var ctx context.Context
if username != "" && password != "" { if username != "" && password != "" {
@ -127,16 +124,13 @@ func NewClient(ephemeralNodeID string, isoPath string, redfishURL string, userna
return ctx, nil, redfish.ErrRedfishMissingConfig{What: "Redfish URL"} return ctx, nil, redfish.ErrRedfishMissingConfig{What: "Redfish URL"}
} }
parsedURL, err := url.Parse(redfishURL) // Retrieve system ID from end of Redfish URL
if err != nil { systemID := redfish.GetResourceIDFromURL(redfishURL)
return ctx, nil, err if len(systemID) == 0 {
return ctx, nil, redfish.ErrRedfishMissingConfig{What: "management URL system ID"}
} }
m := &MockClient{ m := &MockClient{nodeID: systemID}
ephemeralNodeID: ephemeralNodeID,
isoPath: isoPath,
redfishURL: *parsedURL,
}
return ctx, m, nil return ctx, m, nil
} }