Various fixes for the operator

1. Fixed SparkApplication controller to ignore reconcilation caused by SparkApplication not found errors.
2. Fixed the PySpark example.
3. Fixed code for updating the status of a SparkApplication in updateAndExportMetrics.
4. Logging refactoring.
This commit is contained in:
Yinan Li 2018-12-05 14:37:25 -08:00
parent 488cd34b7e
commit 5ff9bd43df
7 changed files with 106 additions and 99 deletions

View File

@ -19,7 +19,7 @@
apiVersion: "sparkoperator.k8s.io/v1alpha1"
kind: SparkApplication
metadata:
name: pyfiles
name: pyspark-pi
namespace: default
spec:
type: Python
@ -27,18 +27,13 @@ spec:
mode: cluster
image: "gcr.io/spark-operator/spark-py:v2.4.0"
imagePullPolicy: Always
mainApplicationFile: local:///opt/spark/examples/src/main/python/pyfiles.py
mainApplicationFile: local:///opt/spark/examples/src/main/python/pi.py
restartPolicy:
type: OnFailure
onFailureRetries: 3
onFailureRetryInterval: 10
onSubmissionFailureRetries: 5
onSubmissionFailureRetryInterval: 20
arguments:
- python2.7
deps:
pyFiles:
- local:///opt/spark/examples/src/main/python/py_container_checks.py
driver:
cores: 0.1
coreLimit: "200m"

View File

@ -46,7 +46,7 @@ spec:
containers:
- name: sparkoperator
image: gcr.io/spark-operator/spark-operator:v2.4.0-v1alpha1-latest
imagePullPolicy: IfNotPresent
imagePullPolicy: Always
command: ["/usr/bin/spark-operator"]
ports:
- containerPort: 10254

View File

@ -46,7 +46,7 @@ spec:
containers:
- name: sparkoperator
image: gcr.io/spark-operator/spark-operator:v2.4.0-v1alpha1-latest
imagePullPolicy: IfNotPresent
imagePullPolicy: Always
volumeMounts:
- name: webhook-certs
mountPath: /etc/webhook-certs

View File

@ -42,7 +42,7 @@ spec:
containers:
- name: sparkoperator
image: gcr.io/spark-operator/spark-operator:v2.4.0-v1alpha1-latest
imagePullPolicy: IfNotPresent
imagePullPolicy: Always
command: ["/usr/bin/spark-operator"]
args:
- -logtostderr

View File

@ -174,42 +174,6 @@ func (c *Controller) onAdd(obj interface{}) {
glog.Infof("SparkApplication %s/%s was added, enqueueing it for submission", app.Namespace, app.Name)
c.enqueue(app)
}
func (c *Controller) updateSparkApplicationStatusWithRetries(
original *v1alpha1.SparkApplication,
updateFunc func(status *v1alpha1.SparkApplicationStatus)) error {
toUpdate := original.DeepCopy()
var lastUpdateErr error
for i := 0; i < maximumUpdateRetries; i++ {
updateFunc(&toUpdate.Status)
if reflect.DeepEqual(original.Status, toUpdate.Status) {
return nil
}
_, err := c.crdClient.SparkoperatorV1alpha1().SparkApplications(toUpdate.Namespace).Update(toUpdate)
if err == nil {
return nil
}
lastUpdateErr = err
// Failed to update to the API server.
// Get the latest version from the API server first and re-apply the update.
name := toUpdate.Name
toUpdate, err = c.crdClient.SparkoperatorV1alpha1().SparkApplications(toUpdate.Namespace).Get(name,
metav1.GetOptions{})
if err != nil {
glog.Errorf("failed to get SparkApplication %s/%s: %v", original.Namespace, name, err)
return err
}
}
if lastUpdateErr != nil {
glog.Errorf("failed to update SparkApplication %s/%s: %v", original.Namespace, original.Name, lastUpdateErr)
return lastUpdateErr
}
return nil
}
func (c *Controller) onUpdate(oldObj, newObj interface{}) {
oldApp := oldObj.(*v1alpha1.SparkApplication)
@ -219,7 +183,7 @@ func (c *Controller) onUpdate(oldObj, newObj interface{}) {
// and end up in an inconsistent state.
if !reflect.DeepEqual(oldApp.Spec, newApp.Spec) {
// Force-set the application status to Invalidating which handles clean-up and application re-run.
if err := c.updateSparkApplicationStatusWithRetries(newApp, func(status *v1alpha1.SparkApplicationStatus) {
if _, err := c.updateSparkApplicationStatusWithRetries(newApp, func(status *v1alpha1.SparkApplicationStatus) {
status.AppState.State = v1alpha1.InvalidatingState
}); err != nil {
c.recorder.Eventf(
@ -239,7 +203,7 @@ func (c *Controller) onUpdate(oldObj, newObj interface{}) {
newApp.Name)
}
}
glog.V(2).Infof("SparkApplication %s/%s enqueued for processing", newApp.GetNamespace(), newApp.GetName())
glog.V(2).Infof("SparkApplication %s/%s was updated, enqueueing it", newApp.Namespace, newApp.Name)
c.enqueue(newApp)
}
@ -294,19 +258,19 @@ func (c *Controller) processNextItem() bool {
return true
}
// Helper Struct to encapsulate current State of the driver pod.
// Helper data structure to encapsulate current state of the driver pod.
type driverState struct {
podName string // Name of the driver pod.
sparkApplicationID string // sparkApplicationID.
sparkApplicationID string // Spark application ID.
nodeName string // Name of the node the driver pod runs on.
podPhase apiv1.PodPhase // Driver pod phase.
completionTime metav1.Time // Time the driver completes.
}
func (c *Controller) getUpdatedAppStatus(app *v1alpha1.SparkApplication) v1alpha1.SparkApplicationStatus {
// Fetch all the pods for the App.
// Fetch all the pods for the application.
selector, err := labels.NewRequirement(config.SparkAppNameLabel, selection.Equals, []string{app.Name})
pods, err := c.podLister.Pods(app.GetNamespace()).List(labels.NewSelector().Add(*selector))
pods, err := c.podLister.Pods(app.Namespace).List(labels.NewSelector().Add(*selector))
if err != nil {
glog.Errorf("failed to get pods for SparkApplication %s/%s: %v", app.Namespace, app.Name, err)
@ -339,7 +303,6 @@ func (c *Controller) getUpdatedAppStatus(app *v1alpha1.SparkApplication) v1alpha
if currentDriverState != nil {
if newAppState, err := driverPodPhaseToApplicationState(currentDriverState.podPhase); err == nil {
// Valid Driver State: Update CRD.
app.Status.DriverInfo.PodName = currentDriverState.podName
app.Status.SparkApplicationID = currentDriverState.sparkApplicationID
if currentDriverState.nodeName != "" {
@ -365,7 +328,6 @@ func (c *Controller) getUpdatedAppStatus(app *v1alpha1.SparkApplication) v1alpha
app.Status.SparkApplicationID = executorApplicationID
}
// Apply Executor Updates
if app.Status.ExecutorState == nil {
app.Status.ExecutorState = make(map[string]v1alpha1.ExecutorState)
}
@ -377,7 +339,7 @@ func (c *Controller) getUpdatedAppStatus(app *v1alpha1.SparkApplication) v1alpha
for name, oldStatus := range app.Status.ExecutorState {
_, exists := executorStateMap[name]
if !isExecutorTerminated(oldStatus) && !exists {
glog.Infof("Executor Pod not found. Assuming pod %s was deleted.", name)
glog.Infof("Executor pod %s not found, assuming it was deleted.", name)
app.Status.ExecutorState[name] = v1alpha1.ExecutorFailedState
}
}
@ -387,10 +349,10 @@ func (c *Controller) getUpdatedAppStatus(app *v1alpha1.SparkApplication) v1alpha
func (c *Controller) handleSparkApplicationDeletion(app *v1alpha1.SparkApplication) *v1alpha1.SparkApplication {
// Validate if the driver pod no longer exists.
if c.validateSparkResourceDeletion(app, false) {
// Successfully deleted driver or driver doesn't exist. Remove it from the Finalizer List.
// Successfully deleted driver or driver doesn't exist. Remove it from the Finalizer list.
app = removeFinalizer(app, sparkDriverRole)
} else {
// CRD deletion requested, lets delete driver pod.
// SparkApplication deletion requested, lets delete driver pod.
err := c.deleteSparkResources(app, false)
if err != nil {
glog.Errorf("failed to delete the driver pod for deleted SparkApplication: %s/%s: %v", app.Namespace,
@ -409,7 +371,7 @@ func shouldRetry(app *v1alpha1.SparkApplication) bool {
if app.Spec.RestartPolicy.Type == v1alpha1.Always {
return true
} else if app.Spec.RestartPolicy.Type == v1alpha1.OnFailure {
// We Retry if we haven't hit the retry limit.
// We retry if we haven't hit the retry limit.
if app.Spec.RestartPolicy.OnFailureRetries != nil && app.Status.ExecutionAttempts <= *app.Spec.RestartPolicy.OnFailureRetries {
return true
}
@ -418,7 +380,7 @@ func shouldRetry(app *v1alpha1.SparkApplication) bool {
if app.Spec.RestartPolicy.Type == v1alpha1.Always {
return true
} else if app.Spec.RestartPolicy.Type == v1alpha1.OnFailure {
// We Retry if we haven't hit the retry limit.
// We retry if we haven't hit the retry limit.
if app.Spec.RestartPolicy.OnSubmissionFailureRetries != nil && app.Status.SubmissionAttempts <= *app.Spec.RestartPolicy.OnSubmissionFailureRetries {
return true
}
@ -427,7 +389,7 @@ func shouldRetry(app *v1alpha1.SparkApplication) bool {
return false
}
// State Machine Transition for Spark Application:
// State Machine for SparkApplication:
//+--------------------------------------------------------------------------------------------------------------------+
//| |
//| +---------+ |
@ -473,22 +435,26 @@ func (c *Controller) syncSparkApplication(key string) error {
if err != nil {
return err
}
if app == nil {
// SparkApplication not found.
return nil
}
appToUpdate := app.DeepCopy()
// CRD Delete requested.
// SparkApplication delete requested.
if !appToUpdate.GetDeletionTimestamp().IsZero() {
appToUpdate = c.handleSparkApplicationDeletion(appToUpdate)
} else {
// Take action based on AppState.
// Take action based on application state.
switch appToUpdate.Status.AppState.State {
case v1alpha1.NewState:
c.recordSparkApplicationEvent(appToUpdate)
appToUpdate.Status.SubmissionAttempts = 0
glog.Infof("Submitting SparkApplication %s", key)
appToUpdate = c.submitSparkApplication(appToUpdate)
case v1alpha1.SucceedingState:
if !shouldRetry(appToUpdate) {
// App will never be retried. Move to Terminal Completed State.
// App will never be retried. Move to terminal CompletedState.
appToUpdate.Status.AppState.State = v1alpha1.CompletedState
c.recordSparkApplicationEvent(appToUpdate)
} else {
@ -501,7 +467,7 @@ func (c *Controller) syncSparkApplication(key string) error {
}
case v1alpha1.FailingState:
if !shouldRetry(appToUpdate) {
// App will never be retried. Move to Terminal Failure State.
// App will never be retried. Move to terminal FailedState.
appToUpdate.Status.AppState.State = v1alpha1.FailedState
c.recordSparkApplicationEvent(appToUpdate)
} else if hasRetryIntervalPassed(appToUpdate.Spec.RestartPolicy.OnFailureRetryInterval, appToUpdate.Status.ExecutionAttempts, appToUpdate.Status.CompletionTime) {
@ -514,11 +480,10 @@ func (c *Controller) syncSparkApplication(key string) error {
}
case v1alpha1.FailedSubmissionState:
if !shouldRetry(appToUpdate) {
// App will never be retried. Move to Terminal Failure State.
// App will never be retried. Move to terminal FailedState.
appToUpdate.Status.AppState.State = v1alpha1.FailedState
c.recordSparkApplicationEvent(appToUpdate)
} else if hasRetryIntervalPassed(appToUpdate.Spec.RestartPolicy.OnSubmissionFailureRetryInterval, appToUpdate.Status.SubmissionAttempts, appToUpdate.Status.LastSubmissionAttemptTime) {
glog.Infof("Submitting SparkApplication %s", key)
appToUpdate = c.submitSparkApplication(appToUpdate)
}
case v1alpha1.InvalidatingState:
@ -532,25 +497,23 @@ func (c *Controller) syncSparkApplication(key string) error {
appToUpdate.Status.ExecutionAttempts = 0
case v1alpha1.PendingRerunState:
if c.validateSparkResourceDeletion(appToUpdate, true) {
// Reset SubmissionAttempts Count since this is a new overall run.
// Reset SubmissionAttempts count since this is a new overall run.
appToUpdate.Status.SubmissionAttempts = 0
// Successfully deleted driver or driver doesn't exist. Remove it from the Finalizer List.
// Successfully deleted driver or driver doesn't exist. Remove it from the Finalizer list.
appToUpdate = removeFinalizer(appToUpdate, sparkDriverRole)
glog.Infof("Submitting SparkApplication %s", key)
appToUpdate = c.submitSparkApplication(appToUpdate)
}
case v1alpha1.SubmittedState, v1alpha1.RunningState:
//App already submitted, get driver and executor pods and update Status.
//Application already submitted, get driver and executor pods and update its status.
appToUpdate.Status = c.getUpdatedAppStatus(appToUpdate)
}
}
// Update CRD if not nil.
if appToUpdate != nil {
glog.V(2).Infof("Trying to update SparkApplication %s/%s, from: [%v] to [%v]", app.Namespace, app.Name, app.Status, appToUpdate.Status)
err = c.updateAppAndExportMetrics(app, appToUpdate)
if err != nil {
glog.Errorf("failed to update SparkApplication %s/%s: %v", app.Namespace, app.GetName(), err)
glog.Errorf("failed to update SparkApplication %s/%s: %v", app.Namespace, app.Name, err)
return err
}
}
@ -574,7 +537,7 @@ func hasRetryIntervalPassed(retryInterval *int64, attemptsDone int32, lastEventT
// submitSparkApplication creates a new submission for the given SparkApplication and submits it using spark-submit.
func (c *Controller) submitSparkApplication(app *v1alpha1.SparkApplication) *v1alpha1.SparkApplication {
// Clone app since configPrometheusMonitoring may update app.Spec which causes an onUpdate callback with Spec update.
// Make a copy since configPrometheusMonitoring may update app.Spec which causes an onUpdate callback.
appToSubmit := app.DeepCopy()
if appToSubmit.Spec.Monitoring != nil && appToSubmit.Spec.Monitoring.Prometheus != nil {
configPrometheusMonitoring(appToSubmit, c.kubeClient)
@ -594,7 +557,7 @@ func (c *Controller) submitSparkApplication(app *v1alpha1.SparkApplication) *v1a
}
// Try submitting the application by running spark-submit.
err = runSparkSubmit(newSubmission(submissionCmdArgs, appToSubmit))
submitted, err := runSparkSubmit(newSubmission(submissionCmdArgs, appToSubmit))
if err != nil {
app.Status = v1alpha1.SparkApplicationStatus{
AppState: v1alpha1.ApplicationState{
@ -605,13 +568,17 @@ func (c *Controller) submitSparkApplication(app *v1alpha1.SparkApplication) *v1a
LastSubmissionAttemptTime: metav1.Now(),
}
c.recordSparkApplicationEvent(app)
glog.Errorf("failed to run spark-submit for SparkApplication %s/%s: %v", appToSubmit.Namespace, appToSubmit.Name, err)
glog.Errorf("failed to run spark-submit for SparkApplication %s/%s: %v", app.Namespace, app.Name, err)
return app
}
if !submitted {
// The application may not have been submitted even if err == nil, e.g., when some
// state update caused an attempt to re-submit the application, in which case no
// error gets returned from runSparkSubmit. If this is the case, we simply return.
return app
}
// Submission Successful or Driver Pod Already exists.
glog.Infof("spark-submit completed for SparkApplication %s/%s", app.Namespace, app.Name)
// Update AppStatus to submitted.
glog.Infof("SparkApplication %s/%s has been submitted", app.Namespace, app.Name)
app.Status = v1alpha1.SparkApplicationStatus{
AppState: v1alpha1.ApplicationState{
State: v1alpha1.SubmittedState,
@ -622,10 +589,9 @@ func (c *Controller) submitSparkApplication(app *v1alpha1.SparkApplication) *v1a
}
c.recordSparkApplicationEvent(app)
// Add driver as a finalizer to prevent SparkApplication deletion till driver is deleted.
// Add a fianlizer to the driver pod to prevent SparkApplication deletion till driver is deleted.
app = addFinalizer(app, sparkDriverRole)
// Create Spark UI Service.
service, err := createSparkUIService(app, c.kubeClient)
if err != nil {
glog.Errorf("failed to create UI service for SparkApplication %s/%s: %v", app.Namespace, app.Name, err)
@ -646,13 +612,52 @@ func (c *Controller) submitSparkApplication(app *v1alpha1.SparkApplication) *v1a
return app
}
func (c *Controller) updateSparkApplicationStatusWithRetries(
original *v1alpha1.SparkApplication,
updateFunc func(status *v1alpha1.SparkApplicationStatus)) (*v1alpha1.SparkApplication, error) {
toUpdate := original.DeepCopy()
var lastUpdateErr error
for i := 0; i < maximumUpdateRetries; i++ {
updateFunc(&toUpdate.Status)
if reflect.DeepEqual(original.Status, toUpdate.Status) {
return toUpdate, nil
}
_, err := c.crdClient.SparkoperatorV1alpha1().SparkApplications(toUpdate.Namespace).Update(toUpdate)
if err == nil {
return toUpdate, nil
}
lastUpdateErr = err
// Failed to update to the API server.
// Get the latest version from the API server first and re-apply the update.
name := toUpdate.Name
toUpdate, err = c.crdClient.SparkoperatorV1alpha1().SparkApplications(toUpdate.Namespace).Get(name,
metav1.GetOptions{})
if err != nil {
glog.Errorf("failed to get SparkApplication %s/%s: %v", original.Namespace, name, err)
return nil, err
}
}
if lastUpdateErr != nil {
glog.Errorf("failed to update SparkApplication %s/%s: %v", original.Namespace, original.Name, lastUpdateErr)
return nil, lastUpdateErr
}
return toUpdate, nil
}
func (c *Controller) updateAppAndExportMetrics(oldApp, newApp *v1alpha1.SparkApplication) error {
// Skip update if nothing changed.
if reflect.DeepEqual(oldApp, newApp) {
return nil
}
app, err := c.crdClient.SparkoperatorV1alpha1().SparkApplications(newApp.Namespace).Update(newApp)
app, err := c.updateSparkApplicationStatusWithRetries(oldApp, func(status *v1alpha1.SparkApplicationStatus) {
*status = newApp.Status
})
// Export metrics if the update was successful.
if err == nil && c.metrics != nil {
c.metrics.exportMetrics(oldApp, app)
@ -661,10 +666,17 @@ func (c *Controller) updateAppAndExportMetrics(oldApp, newApp *v1alpha1.SparkApp
}
func (c *Controller) getSparkApplication(namespace string, name string) (*v1alpha1.SparkApplication, error) {
return c.applicationLister.SparkApplications(namespace).Get(name)
app, err := c.applicationLister.SparkApplications(namespace).Get(name)
if err != nil {
if errors.IsNotFound(err) {
return nil, nil
}
return nil, err
}
return app, nil
}
// Delete driver pod and optional UI Resources(UI/Ingress) created for the Application.
// Delete the driver pod and optional UI resources (Service/Ingress) created for the application.
func (c *Controller) deleteSparkResources(app *v1alpha1.SparkApplication, deleteUI bool) error {
driverPodName := app.Status.DriverInfo.PodName
@ -699,7 +711,7 @@ func (c *Controller) deleteSparkResources(app *v1alpha1.SparkApplication, delete
return nil
}
// Validate that any Spark Resources(driver/UI/Ingress) created for the Application have been deleted.
// Validate that any Spark resources (driver/Service/Ingress) created for the application have been deleted.
func (c *Controller) validateSparkResourceDeletion(app *v1alpha1.SparkApplication, validateUIDeletion bool) bool {
driverPodName := app.Status.DriverInfo.PodName
if driverPodName == "" {
@ -786,7 +798,7 @@ func (c *Controller) recordSparkApplicationEvent(app *v1alpha1.SparkApplication)
apiv1.EventTypeNormal,
"SparkApplicationCompleted",
"SparkApplication %s terminated with state: %v",
app.GetName(),
app.Name,
app.Status.AppState.State)
case v1alpha1.FailedState:
c.recorder.Eventf(
@ -794,7 +806,7 @@ func (c *Controller) recordSparkApplicationEvent(app *v1alpha1.SparkApplication)
apiv1.EventTypeWarning,
"SparkApplicationFailed",
"SparkApplication %s terminated with state: %v",
app.GetName(),
app.Name,
app.Status.AppState.State)
}
}

View File

@ -285,7 +285,7 @@ func TestSyncSparkApplication_SubmissionFailed(t *testing.T) {
assert.True(t, strings.Contains(event, "SparkApplicationSubmissionFailed"))
// Attempt 2: Retry again
updatedApp.Status.LastSubmissionAttemptTime = metav1.Time{metav1.Now().Add(-100 * time.Second)}
updatedApp.Status.LastSubmissionAttemptTime = metav1.Time{Time: metav1.Now().Add(-100 * time.Second)}
ctrl, recorder = newFakeController(updatedApp)
_, err = ctrl.crdClient.SparkoperatorV1alpha1().SparkApplications(app.Namespace).Create(updatedApp)
if err != nil {
@ -304,7 +304,7 @@ func TestSyncSparkApplication_SubmissionFailed(t *testing.T) {
assert.True(t, strings.Contains(event, "SparkApplicationSubmissionFailed"))
// Attempt 3: No more retries
updatedApp.Status.LastSubmissionAttemptTime = metav1.Time{metav1.Now().Add(-100 * time.Second)}
updatedApp.Status.LastSubmissionAttemptTime = metav1.Time{Time: metav1.Now().Add(-100 * time.Second)}
ctrl, recorder = newFakeController(updatedApp)
_, err = ctrl.crdClient.SparkoperatorV1alpha1().SparkApplications(app.Namespace).Create(updatedApp)
if err != nil {
@ -1033,14 +1033,14 @@ func TestSyncSparkApplication_ExecutingState(t *testing.T) {
func TestHasRetryIntervalPassed(t *testing.T) {
// Failure Cases
assert.False(t, hasRetryIntervalPassed(nil, 3, metav1.Time{metav1.Now().Add(-100 * time.Second)}))
assert.False(t, hasRetryIntervalPassed(int64ptr(5), 0, metav1.Time{metav1.Now().Add(-100 * time.Second)}))
assert.False(t, hasRetryIntervalPassed(nil, 3, metav1.Time{Time: metav1.Now().Add(-100 * time.Second)}))
assert.False(t, hasRetryIntervalPassed(int64ptr(5), 0, metav1.Time{Time: metav1.Now().Add(-100 * time.Second)}))
assert.False(t, hasRetryIntervalPassed(int64ptr(5), 3, metav1.Time{}))
// Not enough time passed.
assert.False(t, hasRetryIntervalPassed(int64ptr(50), 3, metav1.Time{metav1.Now().Add(-100 * time.Second)}))
assert.False(t, hasRetryIntervalPassed(int64ptr(50), 3, metav1.Time{Time: metav1.Now().Add(-100 * time.Second)}))
assert.True(t, hasRetryIntervalPassed(int64ptr(50), 3, metav1.Time{metav1.Now().Add(-151 * time.Second)}))
assert.True(t, hasRetryIntervalPassed(int64ptr(50), 3, metav1.Time{Time: metav1.Now().Add(-151 * time.Second)}))
}
func stringptr(s string) *string {

View File

@ -53,7 +53,7 @@ func newSubmission(args []string, app *v1alpha1.SparkApplication) *submission {
}
}
func runSparkSubmit(submission *submission) error {
func runSparkSubmit(submission *submission) (bool, error) {
sparkHome, present := os.LookupEnv(sparkHomeEnvVar)
if !present {
glog.Error("SPARK_HOME is not specified")
@ -61,22 +61,22 @@ func runSparkSubmit(submission *submission) error {
var command = filepath.Join(sparkHome, "/bin/spark-submit")
cmd := execCommand(command, submission.args...)
glog.Infof("spark-submit arguments: %v", cmd.Args)
glog.V(2).Infof("spark-submit arguments: %v", cmd.Args)
if _, err := cmd.Output(); err != nil {
var errorMsg string
if exitErr, ok := err.(*exec.ExitError); ok {
errorMsg = string(exitErr.Stderr)
}
// Already Exists. Do nothing.
// The driver pod of the application already exists.
if strings.Contains(errorMsg, podAlreadyExistsErrorCode) {
glog.Warningf("trying to resubmit an already submitted SparkApplication %s/%s", submission.namespace, submission.name)
return nil
return false, nil
}
return fmt.Errorf("failed to run spark-submit for SparkApplication %s/%s: %v", submission.namespace, submission.name, err)
return false, fmt.Errorf("failed to run spark-submit for SparkApplication %s/%s: %v", submission.namespace, submission.name, err)
}
return nil
return true, nil
}
func buildSubmissionCommandArgs(app *v1alpha1.SparkApplication) ([]string, error) {