Refacto sync pkg (#6163)

* Rename interface to SyncExtracter

* Inline PushParameters fields into SyncParameters

* Move SyncParameters to sync package

* Move ComponentName as parameter to SyncFiles

* Move SyncAdapter as handler

* Build syncclient with dependency injection system

* Move ForcePush logic outside of sync package + remove unused ForceBuild option
This commit is contained in:
Philippe Martin
2022-09-23 16:30:55 +02:00
committed by GitHub
parent dbf594054c
commit 2c0b2ee2a6
17 changed files with 1304 additions and 1348 deletions

View File

@@ -11,6 +11,7 @@ import (
"github.com/redhat-developer/odo/pkg/kclient"
"github.com/redhat-developer/odo/pkg/portForward"
"github.com/redhat-developer/odo/pkg/preference"
"github.com/redhat-developer/odo/pkg/sync"
"github.com/redhat-developer/odo/pkg/testingutil/filesystem"
"github.com/devfile/library/pkg/devfile/parser"
@@ -28,6 +29,7 @@ type DevClient struct {
portForwardClient portForward.Client
watchClient watch.Client
bindingClient binding.Client
syncClient sync.Client
filesystem filesystem.Filesystem
}
@@ -39,6 +41,7 @@ func NewDevClient(
portForwardClient portForward.Client,
watchClient watch.Client,
bindingClient binding.Client,
syncClient sync.Client,
filesystem filesystem.Filesystem,
) *DevClient {
return &DevClient{
@@ -47,6 +50,7 @@ func NewDevClient(
portForwardClient: portForwardClient,
watchClient: watchClient,
bindingClient: bindingClient,
syncClient: syncClient,
filesystem: filesystem,
}
}
@@ -64,7 +68,7 @@ func (o *DevClient) Start(
klog.V(4).Infoln("Creating new adapter")
adapter := component.NewKubernetesAdapter(
o.kubernetesClient, o.prefClient, o.portForwardClient, o.bindingClient,
o.kubernetesClient, o.prefClient, o.portForwardClient, o.bindingClient, o.syncClient,
component.AdapterContext{
ComponentName: componentName,
Context: path,
@@ -146,6 +150,7 @@ func (o *DevClient) regenerateComponentAdapterFromWatchParams(parameters watch.W
o.prefClient,
o.portForwardClient,
o.bindingClient,
o.syncClient,
component.AdapterContext{
ComponentName: parameters.ComponentName,
Context: parameters.Path,

View File

@@ -47,6 +47,7 @@ type Adapter struct {
prefClient preference.Client
portForwardClient portForward.Client
bindingClient binding.Client
syncClient sync.Client
AdapterContext
logger machineoutput.MachineEventLoggingClient
@@ -61,7 +62,6 @@ type AdapterContext struct {
FS filesystem.Filesystem // FS is the object used for building image component if present
}
var _ sync.SyncClient = (*Adapter)(nil)
var _ ComponentAdapter = (*Adapter)(nil)
// NewKubernetesAdapter returns a Devfile adapter for the targeted platform
@@ -70,6 +70,7 @@ func NewKubernetesAdapter(
prefClient preference.Client,
portForwardClient portForward.Client,
bindingClient binding.Client,
syncClient sync.Client,
context AdapterContext,
) Adapter {
return Adapter{
@@ -77,6 +78,7 @@ func NewKubernetesAdapter(
prefClient: prefClient,
portForwardClient: portForwardClient,
bindingClient: bindingClient,
syncClient: syncClient,
AdapterContext: context,
logger: machineoutput.NewMachineEventLoggingClient(),
}
@@ -193,21 +195,27 @@ func (a Adapter) Push(parameters adapters.PushParameters, componentStatus *watch
podChanged := componentStatus.State == watch.StateWaitDeployment
// Get a sync adapter. Check if project files have changed and sync accordingly
syncAdapter := sync.New(&a, a.kubeClient, a.ComponentName)
compInfo := adapters.ComponentInfo{
compInfo := sync.ComponentInfo{
ComponentName: a.ComponentName,
ContainerName: containerName,
PodName: pod.GetName(),
SyncFolder: syncFolder,
}
syncParams := adapters.SyncParameters{
PushParams: parameters,
CompInfo: compInfo,
ComponentExists: deploymentExists,
PodChanged: podChanged,
Files: getSyncFilesFromAttributes(pushDevfileCommands),
syncParams := sync.SyncParameters{
Path: parameters.Path,
WatchFiles: parameters.WatchFiles,
WatchDeletedFiles: parameters.WatchDeletedFiles,
IgnoredFiles: parameters.IgnoredFiles,
DevfileScanIndexForWatch: parameters.DevfileScanIndexForWatch,
SyncExtracter: a.ExtractProjectToComponent,
CompInfo: compInfo,
ForcePush: !deploymentExists || podChanged,
Files: getSyncFilesFromAttributes(pushDevfileCommands),
}
execRequired, err := syncAdapter.SyncFiles(syncParams)
execRequired, err := a.syncClient.SyncFiles(syncParams)
if err != nil {
componentStatus.State = watch.StateReady
return fmt.Errorf("failed to sync to component with name %s: %w", a.ComponentName, err)
@@ -577,7 +585,7 @@ func getFirstContainerWithSourceVolume(containers []corev1.Container) (string, s
}
// ExtractProjectToComponent extracts the project archive(tar) to the target path from the reader stdin
func (a Adapter) ExtractProjectToComponent(componentInfo adapters.ComponentInfo, targetPath string, stdin io.Reader) error {
func (a Adapter) ExtractProjectToComponent(componentInfo sync.ComponentInfo, targetPath string, stdin io.Reader) error {
return a.kubeClient.ExtractProjectToComponent(componentInfo.ContainerName, componentInfo.PodName, targetPath, stdin)
}

View File

@@ -129,7 +129,7 @@ func TestCreateOrUpdateComponent(t *testing.T) {
ctrl := gomock.NewController(t)
fakePrefClient := preference.NewMockClient(ctrl)
fakePrefClient.EXPECT().GetEphemeralSourceVolume()
componentAdapter := NewKubernetesAdapter(fkclient, fakePrefClient, nil, nil, adapterCtx)
componentAdapter := NewKubernetesAdapter(fkclient, fakePrefClient, nil, nil, nil, adapterCtx)
_, _, err := componentAdapter.createOrUpdateComponent(tt.running, tt.envInfo, libdevfile.DevfileCommands{}, nil)
// Checks for unexpected error cases

View File

@@ -19,7 +19,6 @@ import (
"github.com/redhat-developer/odo/pkg/log"
"github.com/redhat-developer/odo/pkg/remotecmd"
"github.com/redhat-developer/odo/pkg/service"
"github.com/redhat-developer/odo/pkg/sync"
"github.com/redhat-developer/odo/pkg/task"
"github.com/redhat-developer/odo/pkg/util"
)
@@ -35,7 +34,6 @@ type adapterHandler struct {
var _ libdevfile.Handler = (*adapterHandler)(nil)
var _ ComponentAdapter = (*adapterHandler)(nil)
var _ sync.SyncClient = (*adapterHandler)(nil)
func (a *adapterHandler) ApplyImage(img devfilev1.Component) error {
return image.BuildPushSpecificImage(a.FS, a.parameters.Path, img, true)

View File

@@ -3,7 +3,6 @@ package component
import (
"fmt"
"github.com/devfile/api/v2/pkg/apis/workspaces/v1alpha2"
devfilev1 "github.com/devfile/api/v2/pkg/apis/workspaces/v1alpha2"
appsv1 "k8s.io/api/apps/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
@@ -41,7 +40,7 @@ func (a *Adapter) pushDevfileKubernetesComponents(
labels map[string]string,
mode string,
reference metav1.OwnerReference,
) ([]v1alpha2.Component, error) {
) ([]devfilev1.Component, error) {
// fetch the "kubernetes inlined components" to create them on cluster
// from odo standpoint, these components contain yaml manifest of ServiceBinding
k8sComponents, err := devfile.GetKubernetesComponentsToPush(a.Devfile)

View File

@@ -12,7 +12,6 @@ type PushParameters struct {
WatchFiles []string // Optional: WatchFiles is the list of changed files detected by odo watch. If empty or nil, odo will check .odo/odo-file-index.json to determine changed files
WatchDeletedFiles []string // Optional: WatchDeletedFiles is the list of deleted files detected by odo watch. If empty or nil, odo will check .odo/odo-file-index.json to determine deleted files
IgnoredFiles []string // IgnoredFiles is the list of files to not push up to a component
ForceBuild bool // ForceBuild determines whether or not to push all of the files up to a component or just files that have changed, added or removed.
Show bool // Show tells whether the devfile command output should be shown on stdout
DevfileBuildCmd string // DevfileBuildCmd takes the build command through the command line and overwrites devfile build command
DevfileRunCmd string // DevfileRunCmd takes the run command through the command line and overwrites devfile run command
@@ -23,23 +22,3 @@ type PushParameters struct {
RandomPorts bool // True to forward containers ports on local random ports
ErrOut io.Writer // Writer to output forwarded port information
}
// SyncParameters is a struct containing the parameters to be used when syncing a devfile component
type SyncParameters struct {
PushParams PushParameters
CompInfo ComponentInfo
PodChanged bool
ComponentExists bool
Files map[string]string
}
// ComponentInfo is a struct that holds information about a component i.e.; pod name, container name, and source mount (if applicable)
type ComponentInfo struct {
PodName string
ContainerName string
SyncFolder string
}
func (ci ComponentInfo) IsEmpty() bool {
return len(ci.ContainerName) == 0
}

View File

@@ -16,6 +16,7 @@ import (
"github.com/redhat-developer/odo/pkg/logs"
"github.com/redhat-developer/odo/pkg/portForward"
"github.com/redhat-developer/odo/pkg/sync"
"github.com/redhat-developer/odo/pkg/alizer"
"github.com/redhat-developer/odo/pkg/dev"
@@ -64,6 +65,8 @@ const (
REGISTRY = "DEP_REGISTRY"
// STATE instantiates client for pkg/state
STATE = "DEP_STATE"
// SYNC instantiates client for pkg/sync
SYNC = "DEP_SYNC"
// WATCH instantiates client for pkg/watch
WATCH = "DEP_WATCH"
/* Add key for new package here */
@@ -75,13 +78,14 @@ var subdeps map[string][]string = map[string][]string{
ALIZER: {REGISTRY},
DELETE_COMPONENT: {KUBERNETES},
DEPLOY: {KUBERNETES},
DEV: {BINDING, KUBERNETES, PORT_FORWARD, PREFERENCE, WATCH, FILESYSTEM},
DEV: {BINDING, FILESYSTEM, KUBERNETES, PORT_FORWARD, PREFERENCE, SYNC, WATCH},
INIT: {ALIZER, FILESYSTEM, PREFERENCE, REGISTRY},
LOGS: {KUBERNETES},
PORT_FORWARD: {KUBERNETES, STATE},
PROJECT: {KUBERNETES_NULLABLE},
REGISTRY: {FILESYSTEM, PREFERENCE},
STATE: {FILESYSTEM},
SYNC: {KUBERNETES},
WATCH: {KUBERNETES, DELETE_COMPONENT, STATE},
BINDING: {PROJECT, KUBERNETES},
/* Add sub-dependencies here, if any */
@@ -102,6 +106,7 @@ type Clientset struct {
ProjectClient project.Client
RegistryClient registry.Client
StateClient state.Client
SyncClient sync.Client
WatchClient watch.Client
/* Add client by alphabetic order */
}
@@ -171,6 +176,9 @@ func Fetch(command *cobra.Command) (*Clientset, error) {
if isDefined(command, STATE) {
dep.StateClient = state.NewStateClient(dep.FS)
}
if isDefined(command, SYNC) {
dep.SyncClient = sync.NewSyncClient(dep.KubernetesClient)
}
if isDefined(command, WATCH) {
dep.WatchClient = watch.NewWatchClient(dep.KubernetesClient, dep.DeleteClient, dep.StateClient)
}
@@ -181,7 +189,7 @@ func Fetch(command *cobra.Command) (*Clientset, error) {
dep.PortForwardClient = portForward.NewPFClient(dep.KubernetesClient, dep.StateClient)
}
if isDefined(command, DEV) {
dep.DevClient = dev.NewDevClient(dep.KubernetesClient, dep.PreferenceClient, dep.PortForwardClient, dep.WatchClient, dep.BindingClient, dep.FS)
dep.DevClient = dev.NewDevClient(dep.KubernetesClient, dep.PreferenceClient, dep.PortForwardClient, dep.WatchClient, dep.BindingClient, dep.SyncClient, dep.FS)
}
/* Instantiate new clients here. Take care to instantiate after all sub-dependencies */

View File

@@ -1,297 +0,0 @@
package sync
import (
"fmt"
"os"
"path/filepath"
"strings"
"github.com/devfile/library/pkg/devfile/generator"
dfutil "github.com/devfile/library/pkg/util"
"github.com/redhat-developer/odo/pkg/devfile/adapters"
"github.com/redhat-developer/odo/pkg/kclient"
"github.com/redhat-developer/odo/pkg/remotecmd"
"github.com/redhat-developer/odo/pkg/util"
"k8s.io/klog"
)
// New instantiates a component adapter
func New(syncClient SyncClient, kubeClient kclient.ClientInterface, componentName string) Adapter {
return Adapter{
kubeClient: kubeClient,
SyncClient: syncClient,
ComponentName: componentName,
}
}
// Adapter is a component adapter implementation for sync
type Adapter struct {
kubeClient kclient.ClientInterface
SyncClient SyncClient
ComponentName string
}
// SyncFiles does a couple of things:
// if files changed/deleted are passed in from watch, it syncs them to the component
// otherwise, it checks which files have changed and syncs the delta
// it returns a boolean execRequired and an error. execRequired tells us if files have
// changed and devfile execution is required
func (a Adapter) SyncFiles(syncParameters adapters.SyncParameters) (bool, error) {
// Whether to write the indexer content to the index file path (resolvePath)
forceWrite := false
// Ret from Indexer function
var ret util.IndexerRet
var deletedFiles []string
var changedFiles []string
pushParameters := syncParameters.PushParams
isForcePush := pushParameters.ForceBuild || !syncParameters.ComponentExists || syncParameters.PodChanged
isWatch := len(pushParameters.WatchFiles) > 0 || len(pushParameters.WatchDeletedFiles) > 0
// When this function is invoked by watch, the logic is:
// 1) If this is the first time that watch has called Push (in this OS process), then generate the file index
// using the file indexer, and use that to sync files (eg don't use changed/deleted files list from watch at
// this stage; these will be found by the indexer run).
// - In the watch scenario, we need to first run the indexer for two reasons:
// - In cases where the index doesn't initially exist, we need to create it (so we can ADD to it in
// later calls to SyncFiles(...) )
// - Even if it does initially exist, there is no guarantee that the remote pod is consistent with it; so
// on our first invocation we need to compare the index with the remote pod (by regenerating the index
// and using the changes files list from that to sync the results.)
//
// 2) For every other push/sync call after the first, don't run the file indexer, instead we use
// the watch events to determine what changed, and ensure that the index is then updated based
// on the watch events (to ensure future calls are correct)
// True if the index was updated based on the deleted/changed files values from the watch (and
// thus the indexer doesn't need to run), false otherwise
indexRegeneratedByWatch := false
// If watch files are specified _and_ this is not the first call (by this process) to SyncFiles by the watch command, then insert the
// changed files into the existing file index, and delete removed files from the index
if isWatch && !syncParameters.PushParams.DevfileScanIndexForWatch {
err := updateIndexWithWatchChanges(pushParameters)
if err != nil {
return false, err
}
changedFiles = pushParameters.WatchFiles
deletedFiles = pushParameters.WatchDeletedFiles
deletedFiles, err = dfutil.RemoveRelativePathFromFiles(deletedFiles, pushParameters.Path)
if err != nil {
return false, fmt.Errorf("unable to remove relative path from list of changed/deleted files: %w", err)
}
indexRegeneratedByWatch = true
}
if !indexRegeneratedByWatch {
// Calculate the files to sync
// Tries to sync the deltas unless it is a forced push
// if it is a forced push (isForcePush) reset the index to do a full sync
absIgnoreRules := dfutil.GetAbsGlobExps(pushParameters.Path, pushParameters.IgnoredFiles)
// Before running the indexer, make sure the .odo folder exists (or else the index file will not get created)
odoFolder := filepath.Join(pushParameters.Path, ".odo")
if _, err := os.Stat(odoFolder); os.IsNotExist(err) {
err = os.Mkdir(odoFolder, 0750)
if err != nil {
return false, fmt.Errorf("unable to create directory: %w", err)
}
}
// If the pod changed, reset the index, which will cause the indexer to walk the directory
// tree and resync all local files.
// If it is a new component, reset index to make sure any previously existing file is cleaned up
if syncParameters.PodChanged || !syncParameters.ComponentExists {
err := util.DeleteIndexFile(pushParameters.Path)
if err != nil {
return false, fmt.Errorf("unable to reset the index file: %w", err)
}
}
// Run the indexer and find the modified/added/deleted/renamed files
var err error
ret, err = util.RunIndexerWithRemote(pushParameters.Path, pushParameters.IgnoredFiles, syncParameters.Files)
if err != nil {
return false, fmt.Errorf("unable to run indexer: %w", err)
}
if len(ret.FilesChanged) > 0 || len(ret.FilesDeleted) > 0 {
forceWrite = true
}
// apply the glob rules from the .gitignore/.odoignore file
// and ignore the files on which the rules apply and filter them out
filesChangedFiltered, filesDeletedFiltered := dfutil.FilterIgnores(ret.FilesChanged, ret.FilesDeleted, absIgnoreRules)
deletedFiles = append(filesDeletedFiltered, ret.RemoteDeleted...)
deletedFiles = append(deletedFiles, ret.RemoteDeleted...)
klog.V(4).Infof("List of files to be deleted: +%v", deletedFiles)
changedFiles = filesChangedFiltered
klog.V(4).Infof("List of files changed: +%v", changedFiles)
if len(filesChangedFiltered) == 0 && len(filesDeletedFiltered) == 0 && !isForcePush {
return false, nil
}
if isForcePush {
deletedFiles = append(deletedFiles, "*")
}
}
err := a.pushLocal(pushParameters.Path,
changedFiles,
deletedFiles,
isForcePush,
pushParameters.IgnoredFiles,
syncParameters.CompInfo,
ret,
)
if err != nil {
return false, fmt.Errorf("failed to sync to component with name %s: %w", a.ComponentName, err)
}
if forceWrite {
err = util.WriteFile(ret.NewFileMap, ret.ResolvedPath)
if err != nil {
return false, fmt.Errorf("failed to write file: %w", err)
}
}
return true, nil
}
// pushLocal syncs source code from the user's disk to the component
func (a Adapter) pushLocal(path string, files []string, delFiles []string, isForcePush bool, globExps []string, compInfo adapters.ComponentInfo, ret util.IndexerRet) error {
klog.V(4).Infof("Push: componentName: %s, path: %s, files: %s, delFiles: %s, isForcePush: %+v", a.ComponentName, path, files, delFiles, isForcePush)
// Edge case: check to see that the path is NOT empty.
emptyDir, err := dfutil.IsEmpty(path)
if err != nil {
return fmt.Errorf("unable to check directory: %s: %w", path, err)
} else if emptyDir {
return fmt.Errorf("directory/file %s is empty", path)
}
// Sync the files to the pod
syncFolder := compInfo.SyncFolder
if syncFolder != generator.DevfileSourceVolumeMount {
// Need to make sure the folder already exists on the component or else sync will fail
klog.V(4).Infof("Creating %s on the remote container if it doesn't already exist", syncFolder)
cmdArr := getCmdToCreateSyncFolder(syncFolder)
_, _, err = remotecmd.ExecuteCommand(cmdArr, a.kubeClient, compInfo.PodName, compInfo.ContainerName, false, nil, nil)
if err != nil {
return err
}
}
// If there were any files deleted locally, delete them remotely too.
if len(delFiles) > 0 {
cmdArr := getCmdToDeleteFiles(delFiles, syncFolder)
_, _, err = remotecmd.ExecuteCommand(cmdArr, a.kubeClient, compInfo.PodName, compInfo.ContainerName, false, nil, nil)
if err != nil {
return err
}
}
if !isForcePush {
if len(files) == 0 && len(delFiles) == 0 {
return nil
}
}
if isForcePush || len(files) > 0 {
klog.V(4).Infof("Copying files %s to pod", strings.Join(files, " "))
err = CopyFile(a.SyncClient, path, compInfo, syncFolder, files, globExps, ret)
if err != nil {
return fmt.Errorf("unable push files to pod: %w", err)
}
}
return nil
}
// updateIndexWithWatchChanges uses the pushParameters.WatchDeletedFiles and pushParamters.WatchFiles to update
// the existing index file; the index file is required to exist when this function is called.
func updateIndexWithWatchChanges(pushParameters adapters.PushParameters) error {
indexFilePath, err := util.ResolveIndexFilePath(pushParameters.Path)
if err != nil {
return fmt.Errorf("unable to resolve path: %s: %w", pushParameters.Path, err)
}
// Check that the path exists
_, err = os.Stat(indexFilePath)
if err != nil {
// This shouldn't happen: in the watch case, SyncFiles should first be called with 'DevfileScanIndexForWatch' set to true, which
// will generate the index. Then, all subsequent invocations of SyncFiles will run with 'DevfileScanIndexForWatch' set to false,
// which will not regenerate the index (merely updating it based on changed files.)
//
// If you see this error it means somehow watch's SyncFiles was called without the index being first generated (likely because the
// above mentioned pushParam wasn't set). See SyncFiles(...) for details.
return fmt.Errorf("resolved path doesn't exist: %s: %w", indexFilePath, err)
}
// Parse the existing index
fileIndex, err := util.ReadFileIndex(indexFilePath)
if err != nil {
return fmt.Errorf("unable to read index from path: %s: %w", indexFilePath, err)
}
rootDir := pushParameters.Path
// Remove deleted files from the existing index
for _, deletedFile := range pushParameters.WatchDeletedFiles {
relativePath, err := util.CalculateFileDataKeyFromPath(deletedFile, rootDir)
if err != nil {
klog.V(4).Infof("Error occurred for %s: %v", deletedFile, err)
continue
}
delete(fileIndex.Files, relativePath)
klog.V(4).Infof("Removing watch deleted file from index: %s", relativePath)
}
// Add changed files to the existing index
for _, addedOrModifiedFile := range pushParameters.WatchFiles {
relativePath, fileData, err := util.GenerateNewFileDataEntry(addedOrModifiedFile, rootDir)
if err != nil {
klog.V(4).Infof("Error occurred for %s: %v", addedOrModifiedFile, err)
continue
}
fileIndex.Files[relativePath] = *fileData
klog.V(4).Infof("Added/updated watched file in index: %s", relativePath)
}
// Write the result
return util.WriteFile(fileIndex.Files, indexFilePath)
}
// getCmdToCreateSyncFolder returns the command used to create the remote sync folder on the running container
func getCmdToCreateSyncFolder(syncFolder string) []string {
return []string{"mkdir", "-p", syncFolder}
}
// getCmdToDeleteFiles returns the command used to delete the remote files on the container that are marked for deletion
func getCmdToDeleteFiles(delFiles []string, syncFolder string) []string {
rmPaths := dfutil.GetRemoteFilesMarkedForDeletion(delFiles, syncFolder)
klog.V(4).Infof("remote files marked for deletion are %+v", rmPaths)
cmdArr := []string{"rm", "-rf"}
for _, remote := range rmPaths {
cmdArr = append(cmdArr, filepath.ToSlash(remote))
}
return cmdArr
}

View File

@@ -1,461 +0,0 @@
package sync
import (
"io/ioutil"
"os"
"path"
"path/filepath"
"reflect"
"testing"
"github.com/devfile/library/pkg/devfile/generator"
"github.com/golang/mock/gomock"
"github.com/redhat-developer/odo/pkg/devfile/adapters"
"github.com/redhat-developer/odo/pkg/kclient"
"github.com/redhat-developer/odo/pkg/sync/mock"
"github.com/redhat-developer/odo/pkg/util"
"github.com/redhat-developer/odo/tests/helper"
)
func TestGetCmdToCreateSyncFolder(t *testing.T) {
tests := []struct {
name string
syncFolder string
want []string
}{
{
name: "Case 1: Sync to /projects",
syncFolder: generator.DevfileSourceVolumeMount,
want: []string{"mkdir", "-p", generator.DevfileSourceVolumeMount},
},
{
name: "Case 2: Sync subdir of /projects",
syncFolder: generator.DevfileSourceVolumeMount + "/someproject",
want: []string{"mkdir", "-p", generator.DevfileSourceVolumeMount + "/someproject"},
},
}
for _, tt := range tests {
cmdArr := getCmdToCreateSyncFolder(tt.syncFolder)
if !reflect.DeepEqual(tt.want, cmdArr) {
t.Errorf("Expected %s, got %s", tt.want, cmdArr)
}
}
}
func TestGetCmdToDeleteFiles(t *testing.T) {
syncFolder := "/projects/hello-world"
tests := []struct {
name string
delFiles []string
syncFolder string
want []string
}{
{
name: "Case 1: One deleted file",
delFiles: []string{"test.txt"},
syncFolder: generator.DevfileSourceVolumeMount,
want: []string{"rm", "-rf", generator.DevfileSourceVolumeMount + "/test.txt"},
},
{
name: "Case 2: Multiple deleted files, default sync folder",
delFiles: []string{"test.txt", "hello.c"},
syncFolder: generator.DevfileSourceVolumeMount,
want: []string{"rm", "-rf", generator.DevfileSourceVolumeMount + "/test.txt", generator.DevfileSourceVolumeMount + "/hello.c"},
},
{
name: "Case 2: Multiple deleted files, different sync folder",
delFiles: []string{"test.txt", "hello.c"},
syncFolder: syncFolder,
want: []string{"rm", "-rf", syncFolder + "/test.txt", syncFolder + "/hello.c"},
},
}
for _, tt := range tests {
cmdArr := getCmdToDeleteFiles(tt.delFiles, tt.syncFolder)
if !reflect.DeepEqual(tt.want, cmdArr) {
t.Errorf("Expected %s, got %s", tt.want, cmdArr)
}
}
}
func TestSyncFiles(t *testing.T) {
testComponentName := "test"
// create a temp dir for the file indexer
directory, e := ioutil.TempDir("", "")
if e != nil {
t.Errorf("TestSyncFiles error: error creating temporary directory for the indexer: %v", e)
}
jsFile, e := os.Create(filepath.Join(directory, "red.js"))
if e != nil {
t.Errorf("TestSyncFiles error: error creating temporary file for the indexer: %v", e)
}
ctrl := gomock.NewController(t)
kc := kclient.NewMockClientInterface(ctrl)
kc.EXPECT().ExecCMDInContainer(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).
Return(nil).AnyTimes()
// Assert that Bar() is invoked.
defer ctrl.Finish()
syncClient := mock.NewMockSyncClient(ctrl)
syncClient.EXPECT().ExtractProjectToComponent(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
errorSyncClient := mock.NewMockSyncClient(ctrl)
errorSyncClient.EXPECT().ExtractProjectToComponent(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
tests := []struct {
name string
client SyncClient
syncParameters adapters.SyncParameters
wantErr bool
wantIsPushRequired bool
}{
{
name: "Case 1: Component does not exist",
client: syncClient,
syncParameters: adapters.SyncParameters{
PushParams: adapters.PushParameters{
Path: directory,
WatchFiles: []string{},
WatchDeletedFiles: []string{},
IgnoredFiles: []string{},
ForceBuild: false,
},
CompInfo: adapters.ComponentInfo{
ContainerName: "abcd",
},
ComponentExists: false,
},
wantErr: false,
wantIsPushRequired: true,
},
{
name: "Case 2: Component does exist",
client: syncClient,
syncParameters: adapters.SyncParameters{
PushParams: adapters.PushParameters{
Path: directory,
WatchFiles: []string{},
WatchDeletedFiles: []string{},
IgnoredFiles: []string{},
ForceBuild: false,
},
CompInfo: adapters.ComponentInfo{
ContainerName: "abcd",
},
ComponentExists: true,
},
wantErr: false,
wantIsPushRequired: false, // always false after case 1
},
{
name: "Case 3: FakeErrorClient error",
client: errorSyncClient,
syncParameters: adapters.SyncParameters{
PushParams: adapters.PushParameters{
Path: directory,
WatchFiles: []string{},
WatchDeletedFiles: []string{},
IgnoredFiles: []string{},
ForceBuild: true,
},
CompInfo: adapters.ComponentInfo{
ContainerName: "abcd",
},
ComponentExists: true,
},
wantErr: true,
wantIsPushRequired: false,
},
{
name: "Case 4: File change",
client: syncClient,
syncParameters: adapters.SyncParameters{
PushParams: adapters.PushParameters{
Path: directory,
WatchFiles: []string{path.Join(directory, "test.log")},
WatchDeletedFiles: []string{},
IgnoredFiles: []string{},
ForceBuild: false,
},
CompInfo: adapters.ComponentInfo{
ContainerName: "abcd",
},
ComponentExists: true,
},
wantErr: false,
wantIsPushRequired: true,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
syncAdapter := New(tt.client, kc, testComponentName)
isPushRequired, err := syncAdapter.SyncFiles(tt.syncParameters)
if !tt.wantErr && err != nil {
t.Errorf("TestSyncFiles error: unexpected error when syncing files %v", err)
} else if !tt.wantErr && isPushRequired != tt.wantIsPushRequired {
t.Errorf("TestSyncFiles error: isPushRequired mismatch, wanted: %v, got: %v", tt.wantIsPushRequired, isPushRequired)
}
})
}
err := jsFile.Close()
if err != nil {
t.Errorf("TestSyncFiles error: error deleting the temp dir %s, err: %v", directory, err)
}
// Remove the temp dir created for the file indexer
err = os.RemoveAll(directory)
if err != nil {
t.Errorf("TestSyncFiles error: error deleting the temp dir %s, err: %v", directory, err)
}
}
func TestPushLocal(t *testing.T) {
testComponentName := "test"
// create a temp dir for the file indexer
directory, e := ioutil.TempDir("", "")
if e != nil {
t.Errorf("TestPushLocal error: error creating temporary directory for the indexer: %v", e)
}
newFilePath := filepath.Join(directory, "foobar.txt")
if err := helper.CreateFileWithContent(newFilePath, "hello world"); err != nil {
t.Errorf("TestPushLocal error: the foobar.txt file was not created: %v", err)
}
ctrl := gomock.NewController(t)
kc := kclient.NewMockClientInterface(ctrl)
kc.EXPECT().ExecCMDInContainer(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).
Return(nil).AnyTimes()
// Assert that Bar() is invoked.
defer ctrl.Finish()
syncClient := mock.NewMockSyncClient(ctrl)
syncClient.EXPECT().ExtractProjectToComponent(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
errorSyncClient := mock.NewMockSyncClient(ctrl)
errorSyncClient.EXPECT().ExtractProjectToComponent(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes()
tests := []struct {
name string
client SyncClient
path string
files []string
delFiles []string
isForcePush bool
compInfo adapters.ComponentInfo
wantErr bool
}{
{
name: "Case 1: File change",
client: syncClient,
path: directory,
files: []string{path.Join(directory, "test.log")},
delFiles: []string{},
isForcePush: false,
compInfo: adapters.ComponentInfo{
ContainerName: "abcd",
},
wantErr: false,
},
{
name: "Case 2: File change with fake error client",
client: errorSyncClient,
path: directory,
files: []string{path.Join(directory, "test.log")},
delFiles: []string{},
isForcePush: false,
compInfo: adapters.ComponentInfo{
ContainerName: "abcd",
},
wantErr: true,
},
{
name: "Case 3: No file change",
client: syncClient,
path: directory,
files: []string{},
delFiles: []string{},
isForcePush: false,
compInfo: adapters.ComponentInfo{
ContainerName: "abcd",
},
wantErr: false,
},
{
name: "Case 4: Deleted file",
client: syncClient,
path: directory,
files: []string{},
delFiles: []string{path.Join(directory, "test.log")},
isForcePush: false,
compInfo: adapters.ComponentInfo{
ContainerName: "abcd",
},
wantErr: false,
},
{
name: "Case 5: Force push",
client: syncClient,
path: directory,
files: []string{},
delFiles: []string{},
isForcePush: true,
compInfo: adapters.ComponentInfo{
ContainerName: "abcd",
},
wantErr: false,
},
{
name: "Case 6: Source mapping folder set",
client: syncClient,
path: directory,
files: []string{},
delFiles: []string{},
isForcePush: false,
compInfo: adapters.ComponentInfo{
ContainerName: "abcd",
SyncFolder: "/some/path",
},
wantErr: false,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
syncAdapter := New(syncClient, kc, testComponentName)
err := syncAdapter.pushLocal(tt.path, tt.files, tt.delFiles, tt.isForcePush, []string{}, tt.compInfo, util.IndexerRet{})
if !tt.wantErr && err != nil {
t.Errorf("TestPushLocal error: error pushing files: %v", err)
}
})
}
// Remove the temp dir created for the file indexer
err := os.RemoveAll(directory)
if err != nil {
t.Errorf("TestPushLocal error: error deleting the temp dir %s", directory)
}
}
func TestUpdateIndexWithWatchChanges(t *testing.T) {
tests := []struct {
name string
initialFilesToCreate []string
watchDeletedFiles []string
watchAddedFiles []string
expectedFilesInIndex []string
}{
{
name: "Case 1 - Watch file deleted should remove file from index",
initialFilesToCreate: []string{"file1", "file2"},
watchDeletedFiles: []string{"file1"},
expectedFilesInIndex: []string{"file2"},
},
{
name: "Case 2 - Watch file added should add file to index",
initialFilesToCreate: []string{"file1"},
watchAddedFiles: []string{"file2"},
expectedFilesInIndex: []string{"file1", "file2"},
},
{
name: "Case 3 - No watch changes should mean no index changes",
initialFilesToCreate: []string{"file1"},
expectedFilesInIndex: []string{"file1"},
},
}
for _, tt := range tests {
// create a temp dir for the fake component
directory, err := ioutil.TempDir("", "")
if err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: error creating temporary directory for the indexer: %v", err)
}
fileIndexPath, err := util.ResolveIndexFilePath(directory)
if err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to resolve index file path: %v", err)
}
if err := os.MkdirAll(filepath.Dir(fileIndexPath), 0750); err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to create directories for %s: %v", fileIndexPath, err)
}
t.Run(tt.name, func(t *testing.T) {
indexData := map[string]util.FileData{}
// Create initial files
for _, fileToCreate := range tt.initialFilesToCreate {
filePath := filepath.Join(directory, fileToCreate)
if err := ioutil.WriteFile(filePath, []byte("non-empty-string"), 0644); err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to write to index file path: %v", err)
}
key, fileDatum, err := util.GenerateNewFileDataEntry(filePath, directory)
if err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to generate new file: %v", err)
}
indexData[key] = *fileDatum
}
// Write the index based on those files
if err := util.WriteFile(indexData, fileIndexPath); err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to write index file: %v", err)
}
pushParams := adapters.PushParameters{
Path: directory,
}
// Add deleted files to pushParams (also delete the files)
for _, deletedFile := range tt.watchDeletedFiles {
deletedFilePath := filepath.Join(directory, deletedFile)
pushParams.WatchDeletedFiles = append(pushParams.WatchDeletedFiles, deletedFilePath)
if err := os.Remove(deletedFilePath); err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to delete file %s %v", deletedFilePath, err)
}
}
// Add added files to pushParams (also create the files)
for _, addedFile := range tt.watchAddedFiles {
addedFilePath := filepath.Join(directory, addedFile)
pushParams.WatchFiles = append(pushParams.WatchFiles, addedFilePath)
if err := ioutil.WriteFile(addedFilePath, []byte("non-empty-string"), 0644); err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to write to index file path: %v", err)
}
}
if err := updateIndexWithWatchChanges(pushParams); err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal: unexpected error: %v", err)
}
postFileIndex, err := util.ReadFileIndex(fileIndexPath)
if err != nil || postFileIndex == nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: read new file index: %v", err)
}
// Locate expected files
if len(postFileIndex.Files) != len(tt.expectedFilesInIndex) {
t.Fatalf("Mismatch between number expected files and actual files in index, post-index: %v expected: %v", postFileIndex.Files, tt.expectedFilesInIndex)
}
for _, expectedFile := range tt.expectedFilesInIndex {
if _, exists := postFileIndex.Files[expectedFile]; !exists {
t.Fatalf("Unable to find '%s' in post index file, %v", expectedFile, postFileIndex.Files)
}
}
})
}
}

211
pkg/sync/copy.go Normal file
View File

@@ -0,0 +1,211 @@
package sync
import (
taro "archive/tar"
"fmt"
"io"
"os"
"path/filepath"
"github.com/redhat-developer/odo/pkg/log"
"github.com/redhat-developer/odo/pkg/testingutil/filesystem"
"github.com/redhat-developer/odo/pkg/util"
dfutil "github.com/devfile/library/pkg/util"
gitignore "github.com/sabhiram/go-gitignore"
"k8s.io/klog"
)
// CopyFile copies localPath directory or list of files in copyFiles list to the directory in running Pod.
// copyFiles is list of changed files captured during `odo watch` as well as binary file path
// During copying binary components, localPath represent base directory path to binary and copyFiles contains path of binary
// During copying local source components, localPath represent base directory path whereas copyFiles is empty
// During `odo watch`, localPath represent base directory path whereas copyFiles contains list of changed Files
func CopyFile(extracter SyncExtracter, localPath string, compInfo ComponentInfo, targetPath string, copyFiles []string, globExps []string, ret util.IndexerRet) error {
// Destination is set to "ToSlash" as all containers being ran within OpenShift / S2I are all
// Linux based and thus: "\opt\app-root\src" would not work correctly.
dest := filepath.ToSlash(filepath.Join(targetPath, filepath.Base(localPath)))
targetPath = filepath.ToSlash(targetPath)
klog.V(4).Infof("CopyFile arguments: localPath %s, dest %s, targetPath %s, copyFiles %s, globalExps %s", localPath, dest, targetPath, copyFiles, globExps)
reader, writer := io.Pipe()
// inspired from https://github.com/kubernetes/kubernetes/blob/master/pkg/kubectl/cmd/cp.go#L235
go func() {
defer writer.Close()
err := makeTar(localPath, dest, writer, copyFiles, globExps, ret, filesystem.DefaultFs{})
if err != nil {
log.Errorf("Error while creating tar: %#v", err)
os.Exit(1)
}
}()
err := extracter(compInfo, targetPath, reader)
if err != nil {
return err
}
return nil
}
// checkFileExist check if given file exists or not
func checkFileExistWithFS(fileName string, fs filesystem.Filesystem) bool {
_, err := fs.Stat(fileName)
return !os.IsNotExist(err)
}
// makeTar function is copied from https://github.com/kubernetes/kubernetes/blob/master/pkg/kubectl/cmd/cp.go#L309
// srcPath is ignored if files is set
func makeTar(srcPath, destPath string, writer io.Writer, files []string, globExps []string, ret util.IndexerRet, fs filesystem.Filesystem) error {
// TODO: use compression here?
tarWriter := taro.NewWriter(writer)
defer tarWriter.Close()
srcPath = filepath.Clean(srcPath)
// "ToSlash" is used as all containers within OpenShift are Linux based
// and thus \opt\app-root\src would be an invalid path. Backward slashes
// are converted to forward.
destPath = filepath.ToSlash(filepath.Clean(destPath))
uniquePaths := make(map[string]bool)
klog.V(4).Infof("makeTar arguments: srcPath: %s, destPath: %s, files: %+v", srcPath, destPath, files)
if len(files) != 0 {
ignoreMatcher := gitignore.CompileIgnoreLines(globExps...)
for _, fileName := range files {
if _, ok := uniquePaths[fileName]; ok {
continue
} else {
uniquePaths[fileName] = true
}
if checkFileExistWithFS(fileName, fs) {
rel, err := filepath.Rel(srcPath, fileName)
if err != nil {
return err
}
matched := ignoreMatcher.MatchesPath(rel)
if matched {
continue
}
// Fetch path of source file relative to that of source base path so that it can be passed to recursiveTar
// which uses path relative to base path for taro header to correctly identify file location when untarred
// now that the file exists, now we need to get the absolute path
fileAbsolutePath, err := dfutil.GetAbsPath(fileName)
if err != nil {
return err
}
klog.V(4).Infof("Got abs path: %s", fileAbsolutePath)
klog.V(4).Infof("Making %s relative to %s", srcPath, fileAbsolutePath)
// We use "FromSlash" to make this OS-based (Windows uses \, Linux & macOS use /)
// we get the relative path by joining the two
destFile, err := filepath.Rel(filepath.FromSlash(srcPath), filepath.FromSlash(fileAbsolutePath))
if err != nil {
return err
}
// Now we get the source file and join it to the base directory.
srcFile := filepath.Join(filepath.Base(srcPath), destFile)
if value, ok := ret.NewFileMap[destFile]; ok && value.RemoteAttribute != "" {
destFile = value.RemoteAttribute
}
klog.V(4).Infof("makeTar srcFile: %s", srcFile)
klog.V(4).Infof("makeTar destFile: %s", destFile)
// The file could be a regular file or even a folder, so use recursiveTar which handles symlinks, regular files and folders
err = linearTar(filepath.Dir(srcPath), srcFile, filepath.Dir(destPath), destFile, tarWriter, fs)
if err != nil {
return err
}
}
}
}
return nil
}
// linearTar function is a modified version of https://github.com/kubernetes/kubernetes/blob/master/pkg/kubectl/cmd/cp.go#L319
func linearTar(srcBase, srcFile, destBase, destFile string, tw *taro.Writer, fs filesystem.Filesystem) error {
if destFile == "" {
return fmt.Errorf("linear Tar error, destFile cannot be empty")
}
klog.V(4).Infof("recursiveTar arguments: srcBase: %s, srcFile: %s, destBase: %s, destFile: %s", srcBase, srcFile, destBase, destFile)
// The destination is a LINUX container and thus we *must* use ToSlash in order
// to get the copying over done correctly..
destBase = filepath.ToSlash(destBase)
destFile = filepath.ToSlash(destFile)
klog.V(4).Infof("Corrected destinations: base: %s file: %s", destBase, destFile)
joinedPath := filepath.Join(srcBase, srcFile)
stat, err := fs.Stat(joinedPath)
if err != nil {
return err
}
if stat.IsDir() {
files, err := fs.ReadDir(joinedPath)
if err != nil {
return err
}
if len(files) == 0 {
// case empty directory
hdr, _ := taro.FileInfoHeader(stat, joinedPath)
hdr.Name = destFile
if err := tw.WriteHeader(hdr); err != nil {
return err
}
}
return nil
} else if stat.Mode()&os.ModeSymlink != 0 {
// case soft link
hdr, _ := taro.FileInfoHeader(stat, joinedPath)
target, err := os.Readlink(joinedPath)
if err != nil {
return err
}
hdr.Linkname = target
hdr.Name = destFile
if err := tw.WriteHeader(hdr); err != nil {
return err
}
} else {
// case regular file or other file type like pipe
hdr, err := taro.FileInfoHeader(stat, joinedPath)
if err != nil {
return err
}
hdr.Name = destFile
err = tw.WriteHeader(hdr)
if err != nil {
return err
}
f, err := fs.Open(joinedPath)
if err != nil {
return err
}
defer f.Close() // #nosec G307
if _, err := io.Copy(tw, f); err != nil {
return err
}
return f.Close()
}
return nil
}

369
pkg/sync/copy_test.go Normal file
View File

@@ -0,0 +1,369 @@
package sync
import (
taro "archive/tar"
"bytes"
"io"
"path"
"path/filepath"
"testing"
"github.com/redhat-developer/odo/pkg/testingutil/filesystem"
"github.com/redhat-developer/odo/pkg/util"
)
func Test_linearTar(t *testing.T) {
// FileType custom type to indicate type of file
type FileType int
const (
// RegularFile enum to represent regular file
RegularFile FileType = 0
// Directory enum to represent directory
Directory FileType = 1
)
fs := filesystem.NewFakeFs()
type args struct {
srcBase string
srcFile string
destBase string
destFile string
data string
}
tests := []struct {
name string
args args
fileType FileType
notExistError bool
wantErr bool
}{
{
name: "case 1: write a regular file",
args: args{
srcBase: filepath.Join("tmp", "dir1"),
srcFile: "red.js",
destBase: filepath.Join("tmp1", "dir2"),
destFile: "red.js",
data: "hi",
},
fileType: RegularFile,
wantErr: false,
},
{
name: "case 2: write a folder",
args: args{
srcBase: filepath.Join("tmp", "dir1"),
srcFile: "dir0",
destBase: filepath.Join("tmp1", "dir2"),
destFile: "dir2",
},
fileType: Directory,
wantErr: false,
},
{
name: "case 3: file source doesn't exist",
args: args{
srcBase: filepath.Join("tmp", "dir1"),
srcFile: "red.js",
destBase: filepath.Join("tmp1", "dir2"),
destFile: "red.js",
data: "hi",
},
fileType: RegularFile,
notExistError: true,
wantErr: true,
},
{
name: "case 4: folder source doesn't exist",
args: args{
srcBase: filepath.Join("tmp", "dir1"),
srcFile: "dir0",
destBase: filepath.Join("tmp1", "dir2"),
destFile: "dir2",
},
fileType: Directory,
notExistError: true,
wantErr: true,
},
{
name: "case 5: dest is empty",
args: args{
srcBase: filepath.Join("tmp", "dir1"),
srcFile: "dir0",
destBase: "",
destFile: "",
},
fileType: Directory,
wantErr: true,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
filepath := path.Join(tt.args.srcBase, tt.args.srcFile)
if tt.fileType == RegularFile {
f, err := fs.Create(filepath)
if err != nil {
t.Fatalf("unexpected error: %v", err)
}
if _, err := io.Copy(f, bytes.NewBuffer([]byte(tt.args.data))); err != nil {
t.Fatalf("unexpected error: %v", err)
}
defer f.Close()
} else {
if err := fs.MkdirAll(filepath, 0755); err != nil {
t.Errorf("unexpected error: %v", err)
}
}
if tt.notExistError == true {
tt.args.srcBase += "blah"
}
reader, writer := io.Pipe()
defer reader.Close()
defer writer.Close()
tarWriter := taro.NewWriter(writer)
go func() {
defer tarWriter.Close()
if err := linearTar(tt.args.srcBase, tt.args.srcFile, tt.args.destBase, tt.args.destFile, tarWriter, fs); (err != nil) != tt.wantErr {
t.Errorf("linearTar() error = %v, wantErr %v", err, tt.wantErr)
}
}()
tarReader := taro.NewReader(reader)
for {
hdr, err := tarReader.Next()
if err == io.EOF {
break
} else if err != nil {
t.Errorf("unexpected error: %v", err)
}
if hdr.Name != tt.args.destFile {
t.Errorf("expected %q as destination filename, saw: %q", tt.args.destFile, hdr.Name)
}
}
})
}
}
func Test_makeTar(t *testing.T) {
fs := filesystem.NewFakeFs()
dir0, err := fs.TempDir("", "dir0")
if err != nil {
t.Errorf("unexpected error: %v", err)
}
_, err = fs.Create(filepath.Join(dir0, "red.js"))
if err != nil {
t.Errorf("unexpected error: %v", err)
}
_, err = fs.Create(filepath.Join(dir0, "README.txt"))
if err != nil {
t.Errorf("unexpected error: %v", err)
}
err = fs.MkdirAll(filepath.Join(dir0, "views"), 0644)
if err != nil {
t.Errorf("unexpected error: %v", err)
}
_, err = fs.Create(filepath.Join(dir0, "views", "view.html"))
if err != nil {
t.Errorf("unexpected error: %v", err)
}
type args struct {
srcPath string
destPath string
files []string
globExps []string
ret util.IndexerRet
}
tests := []struct {
name string
args args
wantFiles map[string]bool
wantErr bool
}{
{
name: "case 1: normal tar making",
args: args{
srcPath: dir0,
destPath: filepath.Join("tmp", "dir1"),
files: []string{
filepath.Join(dir0, "red.js"),
filepath.Join(dir0, "README.txt"),
filepath.Join(dir0, "views"),
filepath.Join(dir0, "views", "view.html")},
globExps: []string{},
ret: util.IndexerRet{
NewFileMap: map[string]util.FileData{
"red.js": {
RemoteAttribute: "red.js",
},
"README.txt": {
RemoteAttribute: "README.txt",
},
"views": {
RemoteAttribute: "views",
},
filepath.Join("views", "view.html"): {
RemoteAttribute: "views/view.html",
},
},
},
},
wantFiles: map[string]bool{
"red.js": true,
"views/view.html": true,
"README.txt": true,
},
},
{
name: "case 2: normal tar making with matching glob expression",
args: args{
srcPath: dir0,
destPath: filepath.Join("tmp", "dir1"),
files: []string{
filepath.Join(dir0, "red.js"),
filepath.Join(dir0, "README.txt"),
filepath.Join(dir0, "views"),
filepath.Join(dir0, "views", "view.html")},
globExps: []string{"README.txt"},
ret: util.IndexerRet{
NewFileMap: map[string]util.FileData{
"red.js": {
RemoteAttribute: "red.js",
},
"README.txt": {
RemoteAttribute: "README.txt",
},
"views": {
RemoteAttribute: "views",
},
filepath.Join("views", "view.html"): {
RemoteAttribute: "views/view.html",
},
},
},
},
wantFiles: map[string]bool{
"red.js": true,
"views/view.html": true,
},
},
{
name: "case 3: normal tar making different remote than local",
args: args{
srcPath: dir0,
destPath: filepath.Join("tmp", "dir1"),
files: []string{
filepath.Join(dir0, "red.js"),
filepath.Join(dir0, "README.txt"),
filepath.Join(dir0, "views"),
filepath.Join(dir0, "views", "view.html")},
globExps: []string{},
ret: util.IndexerRet{
NewFileMap: map[string]util.FileData{
"red.js": {
RemoteAttribute: "red.js",
},
"README.txt": {
RemoteAttribute: "text/README.txt",
},
"views": {
RemoteAttribute: "views",
},
filepath.Join("views", "view.html"): {
RemoteAttribute: "views/view.html",
},
},
},
},
wantFiles: map[string]bool{
"red.js": true,
"views/view.html": true,
"text/README.txt": true,
},
},
{
name: "case 4: ignore no existent file or folder",
args: args{
srcPath: dir0,
destPath: filepath.Join("tmp", "dir1"),
files: []string{
filepath.Join(dir0, "red.js"),
filepath.Join(dir0, "README.txt"),
filepath.Join("blah", "views"),
filepath.Join(dir0, "views", "view.html")},
globExps: []string{},
ret: util.IndexerRet{
NewFileMap: map[string]util.FileData{
"red.js": {
RemoteAttribute: "red.js",
},
"README.txt": {
RemoteAttribute: "text/README.txt",
},
"views": {
RemoteAttribute: "views",
},
filepath.Join("views", "view.html"): {
RemoteAttribute: "views/view.html",
},
},
},
},
wantFiles: map[string]bool{
"red.js": true,
"views/view.html": true,
"text/README.txt": true,
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
reader, writer := io.Pipe()
defer reader.Close()
defer writer.Close()
tarWriter := taro.NewWriter(writer)
go func() {
defer tarWriter.Close()
wantErr := tt.wantErr
if err := makeTar(tt.args.srcPath, tt.args.destPath, writer, tt.args.files, tt.args.globExps, tt.args.ret, fs); (err != nil) != wantErr {
t.Errorf("makeTar() error = %v, wantErr %v", err, tt.wantErr)
return
}
}()
gotFiles := make(map[string]bool)
tarReader := taro.NewReader(reader)
for {
hdr, err := tarReader.Next()
if err == io.EOF {
break
} else if err != nil {
t.Errorf("unexpected error: %v", err)
}
if _, ok := tt.wantFiles[hdr.Name]; !ok {
t.Errorf("unexpected file name in tar, : %q", hdr.Name)
}
gotFiles[hdr.Name] = true
}
for fileName := range tt.wantFiles {
if _, ok := gotFiles[fileName]; !ok {
t.Errorf("missed file, : %q", fileName)
}
}
})
}
}

30
pkg/sync/interface.go Normal file
View File

@@ -0,0 +1,30 @@
package sync
import "io"
// ComponentInfo is a struct that holds information about a component i.e.; component name, pod name, container name, and source mount (if applicable)
type ComponentInfo struct {
ComponentName string
PodName string
ContainerName string
SyncFolder string
}
type SyncExtracter func(ComponentInfo, string, io.Reader) error
// SyncParameters is a struct containing the parameters to be used when syncing a devfile component
type SyncParameters struct {
Path string // Path refers to the parent folder containing the source code to push up to a component
WatchFiles []string // Optional: WatchFiles is the list of changed files detected by odo watch. If empty or nil, odo will check .odo/odo-file-index.json to determine changed files
WatchDeletedFiles []string // Optional: WatchDeletedFiles is the list of deleted files detected by odo watch. If empty or nil, odo will check .odo/odo-file-index.json to determine deleted files
IgnoredFiles []string // IgnoredFiles is the list of files to not push up to a component
DevfileScanIndexForWatch bool // DevfileScanIndexForWatch is true if watch's push should regenerate the index file during SyncFiles, false otherwise. See 'pkg/sync/adapter.go' for details
ForcePush bool
SyncExtracter SyncExtracter
CompInfo ComponentInfo
Files map[string]string
}
type Client interface {
SyncFiles(syncParameters SyncParameters) (bool, error)
}

View File

@@ -1,50 +0,0 @@
// Code generated by MockGen. DO NOT EDIT.
// Source: pkg/sync/sync.go
// Package mock is a generated GoMock package.
package mock
import (
io "io"
reflect "reflect"
gomock "github.com/golang/mock/gomock"
adapters "github.com/redhat-developer/odo/pkg/devfile/adapters"
)
// MockSyncClient is a mock of SyncClient interface.
type MockSyncClient struct {
ctrl *gomock.Controller
recorder *MockSyncClientMockRecorder
}
// MockSyncClientMockRecorder is the mock recorder for MockSyncClient.
type MockSyncClientMockRecorder struct {
mock *MockSyncClient
}
// NewMockSyncClient creates a new mock instance.
func NewMockSyncClient(ctrl *gomock.Controller) *MockSyncClient {
mock := &MockSyncClient{ctrl: ctrl}
mock.recorder = &MockSyncClientMockRecorder{mock}
return mock
}
// EXPECT returns an object that allows the caller to indicate expected use.
func (m *MockSyncClient) EXPECT() *MockSyncClientMockRecorder {
return m.recorder
}
// ExtractProjectToComponent mocks base method.
func (m *MockSyncClient) ExtractProjectToComponent(arg0 adapters.ComponentInfo, arg1 string, arg2 io.Reader) error {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "ExtractProjectToComponent", arg0, arg1, arg2)
ret0, _ := ret[0].(error)
return ret0
}
// ExtractProjectToComponent indicates an expected call of ExtractProjectToComponent.
func (mr *MockSyncClientMockRecorder) ExtractProjectToComponent(arg0, arg1, arg2 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ExtractProjectToComponent", reflect.TypeOf((*MockSyncClient)(nil).ExtractProjectToComponent), arg0, arg1, arg2)
}

View File

@@ -1,216 +1,293 @@
package sync
import (
taro "archive/tar"
"fmt"
"io"
"os"
"path/filepath"
"strings"
"github.com/redhat-developer/odo/pkg/devfile/adapters"
"github.com/redhat-developer/odo/pkg/log"
"github.com/redhat-developer/odo/pkg/testingutil/filesystem"
"github.com/redhat-developer/odo/pkg/util"
"github.com/devfile/library/pkg/devfile/generator"
dfutil "github.com/devfile/library/pkg/util"
gitignore "github.com/sabhiram/go-gitignore"
"github.com/redhat-developer/odo/pkg/kclient"
"github.com/redhat-developer/odo/pkg/remotecmd"
"github.com/redhat-developer/odo/pkg/util"
"k8s.io/klog"
)
type SyncClient interface {
ExtractProjectToComponent(adapters.ComponentInfo, string, io.Reader) error
// SyncClient is a Kubernetes implementationn for sync
type SyncClient struct {
kubeClient kclient.ClientInterface
}
// CopyFile copies localPath directory or list of files in copyFiles list to the directory in running Pod.
// copyFiles is list of changed files captured during `odo watch` as well as binary file path
// During copying binary components, localPath represent base directory path to binary and copyFiles contains path of binary
// During copying local source components, localPath represent base directory path whereas copyFiles is empty
// During `odo watch`, localPath represent base directory path whereas copyFiles contains list of changed Files
func CopyFile(client SyncClient, localPath string, compInfo adapters.ComponentInfo, targetPath string, copyFiles []string, globExps []string, ret util.IndexerRet) error {
var _ Client = (*SyncClient)(nil)
// Destination is set to "ToSlash" as all containers being ran within OpenShift / S2I are all
// Linux based and thus: "\opt\app-root\src" would not work correctly.
dest := filepath.ToSlash(filepath.Join(targetPath, filepath.Base(localPath)))
targetPath = filepath.ToSlash(targetPath)
// NewSyncClient instantiates a new SyncClient
func NewSyncClient(kubeClient kclient.ClientInterface) *SyncClient {
return &SyncClient{
kubeClient: kubeClient,
}
}
klog.V(4).Infof("CopyFile arguments: localPath %s, dest %s, targetPath %s, copyFiles %s, globalExps %s", localPath, dest, targetPath, copyFiles, globExps)
reader, writer := io.Pipe()
// inspired from https://github.com/kubernetes/kubernetes/blob/master/pkg/kubectl/cmd/cp.go#L235
go func() {
defer writer.Close()
// SyncFiles does a couple of things:
// if files changed/deleted are passed in from watch, it syncs them to the component
// otherwise, it checks which files have changed and syncs the delta
// it returns a boolean execRequired and an error. execRequired tells us if files have
// changed and devfile execution is required
func (a SyncClient) SyncFiles(syncParameters SyncParameters) (bool, error) {
// Whether to write the indexer content to the index file path (resolvePath)
forceWrite := false
// Ret from Indexer function
var ret util.IndexerRet
var deletedFiles []string
var changedFiles []string
isWatch := len(syncParameters.WatchFiles) > 0 || len(syncParameters.WatchDeletedFiles) > 0
// When this function is invoked by watch, the logic is:
// 1) If this is the first time that watch has called Push (in this OS process), then generate the file index
// using the file indexer, and use that to sync files (eg don't use changed/deleted files list from watch at
// this stage; these will be found by the indexer run).
// - In the watch scenario, we need to first run the indexer for two reasons:
// - In cases where the index doesn't initially exist, we need to create it (so we can ADD to it in
// later calls to SyncFiles(...) )
// - Even if it does initially exist, there is no guarantee that the remote pod is consistent with it; so
// on our first invocation we need to compare the index with the remote pod (by regenerating the index
// and using the changes files list from that to sync the results.)
//
// 2) For every other push/sync call after the first, don't run the file indexer, instead we use
// the watch events to determine what changed, and ensure that the index is then updated based
// on the watch events (to ensure future calls are correct)
// True if the index was updated based on the deleted/changed files values from the watch (and
// thus the indexer doesn't need to run), false otherwise
indexRegeneratedByWatch := false
// If watch files are specified _and_ this is not the first call (by this process) to SyncFiles by the watch command, then insert the
// changed files into the existing file index, and delete removed files from the index
if isWatch && !syncParameters.DevfileScanIndexForWatch {
err := updateIndexWithWatchChanges(syncParameters)
err := makeTar(localPath, dest, writer, copyFiles, globExps, ret, filesystem.DefaultFs{})
if err != nil {
log.Errorf("Error while creating tar: %#v", err)
os.Exit(1)
return false, err
}
}()
changedFiles = syncParameters.WatchFiles
deletedFiles = syncParameters.WatchDeletedFiles
deletedFiles, err = dfutil.RemoveRelativePathFromFiles(deletedFiles, syncParameters.Path)
if err != nil {
return false, fmt.Errorf("unable to remove relative path from list of changed/deleted files: %w", err)
}
indexRegeneratedByWatch = true
err := client.ExtractProjectToComponent(compInfo, targetPath, reader)
}
if !indexRegeneratedByWatch {
// Calculate the files to sync
// Tries to sync the deltas unless it is a forced push
// if it is a forced push (ForcePush) reset the index to do a full sync
absIgnoreRules := dfutil.GetAbsGlobExps(syncParameters.Path, syncParameters.IgnoredFiles)
// Before running the indexer, make sure the .odo folder exists (or else the index file will not get created)
odoFolder := filepath.Join(syncParameters.Path, ".odo")
if _, err := os.Stat(odoFolder); os.IsNotExist(err) {
err = os.Mkdir(odoFolder, 0750)
if err != nil {
return false, fmt.Errorf("unable to create directory: %w", err)
}
}
// If the pod changed, reset the index, which will cause the indexer to walk the directory
// tree and resync all local files.
// If it is a new component, reset index to make sure any previously existing file is cleaned up
if syncParameters.ForcePush {
err := util.DeleteIndexFile(syncParameters.Path)
if err != nil {
return false, fmt.Errorf("unable to reset the index file: %w", err)
}
}
// Run the indexer and find the modified/added/deleted/renamed files
var err error
ret, err = util.RunIndexerWithRemote(syncParameters.Path, syncParameters.IgnoredFiles, syncParameters.Files)
if err != nil {
return false, fmt.Errorf("unable to run indexer: %w", err)
}
if len(ret.FilesChanged) > 0 || len(ret.FilesDeleted) > 0 {
forceWrite = true
}
// apply the glob rules from the .gitignore/.odoignore file
// and ignore the files on which the rules apply and filter them out
filesChangedFiltered, filesDeletedFiltered := dfutil.FilterIgnores(ret.FilesChanged, ret.FilesDeleted, absIgnoreRules)
deletedFiles = append(filesDeletedFiltered, ret.RemoteDeleted...)
deletedFiles = append(deletedFiles, ret.RemoteDeleted...)
klog.V(4).Infof("List of files to be deleted: +%v", deletedFiles)
changedFiles = filesChangedFiltered
klog.V(4).Infof("List of files changed: +%v", changedFiles)
if len(filesChangedFiltered) == 0 && len(filesDeletedFiltered) == 0 && !syncParameters.ForcePush {
return false, nil
}
if syncParameters.ForcePush {
deletedFiles = append(deletedFiles, "*")
}
}
err := a.pushLocal(syncParameters.Path,
changedFiles,
deletedFiles,
syncParameters.ForcePush,
syncParameters.IgnoredFiles,
syncParameters.CompInfo,
syncParameters.SyncExtracter,
ret,
)
if err != nil {
return err
return false, fmt.Errorf("failed to sync to component with name %s: %w", syncParameters.CompInfo.ComponentName, err)
}
return nil
}
// checkFileExist check if given file exists or not
func checkFileExistWithFS(fileName string, fs filesystem.Filesystem) bool {
_, err := fs.Stat(fileName)
return !os.IsNotExist(err)
}
// makeTar function is copied from https://github.com/kubernetes/kubernetes/blob/master/pkg/kubectl/cmd/cp.go#L309
// srcPath is ignored if files is set
func makeTar(srcPath, destPath string, writer io.Writer, files []string, globExps []string, ret util.IndexerRet, fs filesystem.Filesystem) error {
// TODO: use compression here?
tarWriter := taro.NewWriter(writer)
defer tarWriter.Close()
srcPath = filepath.Clean(srcPath)
// "ToSlash" is used as all containers within OpenShift are Linux based
// and thus \opt\app-root\src would be an invalid path. Backward slashes
// are converted to forward.
destPath = filepath.ToSlash(filepath.Clean(destPath))
uniquePaths := make(map[string]bool)
klog.V(4).Infof("makeTar arguments: srcPath: %s, destPath: %s, files: %+v", srcPath, destPath, files)
if len(files) != 0 {
ignoreMatcher := gitignore.CompileIgnoreLines(globExps...)
for _, fileName := range files {
if _, ok := uniquePaths[fileName]; ok {
continue
} else {
uniquePaths[fileName] = true
}
if checkFileExistWithFS(fileName, fs) {
rel, err := filepath.Rel(srcPath, fileName)
if err != nil {
return err
}
matched := ignoreMatcher.MatchesPath(rel)
if matched {
continue
}
// Fetch path of source file relative to that of source base path so that it can be passed to recursiveTar
// which uses path relative to base path for taro header to correctly identify file location when untarred
// now that the file exists, now we need to get the absolute path
fileAbsolutePath, err := dfutil.GetAbsPath(fileName)
if err != nil {
return err
}
klog.V(4).Infof("Got abs path: %s", fileAbsolutePath)
klog.V(4).Infof("Making %s relative to %s", srcPath, fileAbsolutePath)
// We use "FromSlash" to make this OS-based (Windows uses \, Linux & macOS use /)
// we get the relative path by joining the two
destFile, err := filepath.Rel(filepath.FromSlash(srcPath), filepath.FromSlash(fileAbsolutePath))
if err != nil {
return err
}
// Now we get the source file and join it to the base directory.
srcFile := filepath.Join(filepath.Base(srcPath), destFile)
if value, ok := ret.NewFileMap[destFile]; ok && value.RemoteAttribute != "" {
destFile = value.RemoteAttribute
}
klog.V(4).Infof("makeTar srcFile: %s", srcFile)
klog.V(4).Infof("makeTar destFile: %s", destFile)
// The file could be a regular file or even a folder, so use recursiveTar which handles symlinks, regular files and folders
err = linearTar(filepath.Dir(srcPath), srcFile, filepath.Dir(destPath), destFile, tarWriter, fs)
if err != nil {
return err
}
}
if forceWrite {
err = util.WriteFile(ret.NewFileMap, ret.ResolvedPath)
if err != nil {
return false, fmt.Errorf("failed to write file: %w", err)
}
}
return nil
return true, nil
}
// linearTar function is a modified version of https://github.com/kubernetes/kubernetes/blob/master/pkg/kubectl/cmd/cp.go#L319
func linearTar(srcBase, srcFile, destBase, destFile string, tw *taro.Writer, fs filesystem.Filesystem) error {
if destFile == "" {
return fmt.Errorf("linear Tar error, destFile cannot be empty")
}
// pushLocal syncs source code from the user's disk to the component
func (a SyncClient) pushLocal(path string, files []string, delFiles []string, isForcePush bool, globExps []string, compInfo ComponentInfo, extracter SyncExtracter, ret util.IndexerRet) error {
klog.V(4).Infof("Push: componentName: %s, path: %s, files: %s, delFiles: %s, isForcePush: %+v", compInfo.ComponentName, path, files, delFiles, isForcePush)
klog.V(4).Infof("recursiveTar arguments: srcBase: %s, srcFile: %s, destBase: %s, destFile: %s", srcBase, srcFile, destBase, destFile)
// The destination is a LINUX container and thus we *must* use ToSlash in order
// to get the copying over done correctly..
destBase = filepath.ToSlash(destBase)
destFile = filepath.ToSlash(destFile)
klog.V(4).Infof("Corrected destinations: base: %s file: %s", destBase, destFile)
joinedPath := filepath.Join(srcBase, srcFile)
stat, err := fs.Stat(joinedPath)
// Edge case: check to see that the path is NOT empty.
emptyDir, err := dfutil.IsEmpty(path)
if err != nil {
return err
return fmt.Errorf("unable to check directory: %s: %w", path, err)
} else if emptyDir {
return fmt.Errorf("directory/file %s is empty", path)
}
if stat.IsDir() {
files, err := fs.ReadDir(joinedPath)
// Sync the files to the pod
syncFolder := compInfo.SyncFolder
if syncFolder != generator.DevfileSourceVolumeMount {
// Need to make sure the folder already exists on the component or else sync will fail
klog.V(4).Infof("Creating %s on the remote container if it doesn't already exist", syncFolder)
cmdArr := getCmdToCreateSyncFolder(syncFolder)
_, _, err = remotecmd.ExecuteCommand(cmdArr, a.kubeClient, compInfo.PodName, compInfo.ContainerName, false, nil, nil)
if err != nil {
return err
}
if len(files) == 0 {
// case empty directory
hdr, _ := taro.FileInfoHeader(stat, joinedPath)
hdr.Name = destFile
if err := tw.WriteHeader(hdr); err != nil {
return err
}
}
return nil
} else if stat.Mode()&os.ModeSymlink != 0 {
// case soft link
hdr, _ := taro.FileInfoHeader(stat, joinedPath)
target, err := os.Readlink(joinedPath)
}
// If there were any files deleted locally, delete them remotely too.
if len(delFiles) > 0 {
cmdArr := getCmdToDeleteFiles(delFiles, syncFolder)
_, _, err = remotecmd.ExecuteCommand(cmdArr, a.kubeClient, compInfo.PodName, compInfo.ContainerName, false, nil, nil)
if err != nil {
return err
}
}
hdr.Linkname = target
hdr.Name = destFile
if err := tw.WriteHeader(hdr); err != nil {
return err
if !isForcePush {
if len(files) == 0 && len(delFiles) == 0 {
return nil
}
} else {
// case regular file or other file type like pipe
hdr, err := taro.FileInfoHeader(stat, joinedPath)
}
if isForcePush || len(files) > 0 {
klog.V(4).Infof("Copying files %s to pod", strings.Join(files, " "))
err = CopyFile(extracter, path, compInfo, syncFolder, files, globExps, ret)
if err != nil {
return err
return fmt.Errorf("unable push files to pod: %w", err)
}
hdr.Name = destFile
err = tw.WriteHeader(hdr)
if err != nil {
return err
}
f, err := fs.Open(joinedPath)
if err != nil {
return err
}
defer f.Close() // #nosec G307
if _, err := io.Copy(tw, f); err != nil {
return err
}
return f.Close()
}
return nil
}
// updateIndexWithWatchChanges uses the pushParameters.WatchDeletedFiles and pushParamters.WatchFiles to update
// the existing index file; the index file is required to exist when this function is called.
func updateIndexWithWatchChanges(syncParameters SyncParameters) error {
indexFilePath, err := util.ResolveIndexFilePath(syncParameters.Path)
if err != nil {
return fmt.Errorf("unable to resolve path: %s: %w", syncParameters.Path, err)
}
// Check that the path exists
_, err = os.Stat(indexFilePath)
if err != nil {
// This shouldn't happen: in the watch case, SyncFiles should first be called with 'DevfileScanIndexForWatch' set to true, which
// will generate the index. Then, all subsequent invocations of SyncFiles will run with 'DevfileScanIndexForWatch' set to false,
// which will not regenerate the index (merely updating it based on changed files.)
//
// If you see this error it means somehow watch's SyncFiles was called without the index being first generated (likely because the
// above mentioned pushParam wasn't set). See SyncFiles(...) for details.
return fmt.Errorf("resolved path doesn't exist: %s: %w", indexFilePath, err)
}
// Parse the existing index
fileIndex, err := util.ReadFileIndex(indexFilePath)
if err != nil {
return fmt.Errorf("unable to read index from path: %s: %w", indexFilePath, err)
}
rootDir := syncParameters.Path
// Remove deleted files from the existing index
for _, deletedFile := range syncParameters.WatchDeletedFiles {
relativePath, err := util.CalculateFileDataKeyFromPath(deletedFile, rootDir)
if err != nil {
klog.V(4).Infof("Error occurred for %s: %v", deletedFile, err)
continue
}
delete(fileIndex.Files, relativePath)
klog.V(4).Infof("Removing watch deleted file from index: %s", relativePath)
}
// Add changed files to the existing index
for _, addedOrModifiedFile := range syncParameters.WatchFiles {
relativePath, fileData, err := util.GenerateNewFileDataEntry(addedOrModifiedFile, rootDir)
if err != nil {
klog.V(4).Infof("Error occurred for %s: %v", addedOrModifiedFile, err)
continue
}
fileIndex.Files[relativePath] = *fileData
klog.V(4).Infof("Added/updated watched file in index: %s", relativePath)
}
// Write the result
return util.WriteFile(fileIndex.Files, indexFilePath)
}
// getCmdToCreateSyncFolder returns the command used to create the remote sync folder on the running container
func getCmdToCreateSyncFolder(syncFolder string) []string {
return []string{"mkdir", "-p", syncFolder}
}
// getCmdToDeleteFiles returns the command used to delete the remote files on the container that are marked for deletion
func getCmdToDeleteFiles(delFiles []string, syncFolder string) []string {
rmPaths := dfutil.GetRemoteFilesMarkedForDeletion(delFiles, syncFolder)
klog.V(4).Infof("remote files marked for deletion are %+v", rmPaths)
cmdArr := []string{"rm", "-rf"}
for _, remote := range rmPaths {
cmdArr = append(cmdArr, filepath.ToSlash(remote))
}
return cmdArr
}

View File

@@ -1,367 +1,448 @@
package sync
import (
taro "archive/tar"
"bytes"
"io"
"errors"
io "io"
"io/ioutil"
"os"
"path"
"path/filepath"
"reflect"
"testing"
"github.com/redhat-developer/odo/pkg/testingutil/filesystem"
"github.com/devfile/library/pkg/devfile/generator"
"github.com/golang/mock/gomock"
"github.com/redhat-developer/odo/pkg/kclient"
"github.com/redhat-developer/odo/pkg/util"
"github.com/redhat-developer/odo/tests/helper"
)
func Test_linearTar(t *testing.T) {
// FileType custom type to indicate type of file
type FileType int
const (
// RegularFile enum to represent regular file
RegularFile FileType = 0
// Directory enum to represent directory
Directory FileType = 1
)
fs := filesystem.NewFakeFs()
type args struct {
srcBase string
srcFile string
destBase string
destFile string
data string
}
func TestGetCmdToCreateSyncFolder(t *testing.T) {
tests := []struct {
name string
args args
fileType FileType
notExistError bool
wantErr bool
name string
syncFolder string
want []string
}{
{
name: "case 1: write a regular file",
args: args{
srcBase: filepath.Join("tmp", "dir1"),
srcFile: "red.js",
destBase: filepath.Join("tmp1", "dir2"),
destFile: "red.js",
data: "hi",
},
fileType: RegularFile,
wantErr: false,
name: "Case 1: Sync to /projects",
syncFolder: generator.DevfileSourceVolumeMount,
want: []string{"mkdir", "-p", generator.DevfileSourceVolumeMount},
},
{
name: "case 2: write a folder",
args: args{
srcBase: filepath.Join("tmp", "dir1"),
srcFile: "dir0",
destBase: filepath.Join("tmp1", "dir2"),
destFile: "dir2",
},
fileType: Directory,
wantErr: false,
name: "Case 2: Sync subdir of /projects",
syncFolder: generator.DevfileSourceVolumeMount + "/someproject",
want: []string{"mkdir", "-p", generator.DevfileSourceVolumeMount + "/someproject"},
},
}
for _, tt := range tests {
cmdArr := getCmdToCreateSyncFolder(tt.syncFolder)
if !reflect.DeepEqual(tt.want, cmdArr) {
t.Errorf("Expected %s, got %s", tt.want, cmdArr)
}
}
}
func TestGetCmdToDeleteFiles(t *testing.T) {
syncFolder := "/projects/hello-world"
tests := []struct {
name string
delFiles []string
syncFolder string
want []string
}{
{
name: "Case 1: One deleted file",
delFiles: []string{"test.txt"},
syncFolder: generator.DevfileSourceVolumeMount,
want: []string{"rm", "-rf", generator.DevfileSourceVolumeMount + "/test.txt"},
},
{
name: "case 3: file source doesn't exist",
args: args{
srcBase: filepath.Join("tmp", "dir1"),
srcFile: "red.js",
destBase: filepath.Join("tmp1", "dir2"),
destFile: "red.js",
data: "hi",
},
fileType: RegularFile,
notExistError: true,
wantErr: true,
name: "Case 2: Multiple deleted files, default sync folder",
delFiles: []string{"test.txt", "hello.c"},
syncFolder: generator.DevfileSourceVolumeMount,
want: []string{"rm", "-rf", generator.DevfileSourceVolumeMount + "/test.txt", generator.DevfileSourceVolumeMount + "/hello.c"},
},
{
name: "case 4: folder source doesn't exist",
args: args{
srcBase: filepath.Join("tmp", "dir1"),
srcFile: "dir0",
destBase: filepath.Join("tmp1", "dir2"),
destFile: "dir2",
name: "Case 2: Multiple deleted files, different sync folder",
delFiles: []string{"test.txt", "hello.c"},
syncFolder: syncFolder,
want: []string{"rm", "-rf", syncFolder + "/test.txt", syncFolder + "/hello.c"},
},
}
for _, tt := range tests {
cmdArr := getCmdToDeleteFiles(tt.delFiles, tt.syncFolder)
if !reflect.DeepEqual(tt.want, cmdArr) {
t.Errorf("Expected %s, got %s", tt.want, cmdArr)
}
}
}
func TestSyncFiles(t *testing.T) {
testComponentName := "test"
// create a temp dir for the file indexer
directory, e := ioutil.TempDir("", "")
if e != nil {
t.Errorf("TestSyncFiles error: error creating temporary directory for the indexer: %v", e)
}
jsFile, e := os.Create(filepath.Join(directory, "red.js"))
if e != nil {
t.Errorf("TestSyncFiles error: error creating temporary file for the indexer: %v", e)
}
ctrl := gomock.NewController(t)
kc := kclient.NewMockClientInterface(ctrl)
kc.EXPECT().ExecCMDInContainer(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).
Return(nil).AnyTimes()
// Assert that Bar() is invoked.
defer ctrl.Finish()
syncClient := func(ComponentInfo, string, io.Reader) error {
return nil
}
tests := []struct {
name string
syncParameters SyncParameters
wantErr bool
wantIsPushRequired bool
}{
{
name: "Case 1: Component does not exist",
syncParameters: SyncParameters{
Path: directory,
WatchFiles: []string{},
WatchDeletedFiles: []string{},
IgnoredFiles: []string{},
CompInfo: ComponentInfo{
ContainerName: "abcd",
},
ForcePush: true,
SyncExtracter: syncClient,
},
fileType: Directory,
notExistError: true,
wantErr: true,
wantErr: false,
wantIsPushRequired: true,
},
{
name: "case 5: dest is empty",
args: args{
srcBase: filepath.Join("tmp", "dir1"),
srcFile: "dir0",
destBase: "",
destFile: "",
name: "Case 2: Component does exist",
syncParameters: SyncParameters{
Path: directory,
WatchFiles: []string{},
WatchDeletedFiles: []string{},
IgnoredFiles: []string{},
CompInfo: ComponentInfo{
ContainerName: "abcd",
},
ForcePush: false,
SyncExtracter: syncClient,
},
fileType: Directory,
wantErr: true,
wantErr: false,
wantIsPushRequired: false, // always false after case 1
},
{
name: "Case 3: FakeErrorClient error",
syncParameters: SyncParameters{
Path: directory,
WatchFiles: []string{},
WatchDeletedFiles: []string{},
IgnoredFiles: []string{},
CompInfo: ComponentInfo{
ContainerName: "abcd",
},
ForcePush: false,
SyncExtracter: syncClient,
},
wantErr: true,
wantIsPushRequired: false,
},
{
name: "Case 4: File change",
syncParameters: SyncParameters{
Path: directory,
WatchFiles: []string{path.Join(directory, "test.log")},
WatchDeletedFiles: []string{},
IgnoredFiles: []string{},
CompInfo: ComponentInfo{
ComponentName: testComponentName,
ContainerName: "abcd",
},
ForcePush: false,
SyncExtracter: syncClient,
},
wantErr: false,
wantIsPushRequired: true,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
filepath := path.Join(tt.args.srcBase, tt.args.srcFile)
syncAdapter := NewSyncClient(kc)
isPushRequired, err := syncAdapter.SyncFiles(tt.syncParameters)
if !tt.wantErr && err != nil {
t.Errorf("TestSyncFiles error: unexpected error when syncing files %v", err)
} else if !tt.wantErr && isPushRequired != tt.wantIsPushRequired {
t.Errorf("TestSyncFiles error: isPushRequired mismatch, wanted: %v, got: %v", tt.wantIsPushRequired, isPushRequired)
}
})
}
if tt.fileType == RegularFile {
f, err := fs.Create(filepath)
err := jsFile.Close()
if err != nil {
t.Errorf("TestSyncFiles error: error deleting the temp dir %s, err: %v", directory, err)
}
// Remove the temp dir created for the file indexer
err = os.RemoveAll(directory)
if err != nil {
t.Errorf("TestSyncFiles error: error deleting the temp dir %s, err: %v", directory, err)
}
}
func TestPushLocal(t *testing.T) {
testComponentName := "test"
// create a temp dir for the file indexer
directory, e := ioutil.TempDir("", "")
if e != nil {
t.Errorf("TestPushLocal error: error creating temporary directory for the indexer: %v", e)
}
newFilePath := filepath.Join(directory, "foobar.txt")
if err := helper.CreateFileWithContent(newFilePath, "hello world"); err != nil {
t.Errorf("TestPushLocal error: the foobar.txt file was not created: %v", err)
}
ctrl := gomock.NewController(t)
kc := kclient.NewMockClientInterface(ctrl)
kc.EXPECT().ExecCMDInContainer(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).
Return(nil).AnyTimes()
// Assert that Bar() is invoked.
defer ctrl.Finish()
syncClient := func(ComponentInfo, string, io.Reader) error {
return nil
}
errorSyncClient := func(ComponentInfo, string, io.Reader) error {
return errors.New("err")
}
tests := []struct {
name string
client SyncExtracter
path string
files []string
delFiles []string
isForcePush bool
compInfo ComponentInfo
wantErr bool
}{
{
name: "Case 1: File change",
client: syncClient,
path: directory,
files: []string{path.Join(directory, "test.log")},
delFiles: []string{},
isForcePush: false,
compInfo: ComponentInfo{
ContainerName: "abcd",
},
wantErr: false,
},
{
name: "Case 2: File change with fake error client",
client: errorSyncClient,
path: directory,
files: []string{path.Join(directory, "test.log")},
delFiles: []string{},
isForcePush: false,
compInfo: ComponentInfo{
ContainerName: "abcd",
},
wantErr: true,
},
{
name: "Case 3: No file change",
client: syncClient,
path: directory,
files: []string{},
delFiles: []string{},
isForcePush: false,
compInfo: ComponentInfo{
ContainerName: "abcd",
},
wantErr: false,
},
{
name: "Case 4: Deleted file",
client: syncClient,
path: directory,
files: []string{},
delFiles: []string{path.Join(directory, "test.log")},
isForcePush: false,
compInfo: ComponentInfo{
ContainerName: "abcd",
},
wantErr: false,
},
{
name: "Case 5: Force push",
client: syncClient,
path: directory,
files: []string{},
delFiles: []string{},
isForcePush: true,
compInfo: ComponentInfo{
ContainerName: "abcd",
},
wantErr: false,
},
{
name: "Case 6: Source mapping folder set",
client: syncClient,
path: directory,
files: []string{},
delFiles: []string{},
isForcePush: false,
compInfo: ComponentInfo{
ComponentName: testComponentName,
ContainerName: "abcd",
SyncFolder: "/some/path",
},
wantErr: false,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
syncAdapter := NewSyncClient(kc)
err := syncAdapter.pushLocal(tt.path, tt.files, tt.delFiles, tt.isForcePush, []string{}, tt.compInfo, syncClient, util.IndexerRet{})
if !tt.wantErr && err != nil {
t.Errorf("TestPushLocal error: error pushing files: %v", err)
}
})
}
// Remove the temp dir created for the file indexer
err := os.RemoveAll(directory)
if err != nil {
t.Errorf("TestPushLocal error: error deleting the temp dir %s", directory)
}
}
func TestUpdateIndexWithWatchChanges(t *testing.T) {
tests := []struct {
name string
initialFilesToCreate []string
watchDeletedFiles []string
watchAddedFiles []string
expectedFilesInIndex []string
}{
{
name: "Case 1 - Watch file deleted should remove file from index",
initialFilesToCreate: []string{"file1", "file2"},
watchDeletedFiles: []string{"file1"},
expectedFilesInIndex: []string{"file2"},
},
{
name: "Case 2 - Watch file added should add file to index",
initialFilesToCreate: []string{"file1"},
watchAddedFiles: []string{"file2"},
expectedFilesInIndex: []string{"file1", "file2"},
},
{
name: "Case 3 - No watch changes should mean no index changes",
initialFilesToCreate: []string{"file1"},
expectedFilesInIndex: []string{"file1"},
},
}
for _, tt := range tests {
// create a temp dir for the fake component
directory, err := ioutil.TempDir("", "")
if err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: error creating temporary directory for the indexer: %v", err)
}
fileIndexPath, err := util.ResolveIndexFilePath(directory)
if err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to resolve index file path: %v", err)
}
if err := os.MkdirAll(filepath.Dir(fileIndexPath), 0750); err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to create directories for %s: %v", fileIndexPath, err)
}
t.Run(tt.name, func(t *testing.T) {
indexData := map[string]util.FileData{}
// Create initial files
for _, fileToCreate := range tt.initialFilesToCreate {
filePath := filepath.Join(directory, fileToCreate)
if err := ioutil.WriteFile(filePath, []byte("non-empty-string"), 0644); err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to write to index file path: %v", err)
}
key, fileDatum, err := util.GenerateNewFileDataEntry(filePath, directory)
if err != nil {
t.Fatalf("unexpected error: %v", err)
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to generate new file: %v", err)
}
if _, err := io.Copy(f, bytes.NewBuffer([]byte(tt.args.data))); err != nil {
t.Fatalf("unexpected error: %v", err)
}
defer f.Close()
} else {
if err := fs.MkdirAll(filepath, 0755); err != nil {
t.Errorf("unexpected error: %v", err)
indexData[key] = *fileDatum
}
// Write the index based on those files
if err := util.WriteFile(indexData, fileIndexPath); err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to write index file: %v", err)
}
syncParams := SyncParameters{
Path: directory,
}
// Add deleted files to pushParams (also delete the files)
for _, deletedFile := range tt.watchDeletedFiles {
deletedFilePath := filepath.Join(directory, deletedFile)
syncParams.WatchDeletedFiles = append(syncParams.WatchDeletedFiles, deletedFilePath)
if err := os.Remove(deletedFilePath); err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to delete file %s %v", deletedFilePath, err)
}
}
if tt.notExistError == true {
tt.args.srcBase += "blah"
// Add added files to pushParams (also create the files)
for _, addedFile := range tt.watchAddedFiles {
addedFilePath := filepath.Join(directory, addedFile)
syncParams.WatchFiles = append(syncParams.WatchFiles, addedFilePath)
if err := ioutil.WriteFile(addedFilePath, []byte("non-empty-string"), 0644); err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: unable to write to index file path: %v", err)
}
}
reader, writer := io.Pipe()
defer reader.Close()
defer writer.Close()
if err := updateIndexWithWatchChanges(syncParams); err != nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal: unexpected error: %v", err)
}
tarWriter := taro.NewWriter(writer)
postFileIndex, err := util.ReadFileIndex(fileIndexPath)
if err != nil || postFileIndex == nil {
t.Fatalf("TestUpdateIndexWithWatchChangesLocal error: read new file index: %v", err)
}
go func() {
defer tarWriter.Close()
if err := linearTar(tt.args.srcBase, tt.args.srcFile, tt.args.destBase, tt.args.destFile, tarWriter, fs); (err != nil) != tt.wantErr {
t.Errorf("linearTar() error = %v, wantErr %v", err, tt.wantErr)
}
}()
tarReader := taro.NewReader(reader)
for {
hdr, err := tarReader.Next()
if err == io.EOF {
break
} else if err != nil {
t.Errorf("unexpected error: %v", err)
}
if hdr.Name != tt.args.destFile {
t.Errorf("expected %q as destination filename, saw: %q", tt.args.destFile, hdr.Name)
}
}
})
}
}
func Test_makeTar(t *testing.T) {
fs := filesystem.NewFakeFs()
dir0, err := fs.TempDir("", "dir0")
if err != nil {
t.Errorf("unexpected error: %v", err)
}
_, err = fs.Create(filepath.Join(dir0, "red.js"))
if err != nil {
t.Errorf("unexpected error: %v", err)
}
_, err = fs.Create(filepath.Join(dir0, "README.txt"))
if err != nil {
t.Errorf("unexpected error: %v", err)
}
err = fs.MkdirAll(filepath.Join(dir0, "views"), 0644)
if err != nil {
t.Errorf("unexpected error: %v", err)
}
_, err = fs.Create(filepath.Join(dir0, "views", "view.html"))
if err != nil {
t.Errorf("unexpected error: %v", err)
}
type args struct {
srcPath string
destPath string
files []string
globExps []string
ret util.IndexerRet
}
tests := []struct {
name string
args args
wantFiles map[string]bool
wantErr bool
}{
{
name: "case 1: normal tar making",
args: args{
srcPath: dir0,
destPath: filepath.Join("tmp", "dir1"),
files: []string{
filepath.Join(dir0, "red.js"),
filepath.Join(dir0, "README.txt"),
filepath.Join(dir0, "views"),
filepath.Join(dir0, "views", "view.html")},
globExps: []string{},
ret: util.IndexerRet{
NewFileMap: map[string]util.FileData{
"red.js": {
RemoteAttribute: "red.js",
},
"README.txt": {
RemoteAttribute: "README.txt",
},
"views": {
RemoteAttribute: "views",
},
filepath.Join("views", "view.html"): {
RemoteAttribute: "views/view.html",
},
},
},
},
wantFiles: map[string]bool{
"red.js": true,
"views/view.html": true,
"README.txt": true,
},
},
{
name: "case 2: normal tar making with matching glob expression",
args: args{
srcPath: dir0,
destPath: filepath.Join("tmp", "dir1"),
files: []string{
filepath.Join(dir0, "red.js"),
filepath.Join(dir0, "README.txt"),
filepath.Join(dir0, "views"),
filepath.Join(dir0, "views", "view.html")},
globExps: []string{"README.txt"},
ret: util.IndexerRet{
NewFileMap: map[string]util.FileData{
"red.js": {
RemoteAttribute: "red.js",
},
"README.txt": {
RemoteAttribute: "README.txt",
},
"views": {
RemoteAttribute: "views",
},
filepath.Join("views", "view.html"): {
RemoteAttribute: "views/view.html",
},
},
},
},
wantFiles: map[string]bool{
"red.js": true,
"views/view.html": true,
},
},
{
name: "case 3: normal tar making different remote than local",
args: args{
srcPath: dir0,
destPath: filepath.Join("tmp", "dir1"),
files: []string{
filepath.Join(dir0, "red.js"),
filepath.Join(dir0, "README.txt"),
filepath.Join(dir0, "views"),
filepath.Join(dir0, "views", "view.html")},
globExps: []string{},
ret: util.IndexerRet{
NewFileMap: map[string]util.FileData{
"red.js": {
RemoteAttribute: "red.js",
},
"README.txt": {
RemoteAttribute: "text/README.txt",
},
"views": {
RemoteAttribute: "views",
},
filepath.Join("views", "view.html"): {
RemoteAttribute: "views/view.html",
},
},
},
},
wantFiles: map[string]bool{
"red.js": true,
"views/view.html": true,
"text/README.txt": true,
},
},
{
name: "case 4: ignore no existent file or folder",
args: args{
srcPath: dir0,
destPath: filepath.Join("tmp", "dir1"),
files: []string{
filepath.Join(dir0, "red.js"),
filepath.Join(dir0, "README.txt"),
filepath.Join("blah", "views"),
filepath.Join(dir0, "views", "view.html")},
globExps: []string{},
ret: util.IndexerRet{
NewFileMap: map[string]util.FileData{
"red.js": {
RemoteAttribute: "red.js",
},
"README.txt": {
RemoteAttribute: "text/README.txt",
},
"views": {
RemoteAttribute: "views",
},
filepath.Join("views", "view.html"): {
RemoteAttribute: "views/view.html",
},
},
},
},
wantFiles: map[string]bool{
"red.js": true,
"views/view.html": true,
"text/README.txt": true,
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
reader, writer := io.Pipe()
defer reader.Close()
defer writer.Close()
tarWriter := taro.NewWriter(writer)
go func() {
defer tarWriter.Close()
wantErr := tt.wantErr
if err := makeTar(tt.args.srcPath, tt.args.destPath, writer, tt.args.files, tt.args.globExps, tt.args.ret, fs); (err != nil) != wantErr {
t.Errorf("makeTar() error = %v, wantErr %v", err, tt.wantErr)
return
}
}()
gotFiles := make(map[string]bool)
tarReader := taro.NewReader(reader)
for {
hdr, err := tarReader.Next()
if err == io.EOF {
break
} else if err != nil {
t.Errorf("unexpected error: %v", err)
}
if _, ok := tt.wantFiles[hdr.Name]; !ok {
t.Errorf("unexpected file name in tar, : %q", hdr.Name)
}
gotFiles[hdr.Name] = true
}
for fileName := range tt.wantFiles {
if _, ok := gotFiles[fileName]; !ok {
t.Errorf("missed file, : %q", fileName)
// Locate expected files
if len(postFileIndex.Files) != len(tt.expectedFilesInIndex) {
t.Fatalf("Mismatch between number expected files and actual files in index, post-index: %v expected: %v", postFileIndex.Files, tt.expectedFilesInIndex)
}
for _, expectedFile := range tt.expectedFilesInIndex {
if _, exists := postFileIndex.Files[expectedFile]; !exists {
t.Fatalf("Unable to find '%s' in post index file, %v", expectedFile, postFileIndex.Files)
}
}
})

View File

@@ -445,7 +445,6 @@ func (o *WatchClient) processEvents(
WatchFiles: changedFiles,
WatchDeletedFiles: deletedPaths,
IgnoredFiles: parameters.FileIgnores,
ForceBuild: false,
DevfileBuildCmd: parameters.DevfileBuildCmd,
DevfileRunCmd: parameters.DevfileRunCmd,
DevfileDebugCmd: parameters.DevfileDebugCmd,

View File

@@ -85,5 +85,5 @@ mockgen -source=pkg/binding/backend/interface.go \
mockgen -source=pkg/sync/sync.go \
-package mock \
-destination pkg/sync/mock/sync.go
-package sync \
-destination pkg/sync/mock.go