diff --git a/charts/spark-operator-chart/README.md b/charts/spark-operator-chart/README.md index 0a83dbd2f..65b33f75d 100644 --- a/charts/spark-operator-chart/README.md +++ b/charts/spark-operator-chart/README.md @@ -129,6 +129,9 @@ See [helm uninstall](https://helm.sh/docs/helm/helm_uninstall) for command docum | controller.workqueueRateLimiter.bucketSize | int | `500` | Specifies the maximum number of items that can be in the workqueue at any given time. | | controller.workqueueRateLimiter.maxDelay.enable | bool | `true` | Specifies whether to enable max delay for the workqueue rate limiter. This is useful to avoid losing events when the workqueue is full. | | controller.workqueueRateLimiter.maxDelay.duration | string | `"6h"` | Specifies the maximum delay duration for the workqueue rate limiter. | +| controller.submitter.type | string | `"default"` | Type of submitter to use: 'default' or 'grpc'. | +| controller.submitter.grpcServerAddress | string | `"localhost:50051"` | gRPC server address for alternate Spark submit (used when submitter.type is 'grpc'). | +| controller.submitter.grpcSubmitTimeout | string | `"10s"` | Timeout for gRPC Spark submit (used when submitter.type is 'grpc'). | | webhook.enable | bool | `true` | Specifies whether to enable webhook. | | webhook.replicas | int | `1` | Number of replicas of webhook server. | | webhook.leaderElection.enable | bool | `true` | Specifies whether to enable leader election for webhook. | diff --git a/charts/spark-operator-chart/templates/controller/deployment.yaml b/charts/spark-operator-chart/templates/controller/deployment.yaml index 54673b43e..a1b10bef7 100644 --- a/charts/spark-operator-chart/templates/controller/deployment.yaml +++ b/charts/spark-operator-chart/templates/controller/deployment.yaml @@ -120,6 +120,15 @@ spec: {{- if .Values.controller.maxTrackedExecutorPerApp }} - --max-tracked-executor-per-app={{ .Values.controller.maxTrackedExecutorPerApp }} {{- end }} + {{- if .Values.controller.submitter.type }} + - --submitter-type={{ .Values.controller.submitter.type }} + {{- end }} + {{- if .Values.controller.submitter.grpcServerAddress }} + - --grpc-server-address={{ .Values.controller.submitter.grpcServerAddress }} + {{- end }} + {{- if .Values.controller.submitter.grpcSubmitTimeout }} + - --grpc-submit-timeout={{ .Values.controller.submitter.grpcSubmitTimeout }} + {{- end }} {{- if or .Values.prometheus.metrics.enable .Values.controller.pprof.enable }} ports: {{- if .Values.controller.pprof.enable }} @@ -161,6 +170,35 @@ spec: securityContext: {{- toYaml . | nindent 10 }} {{- end }} + {{- if .Values.controller.submitter.nativeSubmitPlugin.enabled }} + - name: native-submit + image: {{ .Values.controller.submitter.nativeSubmitPlugin.image }} + {{- with .Values.controller.submitter.nativeSubmitPlugin.imagePullPolicy }} + imagePullPolicy: {{ . }} + {{- end }} + {{- with .Values.controller.submitter.nativeSubmitPlugin.ports }} + ports: + {{- toYaml . | nindent 8 }} + {{- end }} + {{- with .Values.controller.submitter.nativeSubmitPlugin.livenessProbe }} + livenessProbe: + {{- toYaml . | nindent 10 }} + {{- end }} + {{- with .Values.controller.submitter.nativeSubmitPlugin.readinessProbe }} + readinessProbe: + {{- toYaml . | nindent 10 }} + {{- end }} + {{- with .Values.controller.submitter.nativeSubmitPlugin.resources }} + resources: + {{- toYaml . | nindent 10 }} + {{- end }} + {{- with .Values.controller.submitter.nativeSubmitPlugin.terminationMessagePath }} + terminationMessagePath: {{ . }} + {{- end }} + {{- with .Values.controller.submitter.nativeSubmitPlugin.terminationMessagePolicy }} + terminationMessagePolicy: {{ . }} + {{- end }} + {{- end }} {{- with .Values.controller.sidecars }} {{- toYaml . | nindent 6 }} {{- end }} diff --git a/charts/spark-operator-chart/tests/controller/deployment_test.yaml b/charts/spark-operator-chart/tests/controller/deployment_test.yaml index 1f17d5166..bf0f01db7 100644 --- a/charts/spark-operator-chart/tests/controller/deployment_test.yaml +++ b/charts/spark-operator-chart/tests/controller/deployment_test.yaml @@ -727,3 +727,71 @@ tests: - contains: path: spec.template.spec.containers[?(@.name=="spark-operator-controller")].args content: --max-tracked-executor-per-app=123 + + - it: Should add native-submit sidecar container if `controller.submitter.nativeSubmitPlugin.enabled` is set to `true` + set: + controller: + submitter: + nativeSubmitPlugin: + enabled: true + image: "docker.io/library/native-submit-plugin:08142025" + imagePullPolicy: "IfNotPresent" + ports: + - containerPort: 50051 + name: metrics + protocol: TCP + livenessProbe: + failureThreshold: 3 + httpGet: + path: /healthz + port: 9090 + scheme: HTTP + periodSeconds: 10 + successThreshold: 1 + timeoutSeconds: 1 + readinessProbe: + failureThreshold: 3 + httpGet: + path: /readyz + port: 9090 + scheme: HTTP + periodSeconds: 10 + successThreshold: 1 + timeoutSeconds: 1 + terminationMessagePath: /dev/termination-log + terminationMessagePolicy: File + resources: + limits: + cpu: 100m + memory: 128Mi + requests: + cpu: 50m + memory: 64Mi + asserts: + - equal: + path: spec.template.spec.containers[?(@.name=="native-submit")].name + value: native-submit + - equal: + path: spec.template.spec.containers[?(@.name=="native-submit")].image + value: docker.io/library/native-submit-plugin:08142025 + - equal: + path: spec.template.spec.containers[?(@.name=="native-submit")].imagePullPolicy + value: IfNotPresent + - contains: + path: spec.template.spec.containers[?(@.name=="native-submit")].ports + content: + name: metrics + containerPort: 50051 + protocol: TCP + - equal: + path: spec.template.spec.containers[?(@.name=="native-submit")].livenessProbe.httpGet.port + value: 9090 + - equal: + path: spec.template.spec.containers[?(@.name=="native-submit")].livenessProbe.httpGet.path + value: /healthz + - equal: + path: spec.template.spec.containers[?(@.name=="native-submit")].readinessProbe.httpGet.port + value: 9090 + - equal: + path: spec.template.spec.containers[?(@.name=="native-submit")].readinessProbe.httpGet.path + value: /readyz diff --git a/charts/spark-operator-chart/values-grpc.yaml b/charts/spark-operator-chart/values-grpc.yaml new file mode 100644 index 000000000..9d15c098d --- /dev/null +++ b/charts/spark-operator-chart/values-grpc.yaml @@ -0,0 +1,55 @@ +# Example values file for gRPC submitter with native submit plugin sidecar +image: + repository: docker.io/library/spark-operator + tag: dev + pullPolicy: IfNotPresent +controller: + submitter: + # Enable gRPC submitter + type: "grpc" + # gRPC server address (points to the sidecar container) + grpcServerAddress: "localhost:50051" + # Timeout for gRPC Spark submit + grpcSubmitTimeout: "10s" + # Enable native submit plugin sidecar + nativeSubmitPlugin: + enabled: true + image: "docker.io/library/native-submit-plugin:08142025" + imagePullPolicy: "IfNotPresent" + port: 50051 + portName: "grpc" + # Container ports configuration + ports: + - containerPort: 50051 + name: metrics + protocol: TCP + # Liveness probe configuration + livenessProbe: + failureThreshold: 3 + httpGet: + path: /healthz + port: 9090 + scheme: HTTP + periodSeconds: 10 + successThreshold: 1 + timeoutSeconds: 1 + # Readiness probe configuration + readinessProbe: + failureThreshold: 3 + httpGet: + path: /readyz + port: 9090 + scheme: HTTP + periodSeconds: 10 + successThreshold: 1 + timeoutSeconds: 1 + # Container termination configuration + terminationMessagePath: /dev/termination-log + terminationMessagePolicy: File + resources: + limits: + cpu: 100m + memory: 128Mi + requests: + cpu: 50m + memory: 64Mi \ No newline at end of file diff --git a/charts/spark-operator-chart/values.yaml b/charts/spark-operator-chart/values.yaml index 0a7ac47b7..5da24db3f 100644 --- a/charts/spark-operator-chart/values.yaml +++ b/charts/spark-operator-chart/values.yaml @@ -198,6 +198,64 @@ controller: # -- Sidecar containers for controller pods. sidecars: [] + + # -- Spark submitter configuration + submitter: + # -- Type of submitter to use: 'default' or 'grpc' + type: "default" + # -- gRPC server address for alternate Spark submit (used when submitter.type is 'grpc') + # When using native submit plugin sidecar, this should be 'localhost:50051' + grpcServerAddress: "localhost:50051" + # -- Timeout for gRPC Spark submit (used when submitter.type is 'grpc') + grpcSubmitTimeout: "10s" + # -- Native submit plugin sidecar configuration + nativeSubmitPlugin: + # -- Enable native submit plugin sidecar + enabled: false + # -- Native submit plugin image + image: "docker.io/library/native-submit-plugin:08142025" + # -- Native submit plugin image pull policy + imagePullPolicy: "IfNotPresent" + # -- Native submit plugin container port + port: 50051 + # -- Native submit plugin container port name + portName: "grpc" + # -- Container ports configuration + ports: + - containerPort: 50051 + name: metrics + protocol: TCP + # -- Liveness probe configuration + livenessProbe: + failureThreshold: 3 + httpGet: + path: /healthz + port: 9090 + scheme: HTTP + periodSeconds: 10 + successThreshold: 1 + timeoutSeconds: 1 + # -- Readiness probe configuration + readinessProbe: + failureThreshold: 3 + httpGet: + path: /readyz + port: 9090 + scheme: HTTP + periodSeconds: 10 + successThreshold: 1 + timeoutSeconds: 1 + # -- Container termination configuration + terminationMessagePath: /dev/termination-log + terminationMessagePolicy: File + # -- Resource requests and limits for native submit plugin + resources: {} + # limits: + # cpu: 100m + # memory: 128Mi + # requests: + # cpu: 50m + # memory: 64Mi # Pod disruption budget for controller to avoid service degradation. podDisruptionBudget: diff --git a/cmd/operator/controller/start.go b/cmd/operator/controller/start.go index 94a9659b4..87c519c10 100644 --- a/cmd/operator/controller/start.go +++ b/cmd/operator/controller/start.go @@ -124,6 +124,12 @@ var ( zapOptions = logzap.Options{} ) +var ( + submitterType string + grpcServerAddress string + grpcSubmitTimeout time.Duration +) + func init() { utilruntime.Must(clientgoscheme.AddToScheme(scheme)) utilruntime.Must(schedulingv1alpha1.AddToScheme(scheme)) @@ -204,6 +210,10 @@ func NewStartCommand() *cobra.Command { command.Flags().StringVar(&pprofBindAddress, "pprof-bind-address", "0", "The address the pprof endpoint binds to. "+ "If not set, it will be 0 in order to disable the pprof server") + command.Flags().StringVar(&submitterType, "submitter-type", "default", "SparkApplication submitter type: 'default' or 'grpc'.") + command.Flags().StringVar(&grpcServerAddress, "grpc-server-address", "localhost:50051", "gRPC server address for alternate Spark submit.") + command.Flags().DurationVar(&grpcSubmitTimeout, "grpc-submit-timeout", 10*time.Second, "Timeout for gRPC Spark submit.") + flagSet := flag.NewFlagSet("controller", flag.ExitOnError) ctrl.RegisterFlags(flagSet) zapOptions.BindFlags(flagSet) @@ -286,7 +296,15 @@ func start() { } } - sparkSubmitter := &sparkapplication.SparkSubmitter{} + var sparkSubmitter sparkapplication.SparkApplicationSubmitter + switch submitterType { + case "grpc": + sparkSubmitter = sparkapplication.NewGRPCSparkSubmitter(grpcServerAddress, grpcSubmitTimeout) + logger.Info("Using gRPC SparkApplication submitter", "address", grpcServerAddress, "timeout", grpcSubmitTimeout) + default: + sparkSubmitter = &sparkapplication.SparkSubmitter{} + logger.Info("Using default SparkApplication submitter") + } // Setup controller for SparkApplication. if err = sparkapplication.NewReconciler( @@ -441,6 +459,9 @@ func newSparkApplicationReconcilerOptions() sparkapplication.Options { SparkApplicationMetrics: sparkApplicationMetrics, SparkExecutorMetrics: sparkExecutorMetrics, MaxTrackedExecutorPerApp: maxTrackedExecutorPerApp, + SubmitterType: submitterType, + GRPCServerAddress: grpcServerAddress, + GRPCSubmitTimeout: grpcSubmitTimeout, } if enableBatchScheduler { options.KubeSchedulerNames = kubeSchedulerNames diff --git a/config/crd/kustomization.yaml b/config/crd/kustomization.yaml index 3d5605b3e..3df68a3eb 100644 --- a/config/crd/kustomization.yaml +++ b/config/crd/kustomization.yaml @@ -4,6 +4,7 @@ resources: - bases/sparkoperator.k8s.io_scheduledsparkapplications.yaml - bases/sparkoperator.k8s.io_sparkapplications.yaml +- bases/sparkoperator.k8s.io_sparkconnects.yaml # +kubebuilder:scaffold:crdkustomizeresource patches: diff --git a/internal/controller/sparkapplication/controller.go b/internal/controller/sparkapplication/controller.go index a5adf1ce7..3d80013c3 100644 --- a/internal/controller/sparkapplication/controller.go +++ b/internal/controller/sparkapplication/controller.go @@ -69,6 +69,11 @@ type Options struct { SparkExecutorMetrics *metrics.SparkExecutorMetrics MaxTrackedExecutorPerApp int + + // Submitter configuration + SubmitterType string // "default" or "grpc" + GRPCServerAddress string // gRPC server address, e.g., "localhost:50051" + GRPCSubmitTimeout time.Duration // gRPC submit timeout, e.g., 10 * time.Second } // Reconciler reconciles a SparkApplication object. diff --git a/internal/controller/sparkapplication/grpc_submitter.go b/internal/controller/sparkapplication/grpc_submitter.go new file mode 100644 index 000000000..6b76421df --- /dev/null +++ b/internal/controller/sparkapplication/grpc_submitter.go @@ -0,0 +1,309 @@ +/* +Copyright 2024 The Kubeflow authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + https://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package sparkapplication + +import ( + "context" + "fmt" + "time" + + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/protobuf/types/known/wrapperspb" + "sigs.k8s.io/controller-runtime/pkg/log" + + proto "github.com/kubeflow/spark-operator/v2/proto" + + "github.com/kubeflow/spark-operator/v2/api/v1beta2" +) + +// Patch point for grpc.DialContext (for testing) +var grpcDialContext = grpc.DialContext + +// GRPCSparkSubmitter submits a SparkApplication using gRPC. +type GRPCSparkSubmitter struct { + serverAddress string + timeout time.Duration +} + +// NewGRPCSparkSubmitter creates a new GRPCSparkSubmitter instance. +func NewGRPCSparkSubmitter(serverAddress string, timeout time.Duration) *GRPCSparkSubmitter { + return &GRPCSparkSubmitter{ + serverAddress: serverAddress, + timeout: timeout, + } +} + +// GRPCSparkSubmitter implements SparkApplicationSubmitter interface. +var _ SparkApplicationSubmitter = &GRPCSparkSubmitter{} + +// Submit implements SparkApplicationSubmitter interface. +func (g *GRPCSparkSubmitter) Submit(ctx context.Context, app *v1beta2.SparkApplication) error { + logger := log.FromContext(ctx) + + // Create a context with timeout + ctx, cancel := context.WithTimeout(ctx, g.timeout) + defer cancel() + + // Connect to the gRPC server + conn, err := grpcDialContext(ctx, g.serverAddress, grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + return fmt.Errorf("failed to connect to gRPC server at %s: %v", g.serverAddress, err) + } + defer conn.Close() + + // Create gRPC client + client := proto.NewSparkSubmitServiceClient(conn) + + // Convert v1beta2.SparkApplication to protobuf SparkApplication + protoApp, err := g.convertToProto(app) + if err != nil { + return fmt.Errorf("failed to convert SparkApplication to protobuf: %v", err) + } + + // Create the request + request := &proto.RunAltSparkSubmitRequest{ + SparkApplication: protoApp, + SubmissionId: app.Status.SubmissionID, + } + + // Call the gRPC service + logger.Info("Submitting SparkApplication via gRPC", "server", g.serverAddress, "submissionId", app.Status.SubmissionID) + response, err := client.RunAltSparkSubmit(ctx, request) + if err != nil { + return fmt.Errorf("failed to submit SparkApplication via gRPC: %v", err) + } + + // Check the response + if !response.Success { + return fmt.Errorf("gRPC submission failed: %s", response.ErrorMessage) + } + + logger.Info("Successfully submitted SparkApplication via gRPC", "submissionId", app.Status.SubmissionID) + return nil +} + +// convertToProto converts a v1beta2.SparkApplication to a protobuf SparkApplication. +func (g *GRPCSparkSubmitter) convertToProto(app *v1beta2.SparkApplication) (*proto.SparkApplication, error) { + // Convert metadata + metadata := &proto.ObjectMeta{ + Name: app.Name, + Namespace: app.Namespace, + Labels: app.Labels, + Annotations: app.Annotations, + } + + // Convert spec + spec, err := g.convertSpecToProto(&app.Spec) + if err != nil { + return nil, fmt.Errorf("failed to convert spec: %v", err) + } + + // Convert status + status := &proto.SparkApplicationStatus{ + ApplicationState: string(app.Status.AppState.State), + SparkApplicationId: app.Status.SparkApplicationID, + SubmissionId: app.Status.SubmissionID, + } + + return &proto.SparkApplication{ + Metadata: metadata, + Spec: spec, + Status: status, + }, nil +} + +// convertSpecToProto converts a v1beta2.SparkApplicationSpec to a protobuf SparkApplicationSpec. +func (g *GRPCSparkSubmitter) convertSpecToProto(spec *v1beta2.SparkApplicationSpec) (*proto.SparkApplicationSpec, error) { + protoSpec := &proto.SparkApplicationSpec{ + Type: g.convertApplicationType(spec.Type), + Mode: g.convertDeployMode(spec.Mode), + SparkConf: spec.SparkConf, + HadoopConf: spec.HadoopConf, + Arguments: spec.Arguments, + SparkVersion: spec.SparkVersion, + PythonVersion: func() string { + if spec.PythonVersion != nil { + return *spec.PythonVersion + } + return "" + }(), + } + + // Convert optional string fields + if spec.Image != nil { + protoSpec.Image = wrapperspb.String(*spec.Image) + } + if spec.ImagePullPolicy != nil { + protoSpec.ImagePullPolicy = wrapperspb.String(*spec.ImagePullPolicy) + } + if spec.SparkConfigMap != nil { + protoSpec.SparkConfigMap = wrapperspb.String(*spec.SparkConfigMap) + } + if spec.HadoopConfigMap != nil { + protoSpec.HadoopConfigMap = wrapperspb.String(*spec.HadoopConfigMap) + } + if spec.MainClass != nil { + protoSpec.MainClass = wrapperspb.String(*spec.MainClass) + } + if spec.MainApplicationFile != nil { + protoSpec.MainApplicationFile = wrapperspb.String(*spec.MainApplicationFile) + } + if spec.ProxyUser != nil { + protoSpec.ProxyUser = wrapperspb.String(*spec.ProxyUser) + } + if spec.MemoryOverheadFactor != nil { + protoSpec.MemoryOverheadFactor = wrapperspb.String(*spec.MemoryOverheadFactor) + } + if spec.BatchScheduler != nil { + protoSpec.BatchScheduler = wrapperspb.String(*spec.BatchScheduler) + } + if spec.TimeToLiveSeconds != nil { + protoSpec.TimeToLiveSeconds = wrapperspb.Int64(*spec.TimeToLiveSeconds) + } + if spec.FailureRetries != nil { + protoSpec.FailureRetries = wrapperspb.Int32(*spec.FailureRetries) + } + if spec.RetryInterval != nil { + protoSpec.RetryInterval = wrapperspb.Int64(*spec.RetryInterval) + } + + // Convert image pull secrets + if len(spec.ImagePullSecrets) > 0 { + protoSpec.ImagePullSecrets = spec.ImagePullSecrets + } + + // Convert dependencies + if spec.Deps.Jars != nil || spec.Deps.Files != nil || spec.Deps.PyFiles != nil || + spec.Deps.Packages != nil || spec.Deps.ExcludePackages != nil || + spec.Deps.Repositories != nil || spec.Deps.Archives != nil { + protoSpec.Deps = &proto.Dependencies{ + Jars: spec.Deps.Jars, + Files: spec.Deps.Files, + PyFiles: spec.Deps.PyFiles, + Packages: spec.Deps.Packages, + ExcludePackages: spec.Deps.ExcludePackages, + Repositories: spec.Deps.Repositories, + Archives: spec.Deps.Archives, + } + } + + // Convert dynamic allocation + if spec.DynamicAllocation != nil { + protoSpec.DynamicAllocation = &proto.DynamicAllocation{ + Enabled: spec.DynamicAllocation.Enabled, + InitialExecutors: int32(0), + MinExecutors: int32(0), + MaxExecutors: int32(0), + ShuffleTrackingTimeout: func() int64 { + if spec.DynamicAllocation.ShuffleTrackingTimeout != nil { + return *spec.DynamicAllocation.ShuffleTrackingTimeout + } + return 0 + }(), + } + if spec.DynamicAllocation.InitialExecutors != nil { + protoSpec.DynamicAllocation.InitialExecutors = *spec.DynamicAllocation.InitialExecutors + } + if spec.DynamicAllocation.MinExecutors != nil { + protoSpec.DynamicAllocation.MinExecutors = *spec.DynamicAllocation.MinExecutors + } + if spec.DynamicAllocation.MaxExecutors != nil { + protoSpec.DynamicAllocation.MaxExecutors = *spec.DynamicAllocation.MaxExecutors + } + } + + // Convert restart policy + if spec.RestartPolicy.Type != "" { + protoSpec.RestartPolicy = &proto.RestartPolicy{ + Type: string(spec.RestartPolicy.Type), + } + } + + // Convert driver spec + if spec.Driver.Cores != nil || spec.Driver.Memory != nil || spec.Driver.ServiceAccount != nil { + protoSpec.Driver = &proto.DriverSpec{} + + if spec.Driver.Cores != nil { + protoSpec.Driver.SparkPodSpec = &proto.SparkPodSpec{ + Cores: wrapperspb.Int32(*spec.Driver.Cores), + } + } + if spec.Driver.Memory != nil { + if protoSpec.Driver.SparkPodSpec == nil { + protoSpec.Driver.SparkPodSpec = &proto.SparkPodSpec{} + } + protoSpec.Driver.SparkPodSpec.Memory = *spec.Driver.Memory + } + if spec.Driver.ServiceAccount != nil { + if protoSpec.Driver.SparkPodSpec == nil { + protoSpec.Driver.SparkPodSpec = &proto.SparkPodSpec{} + } + protoSpec.Driver.SparkPodSpec.ServiceAccount = wrapperspb.String(*spec.Driver.ServiceAccount) + } + } + + // Convert executor spec + if spec.Executor.Instances != nil || spec.Executor.Cores != nil || spec.Executor.Memory != nil { + protoSpec.Executor = &proto.ExecutorSpec{} + + if spec.Executor.Instances != nil { + protoSpec.Executor.Instances = wrapperspb.Int32(*spec.Executor.Instances) + } + if spec.Executor.Cores != nil || spec.Executor.Memory != nil { + protoSpec.Executor.SparkPodSpec = &proto.SparkPodSpec{} + if spec.Executor.Cores != nil { + protoSpec.Executor.SparkPodSpec.Cores = wrapperspb.Int32(*spec.Executor.Cores) + } + if spec.Executor.Memory != nil { + protoSpec.Executor.SparkPodSpec.Memory = *spec.Executor.Memory + } + } + } + + return protoSpec, nil +} + +// convertApplicationType converts v1beta2.SparkApplicationType to protobuf SparkApplicationType. +func (g *GRPCSparkSubmitter) convertApplicationType(appType v1beta2.SparkApplicationType) proto.SparkApplicationType { + switch appType { + case v1beta2.SparkApplicationTypeJava: + return proto.SparkApplicationType_SPARK_APPLICATION_TYPE_JAVA + case v1beta2.SparkApplicationTypeScala: + return proto.SparkApplicationType_SPARK_APPLICATION_TYPE_SCALA + case v1beta2.SparkApplicationTypePython: + return proto.SparkApplicationType_SPARK_APPLICATION_TYPE_PYTHON + case v1beta2.SparkApplicationTypeR: + return proto.SparkApplicationType_SPARK_APPLICATION_TYPE_R + default: + return proto.SparkApplicationType_SPARK_APPLICATION_TYPE_UNSPECIFIED + } +} + +// convertDeployMode converts v1beta2.DeployMode to protobuf DeployMode. +func (g *GRPCSparkSubmitter) convertDeployMode(mode v1beta2.DeployMode) proto.DeployMode { + switch mode { + case v1beta2.DeployModeCluster: + return proto.DeployMode_DEPLOY_MODE_CLUSTER + case v1beta2.DeployModeClient: + return proto.DeployMode_DEPLOY_MODE_CLIENT + case v1beta2.DeployModeInClusterClient: + return proto.DeployMode_DEPLOY_MODE_IN_CLUSTER_CLIENT + default: + return proto.DeployMode_DEPLOY_MODE_UNSPECIFIED + } +} diff --git a/internal/controller/sparkapplication/grpc_submitter_test.go b/internal/controller/sparkapplication/grpc_submitter_test.go new file mode 100644 index 000000000..53592a512 --- /dev/null +++ b/internal/controller/sparkapplication/grpc_submitter_test.go @@ -0,0 +1,90 @@ +package sparkapplication + +import ( + "context" + "net" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "google.golang.org/grpc" + "google.golang.org/grpc/test/bufconn" + + "github.com/kubeflow/spark-operator/v2/api/v1beta2" + proto "github.com/kubeflow/spark-operator/v2/proto" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" +) + +const bufSize = 1024 * 1024 + +// mockServer implements the proto.SparkSubmitServiceServer interface +// and records the last request received. +type mockServer struct { + proto.UnimplementedSparkSubmitServiceServer + lastRequest *proto.RunAltSparkSubmitRequest + response *proto.RunAltSparkSubmitResponse +} + +func (m *mockServer) RunAltSparkSubmit(ctx context.Context, req *proto.RunAltSparkSubmitRequest) (*proto.RunAltSparkSubmitResponse, error) { + m.lastRequest = req + return m.response, nil +} + +func startMockGRPCServer(t *testing.T, response *proto.RunAltSparkSubmitResponse) (*grpc.ClientConn, *mockServer, func()) { + listener := bufconn.Listen(bufSize) + server := grpc.NewServer() + mock := &mockServer{response: response} + proto.RegisterSparkSubmitServiceServer(server, mock) + go func() { + _ = server.Serve(listener) + }() + dialer := func(context.Context, string) (net.Conn, error) { + return listener.Dial() + } + conn, err := grpc.DialContext(context.Background(), "bufnet", grpc.WithContextDialer(dialer), grpc.WithInsecure()) + if err != nil { + t.Fatalf("Failed to dial bufnet: %v", err) + } + cleanup := func() { + server.Stop() + conn.Close() + } + return conn, mock, cleanup +} + +func TestGRPCSparkSubmitter_Submit_Success(t *testing.T) { + conn, mock, cleanup := startMockGRPCServer(t, &proto.RunAltSparkSubmitResponse{Success: true}) + defer cleanup() + + submitter := &GRPCSparkSubmitter{ + serverAddress: "bufnet:12345", // Add a port to make it valid + timeout: 2 * time.Second, + } + + // Patch grpc.DialContext to use our bufconn + oldDialContext := grpcDialContext + grpcDialContext = func(ctx context.Context, target string, opts ...grpc.DialOption) (*grpc.ClientConn, error) { + return conn, nil + } + defer func() { grpcDialContext = oldDialContext }() + + app := &v1beta2.SparkApplication{ + ObjectMeta: metav1.ObjectMeta{ + Name: "test-app", + Namespace: "default", + }, + Spec: v1beta2.SparkApplicationSpec{ + Type: v1beta2.SparkApplicationTypeJava, + Mode: v1beta2.DeployModeCluster, + }, + Status: v1beta2.SparkApplicationStatus{ + SubmissionID: "sub-123", + }, + } + + err := submitter.Submit(context.Background(), app) + assert.NoError(t, err) + assert.NotNil(t, mock.lastRequest) + assert.Equal(t, "sub-123", mock.lastRequest.SubmissionId) + assert.Equal(t, "test-app", mock.lastRequest.SparkApplication.Metadata.Name) +} diff --git a/proto/spark_submit.pb.go b/proto/spark_submit.pb.go new file mode 100644 index 000000000..29495cf08 --- /dev/null +++ b/proto/spark_submit.pb.go @@ -0,0 +1,9295 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.6 +// protoc v5.29.3 +// source: proto/spark_submit.proto + +package spark + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" + wrapperspb "google.golang.org/protobuf/types/known/wrapperspb" + reflect "reflect" + sync "sync" + unsafe "unsafe" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type ManagedFieldsOperationType int32 + +const ( + ManagedFieldsOperationType_MANAGED_FIELDS_OPERATION_TYPE_UNSPECIFIED ManagedFieldsOperationType = 0 + ManagedFieldsOperationType_MANAGED_FIELDS_OPERATION_TYPE_APPLY ManagedFieldsOperationType = 1 + ManagedFieldsOperationType_MANAGED_FIELDS_OPERATION_TYPE_UPDATE ManagedFieldsOperationType = 2 +) + +// Enum value maps for ManagedFieldsOperationType. +var ( + ManagedFieldsOperationType_name = map[int32]string{ + 0: "MANAGED_FIELDS_OPERATION_TYPE_UNSPECIFIED", + 1: "MANAGED_FIELDS_OPERATION_TYPE_APPLY", + 2: "MANAGED_FIELDS_OPERATION_TYPE_UPDATE", + } + ManagedFieldsOperationType_value = map[string]int32{ + "MANAGED_FIELDS_OPERATION_TYPE_UNSPECIFIED": 0, + "MANAGED_FIELDS_OPERATION_TYPE_APPLY": 1, + "MANAGED_FIELDS_OPERATION_TYPE_UPDATE": 2, + } +) + +func (x ManagedFieldsOperationType) Enum() *ManagedFieldsOperationType { + p := new(ManagedFieldsOperationType) + *p = x + return p +} + +func (x ManagedFieldsOperationType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ManagedFieldsOperationType) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[0].Descriptor() +} + +func (ManagedFieldsOperationType) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[0] +} + +func (x ManagedFieldsOperationType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ManagedFieldsOperationType.Descriptor instead. +func (ManagedFieldsOperationType) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{0} +} + +type SparkApplicationType int32 + +const ( + SparkApplicationType_SPARK_APPLICATION_TYPE_UNSPECIFIED SparkApplicationType = 0 + SparkApplicationType_SPARK_APPLICATION_TYPE_JAVA SparkApplicationType = 1 + SparkApplicationType_SPARK_APPLICATION_TYPE_SCALA SparkApplicationType = 2 + SparkApplicationType_SPARK_APPLICATION_TYPE_PYTHON SparkApplicationType = 3 + SparkApplicationType_SPARK_APPLICATION_TYPE_R SparkApplicationType = 4 +) + +// Enum value maps for SparkApplicationType. +var ( + SparkApplicationType_name = map[int32]string{ + 0: "SPARK_APPLICATION_TYPE_UNSPECIFIED", + 1: "SPARK_APPLICATION_TYPE_JAVA", + 2: "SPARK_APPLICATION_TYPE_SCALA", + 3: "SPARK_APPLICATION_TYPE_PYTHON", + 4: "SPARK_APPLICATION_TYPE_R", + } + SparkApplicationType_value = map[string]int32{ + "SPARK_APPLICATION_TYPE_UNSPECIFIED": 0, + "SPARK_APPLICATION_TYPE_JAVA": 1, + "SPARK_APPLICATION_TYPE_SCALA": 2, + "SPARK_APPLICATION_TYPE_PYTHON": 3, + "SPARK_APPLICATION_TYPE_R": 4, + } +) + +func (x SparkApplicationType) Enum() *SparkApplicationType { + p := new(SparkApplicationType) + *p = x + return p +} + +func (x SparkApplicationType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SparkApplicationType) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[1].Descriptor() +} + +func (SparkApplicationType) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[1] +} + +func (x SparkApplicationType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SparkApplicationType.Descriptor instead. +func (SparkApplicationType) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{1} +} + +type DeployMode int32 + +const ( + DeployMode_DEPLOY_MODE_UNSPECIFIED DeployMode = 0 + DeployMode_DEPLOY_MODE_CLUSTER DeployMode = 1 + DeployMode_DEPLOY_MODE_CLIENT DeployMode = 2 + DeployMode_DEPLOY_MODE_IN_CLUSTER_CLIENT DeployMode = 3 +) + +// Enum value maps for DeployMode. +var ( + DeployMode_name = map[int32]string{ + 0: "DEPLOY_MODE_UNSPECIFIED", + 1: "DEPLOY_MODE_CLUSTER", + 2: "DEPLOY_MODE_CLIENT", + 3: "DEPLOY_MODE_IN_CLUSTER_CLIENT", + } + DeployMode_value = map[string]int32{ + "DEPLOY_MODE_UNSPECIFIED": 0, + "DEPLOY_MODE_CLUSTER": 1, + "DEPLOY_MODE_CLIENT": 2, + "DEPLOY_MODE_IN_CLUSTER_CLIENT": 3, + } +) + +func (x DeployMode) Enum() *DeployMode { + p := new(DeployMode) + *p = x + return p +} + +func (x DeployMode) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (DeployMode) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[2].Descriptor() +} + +func (DeployMode) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[2] +} + +func (x DeployMode) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use DeployMode.Descriptor instead. +func (DeployMode) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{2} +} + +type ServiceType int32 + +const ( + ServiceType_SERVICE_TYPE_UNSPECIFIED ServiceType = 0 + ServiceType_SERVICE_TYPE_CLUSTER_IP ServiceType = 1 + ServiceType_SERVICE_TYPE_NODE_PORT ServiceType = 2 + ServiceType_SERVICE_TYPE_LOAD_BALANCER ServiceType = 3 + ServiceType_SERVICE_TYPE_EXTERNAL_NAME ServiceType = 4 +) + +// Enum value maps for ServiceType. +var ( + ServiceType_name = map[int32]string{ + 0: "SERVICE_TYPE_UNSPECIFIED", + 1: "SERVICE_TYPE_CLUSTER_IP", + 2: "SERVICE_TYPE_NODE_PORT", + 3: "SERVICE_TYPE_LOAD_BALANCER", + 4: "SERVICE_TYPE_EXTERNAL_NAME", + } + ServiceType_value = map[string]int32{ + "SERVICE_TYPE_UNSPECIFIED": 0, + "SERVICE_TYPE_CLUSTER_IP": 1, + "SERVICE_TYPE_NODE_PORT": 2, + "SERVICE_TYPE_LOAD_BALANCER": 3, + "SERVICE_TYPE_EXTERNAL_NAME": 4, + } +) + +func (x ServiceType) Enum() *ServiceType { + p := new(ServiceType) + *p = x + return p +} + +func (x ServiceType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ServiceType) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[3].Descriptor() +} + +func (ServiceType) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[3] +} + +func (x ServiceType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ServiceType.Descriptor instead. +func (ServiceType) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{3} +} + +type DNSPolicy int32 + +const ( + DNSPolicy_DNS_POLICY_UNSPECIFIED DNSPolicy = 0 + DNSPolicy_DNS_POLICY_CLUSTER_FIRST_WITH_HOST_NET DNSPolicy = 1 + DNSPolicy_DNS_POLICY_CLUSTER_FIRST DNSPolicy = 2 + DNSPolicy_DNS_POLICY_DEFAULT DNSPolicy = 3 + DNSPolicy_DNS_POLICY_NONE DNSPolicy = 4 +) + +// Enum value maps for DNSPolicy. +var ( + DNSPolicy_name = map[int32]string{ + 0: "DNS_POLICY_UNSPECIFIED", + 1: "DNS_POLICY_CLUSTER_FIRST_WITH_HOST_NET", + 2: "DNS_POLICY_CLUSTER_FIRST", + 3: "DNS_POLICY_DEFAULT", + 4: "DNS_POLICY_NONE", + } + DNSPolicy_value = map[string]int32{ + "DNS_POLICY_UNSPECIFIED": 0, + "DNS_POLICY_CLUSTER_FIRST_WITH_HOST_NET": 1, + "DNS_POLICY_CLUSTER_FIRST": 2, + "DNS_POLICY_DEFAULT": 3, + "DNS_POLICY_NONE": 4, + } +) + +func (x DNSPolicy) Enum() *DNSPolicy { + p := new(DNSPolicy) + *p = x + return p +} + +func (x DNSPolicy) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (DNSPolicy) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[4].Descriptor() +} + +func (DNSPolicy) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[4] +} + +func (x DNSPolicy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use DNSPolicy.Descriptor instead. +func (DNSPolicy) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{4} +} + +type PodConditionType int32 + +const ( + PodConditionType_POD_CONDITION_TYPE_UNSPECIFIED PodConditionType = 0 + PodConditionType_POD_CONDITION_TYPE_CONTAINERS_READY PodConditionType = 1 + PodConditionType_POD_CONDITION_TYPE_INITIALIZED PodConditionType = 2 + PodConditionType_POD_CONDITION_TYPE_READY PodConditionType = 3 + PodConditionType_POD_CONDITION_TYPE_POD_SCHEDULED PodConditionType = 4 + PodConditionType_POD_CONDITION_TYPE_DISRUPTION_TARGET PodConditionType = 5 + PodConditionType_POD_CONDITION_TYPE_POD_READY_TO_START_CONTAINERS PodConditionType = 6 +) + +// Enum value maps for PodConditionType. +var ( + PodConditionType_name = map[int32]string{ + 0: "POD_CONDITION_TYPE_UNSPECIFIED", + 1: "POD_CONDITION_TYPE_CONTAINERS_READY", + 2: "POD_CONDITION_TYPE_INITIALIZED", + 3: "POD_CONDITION_TYPE_READY", + 4: "POD_CONDITION_TYPE_POD_SCHEDULED", + 5: "POD_CONDITION_TYPE_DISRUPTION_TARGET", + 6: "POD_CONDITION_TYPE_POD_READY_TO_START_CONTAINERS", + } + PodConditionType_value = map[string]int32{ + "POD_CONDITION_TYPE_UNSPECIFIED": 0, + "POD_CONDITION_TYPE_CONTAINERS_READY": 1, + "POD_CONDITION_TYPE_INITIALIZED": 2, + "POD_CONDITION_TYPE_READY": 3, + "POD_CONDITION_TYPE_POD_SCHEDULED": 4, + "POD_CONDITION_TYPE_DISRUPTION_TARGET": 5, + "POD_CONDITION_TYPE_POD_READY_TO_START_CONTAINERS": 6, + } +) + +func (x PodConditionType) Enum() *PodConditionType { + p := new(PodConditionType) + *p = x + return p +} + +func (x PodConditionType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (PodConditionType) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[5].Descriptor() +} + +func (PodConditionType) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[5] +} + +func (x PodConditionType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use PodConditionType.Descriptor instead. +func (PodConditionType) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{5} +} + +type UnsatisfiableConstraintAction int32 + +const ( + UnsatisfiableConstraintAction_UNSATISFIABLE_CONSTRAINT_ACTION_UNSPECIFIED UnsatisfiableConstraintAction = 0 + UnsatisfiableConstraintAction_UNSATISFIABLE_CONSTRAINT_ACTION_DO_NOT_SCHEDULE UnsatisfiableConstraintAction = 1 + UnsatisfiableConstraintAction_UNSATISFIABLE_CONSTRAINT_ACTION_SCHEDULE_ANYWAY UnsatisfiableConstraintAction = 2 +) + +// Enum value maps for UnsatisfiableConstraintAction. +var ( + UnsatisfiableConstraintAction_name = map[int32]string{ + 0: "UNSATISFIABLE_CONSTRAINT_ACTION_UNSPECIFIED", + 1: "UNSATISFIABLE_CONSTRAINT_ACTION_DO_NOT_SCHEDULE", + 2: "UNSATISFIABLE_CONSTRAINT_ACTION_SCHEDULE_ANYWAY", + } + UnsatisfiableConstraintAction_value = map[string]int32{ + "UNSATISFIABLE_CONSTRAINT_ACTION_UNSPECIFIED": 0, + "UNSATISFIABLE_CONSTRAINT_ACTION_DO_NOT_SCHEDULE": 1, + "UNSATISFIABLE_CONSTRAINT_ACTION_SCHEDULE_ANYWAY": 2, + } +) + +func (x UnsatisfiableConstraintAction) Enum() *UnsatisfiableConstraintAction { + p := new(UnsatisfiableConstraintAction) + *p = x + return p +} + +func (x UnsatisfiableConstraintAction) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (UnsatisfiableConstraintAction) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[6].Descriptor() +} + +func (UnsatisfiableConstraintAction) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[6] +} + +func (x UnsatisfiableConstraintAction) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use UnsatisfiableConstraintAction.Descriptor instead. +func (UnsatisfiableConstraintAction) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{6} +} + +type NodeInclusionPolicy int32 + +const ( + NodeInclusionPolicy_NODE_INCLUSION_POLICY_UNSPECIFIED NodeInclusionPolicy = 0 + NodeInclusionPolicy_NODE_INCLUSION_POLCIY_IGNORE NodeInclusionPolicy = 1 + NodeInclusionPolicy_NODE_INCLUSION_POLCIY_HONOR NodeInclusionPolicy = 2 +) + +// Enum value maps for NodeInclusionPolicy. +var ( + NodeInclusionPolicy_name = map[int32]string{ + 0: "NODE_INCLUSION_POLICY_UNSPECIFIED", + 1: "NODE_INCLUSION_POLCIY_IGNORE", + 2: "NODE_INCLUSION_POLCIY_HONOR", + } + NodeInclusionPolicy_value = map[string]int32{ + "NODE_INCLUSION_POLICY_UNSPECIFIED": 0, + "NODE_INCLUSION_POLCIY_IGNORE": 1, + "NODE_INCLUSION_POLCIY_HONOR": 2, + } +) + +func (x NodeInclusionPolicy) Enum() *NodeInclusionPolicy { + p := new(NodeInclusionPolicy) + *p = x + return p +} + +func (x NodeInclusionPolicy) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (NodeInclusionPolicy) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[7].Descriptor() +} + +func (NodeInclusionPolicy) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[7] +} + +func (x NodeInclusionPolicy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use NodeInclusionPolicy.Descriptor instead. +func (NodeInclusionPolicy) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{7} +} + +type SecretType int32 + +const ( + SecretType_SECRET_TYPE_UNSPECIFIED SecretType = 0 + SecretType_SECRET_TYPE_GCP_SERVICE_ACCOUNT SecretType = 1 + SecretType_SECRET_TYPE_HADOOP_DELEGATION_TOKEN SecretType = 2 + SecretType_SECRET_TYPE_GENERIC SecretType = 3 +) + +// Enum value maps for SecretType. +var ( + SecretType_name = map[int32]string{ + 0: "SECRET_TYPE_UNSPECIFIED", + 1: "SECRET_TYPE_GCP_SERVICE_ACCOUNT", + 2: "SECRET_TYPE_HADOOP_DELEGATION_TOKEN", + 3: "SECRET_TYPE_GENERIC", + } + SecretType_value = map[string]int32{ + "SECRET_TYPE_UNSPECIFIED": 0, + "SECRET_TYPE_GCP_SERVICE_ACCOUNT": 1, + "SECRET_TYPE_HADOOP_DELEGATION_TOKEN": 2, + "SECRET_TYPE_GENERIC": 3, + } +) + +func (x SecretType) Enum() *SecretType { + p := new(SecretType) + *p = x + return p +} + +func (x SecretType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SecretType) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[8].Descriptor() +} + +func (SecretType) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[8] +} + +func (x SecretType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SecretType.Descriptor instead. +func (SecretType) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{8} +} + +type LabelSelectorOperator int32 + +const ( + LabelSelectorOperator_LABEL_SELECTOR_OPERATOR_UNSPECIFIED LabelSelectorOperator = 0 + LabelSelectorOperator_LABEL_SELECTOR_OPERATOR_IN LabelSelectorOperator = 1 + LabelSelectorOperator_LABEL_SELECTOR_OPERATOR_NOT_IN LabelSelectorOperator = 2 + LabelSelectorOperator_LABEL_SELECTOR_OPERATOR_EXISTS LabelSelectorOperator = 3 + LabelSelectorOperator_LABEL_SELECTOR_OPERATOR_DOES_NOT_EXIST LabelSelectorOperator = 4 +) + +// Enum value maps for LabelSelectorOperator. +var ( + LabelSelectorOperator_name = map[int32]string{ + 0: "LABEL_SELECTOR_OPERATOR_UNSPECIFIED", + 1: "LABEL_SELECTOR_OPERATOR_IN", + 2: "LABEL_SELECTOR_OPERATOR_NOT_IN", + 3: "LABEL_SELECTOR_OPERATOR_EXISTS", + 4: "LABEL_SELECTOR_OPERATOR_DOES_NOT_EXIST", + } + LabelSelectorOperator_value = map[string]int32{ + "LABEL_SELECTOR_OPERATOR_UNSPECIFIED": 0, + "LABEL_SELECTOR_OPERATOR_IN": 1, + "LABEL_SELECTOR_OPERATOR_NOT_IN": 2, + "LABEL_SELECTOR_OPERATOR_EXISTS": 3, + "LABEL_SELECTOR_OPERATOR_DOES_NOT_EXIST": 4, + } +) + +func (x LabelSelectorOperator) Enum() *LabelSelectorOperator { + p := new(LabelSelectorOperator) + *p = x + return p +} + +func (x LabelSelectorOperator) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (LabelSelectorOperator) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[9].Descriptor() +} + +func (LabelSelectorOperator) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[9] +} + +func (x LabelSelectorOperator) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use LabelSelectorOperator.Descriptor instead. +func (LabelSelectorOperator) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{9} +} + +type NodeSelectorOperator int32 + +const ( + NodeSelectorOperator_NODE_SELECTOR_OPERATOR_UNSPECIFIED NodeSelectorOperator = 0 + NodeSelectorOperator_NODE_SELECTOR_OPERATOR_IN NodeSelectorOperator = 1 + NodeSelectorOperator_NODE_SELECTOR_OPERATOR_NOT_IN NodeSelectorOperator = 2 + NodeSelectorOperator_NODE_SELECTOR_OPERATOR_EXISTS NodeSelectorOperator = 3 + NodeSelectorOperator_NODE_SELECTOR_OPERATOR_DOES_NOT_EXIST NodeSelectorOperator = 4 + NodeSelectorOperator_NODE_SELECTOR_OPERATOR_GT NodeSelectorOperator = 5 + NodeSelectorOperator_NODE_SELECTOR_OPERATOR_LT NodeSelectorOperator = 6 +) + +// Enum value maps for NodeSelectorOperator. +var ( + NodeSelectorOperator_name = map[int32]string{ + 0: "NODE_SELECTOR_OPERATOR_UNSPECIFIED", + 1: "NODE_SELECTOR_OPERATOR_IN", + 2: "NODE_SELECTOR_OPERATOR_NOT_IN", + 3: "NODE_SELECTOR_OPERATOR_EXISTS", + 4: "NODE_SELECTOR_OPERATOR_DOES_NOT_EXIST", + 5: "NODE_SELECTOR_OPERATOR_GT", + 6: "NODE_SELECTOR_OPERATOR_LT", + } + NodeSelectorOperator_value = map[string]int32{ + "NODE_SELECTOR_OPERATOR_UNSPECIFIED": 0, + "NODE_SELECTOR_OPERATOR_IN": 1, + "NODE_SELECTOR_OPERATOR_NOT_IN": 2, + "NODE_SELECTOR_OPERATOR_EXISTS": 3, + "NODE_SELECTOR_OPERATOR_DOES_NOT_EXIST": 4, + "NODE_SELECTOR_OPERATOR_GT": 5, + "NODE_SELECTOR_OPERATOR_LT": 6, + } +) + +func (x NodeSelectorOperator) Enum() *NodeSelectorOperator { + p := new(NodeSelectorOperator) + *p = x + return p +} + +func (x NodeSelectorOperator) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (NodeSelectorOperator) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[10].Descriptor() +} + +func (NodeSelectorOperator) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[10] +} + +func (x NodeSelectorOperator) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use NodeSelectorOperator.Descriptor instead. +func (NodeSelectorOperator) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{10} +} + +type TaintEffect int32 + +const ( + TaintEffect_TAINT_EFFECT_UNSPECIFIED TaintEffect = 0 + TaintEffect_TAINT_EFFECT_NO_SCHEDULE TaintEffect = 1 + TaintEffect_TAINT_EFFECT_PREFER_NO_SCHEDULE TaintEffect = 2 + TaintEffect_TAINT_EFFECT_NO_SCHEDULE_NO_ADMIT TaintEffect = 3 + TaintEffect_TAINT_EFFECT_NO_EXECUTE TaintEffect = 4 +) + +// Enum value maps for TaintEffect. +var ( + TaintEffect_name = map[int32]string{ + 0: "TAINT_EFFECT_UNSPECIFIED", + 1: "TAINT_EFFECT_NO_SCHEDULE", + 2: "TAINT_EFFECT_PREFER_NO_SCHEDULE", + 3: "TAINT_EFFECT_NO_SCHEDULE_NO_ADMIT", + 4: "TAINT_EFFECT_NO_EXECUTE", + } + TaintEffect_value = map[string]int32{ + "TAINT_EFFECT_UNSPECIFIED": 0, + "TAINT_EFFECT_NO_SCHEDULE": 1, + "TAINT_EFFECT_PREFER_NO_SCHEDULE": 2, + "TAINT_EFFECT_NO_SCHEDULE_NO_ADMIT": 3, + "TAINT_EFFECT_NO_EXECUTE": 4, + } +) + +func (x TaintEffect) Enum() *TaintEffect { + p := new(TaintEffect) + *p = x + return p +} + +func (x TaintEffect) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (TaintEffect) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[11].Descriptor() +} + +func (TaintEffect) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[11] +} + +func (x TaintEffect) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use TaintEffect.Descriptor instead. +func (TaintEffect) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{11} +} + +type TolerationOperator int32 + +const ( + TolerationOperator_TOLERATION_OPERATOR_UNSPECIFIED TolerationOperator = 0 + TolerationOperator_TOLERATION_OPERATOR_EXISTS TolerationOperator = 1 + TolerationOperator_TOLERATION_OPERATOR_EQUAL TolerationOperator = 2 +) + +// Enum value maps for TolerationOperator. +var ( + TolerationOperator_name = map[int32]string{ + 0: "TOLERATION_OPERATOR_UNSPECIFIED", + 1: "TOLERATION_OPERATOR_EXISTS", + 2: "TOLERATION_OPERATOR_EQUAL", + } + TolerationOperator_value = map[string]int32{ + "TOLERATION_OPERATOR_UNSPECIFIED": 0, + "TOLERATION_OPERATOR_EXISTS": 1, + "TOLERATION_OPERATOR_EQUAL": 2, + } +) + +func (x TolerationOperator) Enum() *TolerationOperator { + p := new(TolerationOperator) + *p = x + return p +} + +func (x TolerationOperator) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (TolerationOperator) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[12].Descriptor() +} + +func (TolerationOperator) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[12] +} + +func (x TolerationOperator) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use TolerationOperator.Descriptor instead. +func (TolerationOperator) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{12} +} + +type PodFSGroupChangePolicy int32 + +const ( + PodFSGroupChangePolicy_POD_FS_GROUP_CHANGE_POLICY_UNSPECIFIED PodFSGroupChangePolicy = 0 + PodFSGroupChangePolicy_POD_FS_GROUP_CHANGE_POLICY_ON_ROOT_MISMATCH PodFSGroupChangePolicy = 1 + PodFSGroupChangePolicy_POD_FS_GROUP_CHANGE_POLICY_ALWAYS PodFSGroupChangePolicy = 2 +) + +// Enum value maps for PodFSGroupChangePolicy. +var ( + PodFSGroupChangePolicy_name = map[int32]string{ + 0: "POD_FS_GROUP_CHANGE_POLICY_UNSPECIFIED", + 1: "POD_FS_GROUP_CHANGE_POLICY_ON_ROOT_MISMATCH", + 2: "POD_FS_GROUP_CHANGE_POLICY_ALWAYS", + } + PodFSGroupChangePolicy_value = map[string]int32{ + "POD_FS_GROUP_CHANGE_POLICY_UNSPECIFIED": 0, + "POD_FS_GROUP_CHANGE_POLICY_ON_ROOT_MISMATCH": 1, + "POD_FS_GROUP_CHANGE_POLICY_ALWAYS": 2, + } +) + +func (x PodFSGroupChangePolicy) Enum() *PodFSGroupChangePolicy { + p := new(PodFSGroupChangePolicy) + *p = x + return p +} + +func (x PodFSGroupChangePolicy) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (PodFSGroupChangePolicy) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[13].Descriptor() +} + +func (PodFSGroupChangePolicy) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[13] +} + +func (x PodFSGroupChangePolicy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use PodFSGroupChangePolicy.Descriptor instead. +func (PodFSGroupChangePolicy) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{13} +} + +type Protocol int32 + +const ( + Protocol_PROTOCOL_UNSPECIFIED Protocol = 0 + Protocol_PROTOCOL_TCP Protocol = 1 + Protocol_PROTOCOL_UDP Protocol = 2 + Protocol_PROTOCOL_SCTP Protocol = 3 +) + +// Enum value maps for Protocol. +var ( + Protocol_name = map[int32]string{ + 0: "PROTOCOL_UNSPECIFIED", + 1: "PROTOCOL_TCP", + 2: "PROTOCOL_UDP", + 3: "PROTOCOL_SCTP", + } + Protocol_value = map[string]int32{ + "PROTOCOL_UNSPECIFIED": 0, + "PROTOCOL_TCP": 1, + "PROTOCOL_UDP": 2, + "PROTOCOL_SCTP": 3, + } +) + +func (x Protocol) Enum() *Protocol { + p := new(Protocol) + *p = x + return p +} + +func (x Protocol) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (Protocol) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[14].Descriptor() +} + +func (Protocol) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[14] +} + +func (x Protocol) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use Protocol.Descriptor instead. +func (Protocol) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{14} +} + +type Format int32 + +const ( + Format_FORMAT_UNSPECIFIED Format = 0 + Format_DECIMAL_EXPONENT Format = 1 // e.g., 12e6 + Format_BINARY_SI Format = 2 // e.g., 12Mi (12 * 2^20) + Format_DECIMAL_SI Format = 3 // e.g., 12M (12 * 10^6) +) + +// Enum value maps for Format. +var ( + Format_name = map[int32]string{ + 0: "FORMAT_UNSPECIFIED", + 1: "DECIMAL_EXPONENT", + 2: "BINARY_SI", + 3: "DECIMAL_SI", + } + Format_value = map[string]int32{ + "FORMAT_UNSPECIFIED": 0, + "DECIMAL_EXPONENT": 1, + "BINARY_SI": 2, + "DECIMAL_SI": 3, + } +) + +func (x Format) Enum() *Format { + p := new(Format) + *p = x + return p +} + +func (x Format) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (Format) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[15].Descriptor() +} + +func (Format) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[15] +} + +func (x Format) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use Format.Descriptor instead. +func (Format) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{15} +} + +type ResourceResizeRestartPolicy int32 + +const ( + ResourceResizeRestartPolicy_RESOURCE_RESIZE_RESTART_POLICY_UNSPECIFIED ResourceResizeRestartPolicy = 0 + ResourceResizeRestartPolicy_RESOURCE_RESIZE_RESTART_POLICY_NOT_REQUIRED ResourceResizeRestartPolicy = 1 + ResourceResizeRestartPolicy_RESOURCE_RESIZE_RESTART_POLICY_RESTART_CONTAINER ResourceResizeRestartPolicy = 2 +) + +// Enum value maps for ResourceResizeRestartPolicy. +var ( + ResourceResizeRestartPolicy_name = map[int32]string{ + 0: "RESOURCE_RESIZE_RESTART_POLICY_UNSPECIFIED", + 1: "RESOURCE_RESIZE_RESTART_POLICY_NOT_REQUIRED", + 2: "RESOURCE_RESIZE_RESTART_POLICY_RESTART_CONTAINER", + } + ResourceResizeRestartPolicy_value = map[string]int32{ + "RESOURCE_RESIZE_RESTART_POLICY_UNSPECIFIED": 0, + "RESOURCE_RESIZE_RESTART_POLICY_NOT_REQUIRED": 1, + "RESOURCE_RESIZE_RESTART_POLICY_RESTART_CONTAINER": 2, + } +) + +func (x ResourceResizeRestartPolicy) Enum() *ResourceResizeRestartPolicy { + p := new(ResourceResizeRestartPolicy) + *p = x + return p +} + +func (x ResourceResizeRestartPolicy) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ResourceResizeRestartPolicy) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[16].Descriptor() +} + +func (ResourceResizeRestartPolicy) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[16] +} + +func (x ResourceResizeRestartPolicy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ResourceResizeRestartPolicy.Descriptor instead. +func (ResourceResizeRestartPolicy) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{16} +} + +type ContainerRestartPolicy int32 + +const ( + ContainerRestartPolicy_CONTAINER_RESTART_POLICY_UNSPECIFIED ContainerRestartPolicy = 0 + ContainerRestartPolicy_CONTAINER_RESTART_POLICY_ALWAYS ContainerRestartPolicy = 1 +) + +// Enum value maps for ContainerRestartPolicy. +var ( + ContainerRestartPolicy_name = map[int32]string{ + 0: "CONTAINER_RESTART_POLICY_UNSPECIFIED", + 1: "CONTAINER_RESTART_POLICY_ALWAYS", + } + ContainerRestartPolicy_value = map[string]int32{ + "CONTAINER_RESTART_POLICY_UNSPECIFIED": 0, + "CONTAINER_RESTART_POLICY_ALWAYS": 1, + } +) + +func (x ContainerRestartPolicy) Enum() *ContainerRestartPolicy { + p := new(ContainerRestartPolicy) + *p = x + return p +} + +func (x ContainerRestartPolicy) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ContainerRestartPolicy) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[17].Descriptor() +} + +func (ContainerRestartPolicy) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[17] +} + +func (x ContainerRestartPolicy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ContainerRestartPolicy.Descriptor instead. +func (ContainerRestartPolicy) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{17} +} + +type TerminationMessagePolicy int32 + +const ( + TerminationMessagePolicy_TERMINATION_MESSAGE_POLICY_UNSPECIFIED TerminationMessagePolicy = 0 + TerminationMessagePolicy_TERMINATION_MESSAGE_POLICY_FILE TerminationMessagePolicy = 1 + TerminationMessagePolicy_TERMINATION_MESSAGE_FALLBACK_TO_LOGS_ON_ERROR TerminationMessagePolicy = 2 +) + +// Enum value maps for TerminationMessagePolicy. +var ( + TerminationMessagePolicy_name = map[int32]string{ + 0: "TERMINATION_MESSAGE_POLICY_UNSPECIFIED", + 1: "TERMINATION_MESSAGE_POLICY_FILE", + 2: "TERMINATION_MESSAGE_FALLBACK_TO_LOGS_ON_ERROR", + } + TerminationMessagePolicy_value = map[string]int32{ + "TERMINATION_MESSAGE_POLICY_UNSPECIFIED": 0, + "TERMINATION_MESSAGE_POLICY_FILE": 1, + "TERMINATION_MESSAGE_FALLBACK_TO_LOGS_ON_ERROR": 2, + } +) + +func (x TerminationMessagePolicy) Enum() *TerminationMessagePolicy { + p := new(TerminationMessagePolicy) + *p = x + return p +} + +func (x TerminationMessagePolicy) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (TerminationMessagePolicy) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[18].Descriptor() +} + +func (TerminationMessagePolicy) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[18] +} + +func (x TerminationMessagePolicy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use TerminationMessagePolicy.Descriptor instead. +func (TerminationMessagePolicy) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{18} +} + +type PullPolicy int32 + +const ( + PullPolicy_PULL_POLICY_UNSPECIFIED PullPolicy = 0 + PullPolicy_PULL_POLICY_ALWAYS PullPolicy = 1 + PullPolicy_PULL_POLICY_NEVER PullPolicy = 2 + PullPolicy_PULL_POLICY_IF_NOT_PRESENT PullPolicy = 3 +) + +// Enum value maps for PullPolicy. +var ( + PullPolicy_name = map[int32]string{ + 0: "PULL_POLICY_UNSPECIFIED", + 1: "PULL_POLICY_ALWAYS", + 2: "PULL_POLICY_NEVER", + 3: "PULL_POLICY_IF_NOT_PRESENT", + } + PullPolicy_value = map[string]int32{ + "PULL_POLICY_UNSPECIFIED": 0, + "PULL_POLICY_ALWAYS": 1, + "PULL_POLICY_NEVER": 2, + "PULL_POLICY_IF_NOT_PRESENT": 3, + } +) + +func (x PullPolicy) Enum() *PullPolicy { + p := new(PullPolicy) + *p = x + return p +} + +func (x PullPolicy) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (PullPolicy) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[19].Descriptor() +} + +func (PullPolicy) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[19] +} + +func (x PullPolicy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use PullPolicy.Descriptor instead. +func (PullPolicy) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{19} +} + +type ProcMountType int32 + +const ( + ProcMountType_PROC_MOUNT_TYPE_UNSPECIFIED ProcMountType = 0 //default value + ProcMountType_PROC_MOUNT_TYPE_DEFAULT ProcMountType = 1 + ProcMountType_PROC_MOUNT_TYPE_UNMASKED ProcMountType = 2 +) + +// Enum value maps for ProcMountType. +var ( + ProcMountType_name = map[int32]string{ + 0: "PROC_MOUNT_TYPE_UNSPECIFIED", + 1: "PROC_MOUNT_TYPE_DEFAULT", + 2: "PROC_MOUNT_TYPE_UNMASKED", + } + ProcMountType_value = map[string]int32{ + "PROC_MOUNT_TYPE_UNSPECIFIED": 0, + "PROC_MOUNT_TYPE_DEFAULT": 1, + "PROC_MOUNT_TYPE_UNMASKED": 2, + } +) + +func (x ProcMountType) Enum() *ProcMountType { + p := new(ProcMountType) + *p = x + return p +} + +func (x ProcMountType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ProcMountType) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[20].Descriptor() +} + +func (ProcMountType) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[20] +} + +func (x ProcMountType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ProcMountType.Descriptor instead. +func (ProcMountType) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{20} +} + +// Enum for SeccompProfileType +type SeccompProfileType int32 + +const ( + SeccompProfileType_SECCOMP_PROFILE_TYPE_UNSPECIFIED SeccompProfileType = 0 + SeccompProfileType_SECCOMP_PROFILE_TYPE_LOCALHOST SeccompProfileType = 1 + SeccompProfileType_SECCOMP_PROFILE_TYPE_RUNTIME_DEFAULT SeccompProfileType = 2 + SeccompProfileType_SECCOMP_PROFILE_TYPE_UNCONFINED SeccompProfileType = 3 +) + +// Enum value maps for SeccompProfileType. +var ( + SeccompProfileType_name = map[int32]string{ + 0: "SECCOMP_PROFILE_TYPE_UNSPECIFIED", + 1: "SECCOMP_PROFILE_TYPE_LOCALHOST", + 2: "SECCOMP_PROFILE_TYPE_RUNTIME_DEFAULT", + 3: "SECCOMP_PROFILE_TYPE_UNCONFINED", + } + SeccompProfileType_value = map[string]int32{ + "SECCOMP_PROFILE_TYPE_UNSPECIFIED": 0, + "SECCOMP_PROFILE_TYPE_LOCALHOST": 1, + "SECCOMP_PROFILE_TYPE_RUNTIME_DEFAULT": 2, + "SECCOMP_PROFILE_TYPE_UNCONFINED": 3, + } +) + +func (x SeccompProfileType) Enum() *SeccompProfileType { + p := new(SeccompProfileType) + *p = x + return p +} + +func (x SeccompProfileType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (SeccompProfileType) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[21].Descriptor() +} + +func (SeccompProfileType) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[21] +} + +func (x SeccompProfileType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use SeccompProfileType.Descriptor instead. +func (SeccompProfileType) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{21} +} + +// Enum for URI Scheme +type URIScheme int32 + +const ( + URIScheme_URISCHEME_UNSPECIFIED URIScheme = 0 + URIScheme_URISCHEME_HTTP URIScheme = 1 // http:// + URIScheme_URISCHEME_HTTPS URIScheme = 2 // https:// +) + +// Enum value maps for URIScheme. +var ( + URIScheme_name = map[int32]string{ + 0: "URISCHEME_UNSPECIFIED", + 1: "URISCHEME_HTTP", + 2: "URISCHEME_HTTPS", + } + URIScheme_value = map[string]int32{ + "URISCHEME_UNSPECIFIED": 0, + "URISCHEME_HTTP": 1, + "URISCHEME_HTTPS": 2, + } +) + +func (x URIScheme) Enum() *URIScheme { + p := new(URIScheme) + *p = x + return p +} + +func (x URIScheme) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (URIScheme) Descriptor() protoreflect.EnumDescriptor { + return file_proto_spark_submit_proto_enumTypes[22].Descriptor() +} + +func (URIScheme) Type() protoreflect.EnumType { + return &file_proto_spark_submit_proto_enumTypes[22] +} + +func (x URIScheme) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use URIScheme.Descriptor instead. +func (URIScheme) EnumDescriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{22} +} + +// SparkApplicationSpec +type SparkApplicationSpec struct { + state protoimpl.MessageState `protogen:"open.v1"` + Type SparkApplicationType `protobuf:"varint,1,opt,name=type,proto3,enum=spark.SparkApplicationType" json:"type,omitempty"` + Mode DeployMode `protobuf:"varint,2,opt,name=mode,proto3,enum=spark.DeployMode" json:"mode,omitempty"` + Image *wrapperspb.StringValue `protobuf:"bytes,3,opt,name=image,proto3" json:"image,omitempty"` + ImagePullPolicy *wrapperspb.StringValue `protobuf:"bytes,4,opt,name=image_pull_policy,json=imagePullPolicy,proto3" json:"image_pull_policy,omitempty"` + ImagePullSecrets []string `protobuf:"bytes,5,rep,name=image_pull_secrets,json=imagePullSecrets,proto3" json:"image_pull_secrets,omitempty"` + SparkConf map[string]string `protobuf:"bytes,6,rep,name=spark_conf,json=sparkConf,proto3" json:"spark_conf,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + HadoopConf map[string]string `protobuf:"bytes,7,rep,name=hadoop_conf,json=hadoopConf,proto3" json:"hadoop_conf,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + SparkConfigMap *wrapperspb.StringValue `protobuf:"bytes,8,opt,name=spark_config_map,json=sparkConfigMap,proto3" json:"spark_config_map,omitempty"` + HadoopConfigMap *wrapperspb.StringValue `protobuf:"bytes,9,opt,name=hadoop_config_map,json=hadoopConfigMap,proto3" json:"hadoop_config_map,omitempty"` + Arguments []string `protobuf:"bytes,10,rep,name=arguments,proto3" json:"arguments,omitempty"` + MainClass *wrapperspb.StringValue `protobuf:"bytes,11,opt,name=main_class,json=mainClass,proto3" json:"main_class,omitempty"` + MainApplicationFile *wrapperspb.StringValue `protobuf:"bytes,12,opt,name=main_application_file,json=mainApplicationFile,proto3" json:"main_application_file,omitempty"` + ProxyUser *wrapperspb.StringValue `protobuf:"bytes,13,opt,name=proxy_user,json=proxyUser,proto3" json:"proxy_user,omitempty"` + FailureRetries *wrapperspb.Int32Value `protobuf:"bytes,15,opt,name=failure_retries,json=failureRetries,proto3" json:"failure_retries,omitempty"` + RetryInterval *wrapperspb.Int64Value `protobuf:"bytes,16,opt,name=retry_interval,json=retryInterval,proto3" json:"retry_interval,omitempty"` + MemoryOverheadFactor *wrapperspb.StringValue `protobuf:"bytes,17,opt,name=memory_overhead_factor,json=memoryOverheadFactor,proto3" json:"memory_overhead_factor,omitempty"` + Monitoring *MonitoringSpec `protobuf:"bytes,18,opt,name=monitoring,proto3" json:"monitoring,omitempty"` + BatchScheduler *wrapperspb.StringValue `protobuf:"bytes,19,opt,name=batch_scheduler,json=batchScheduler,proto3" json:"batch_scheduler,omitempty"` + TimeToLiveSeconds *wrapperspb.Int64Value `protobuf:"bytes,20,opt,name=time_to_live_seconds,json=timeToLiveSeconds,proto3" json:"time_to_live_seconds,omitempty"` + BatchSchedulerConfiguration *BatchSchedulerConfiguration `protobuf:"bytes,21,opt,name=batch_scheduler_configuration,json=batchSchedulerConfiguration,proto3" json:"batch_scheduler_configuration,omitempty"` + Driver *DriverSpec `protobuf:"bytes,22,opt,name=driver,proto3" json:"driver,omitempty"` + Executor *ExecutorSpec `protobuf:"bytes,23,opt,name=executor,proto3" json:"executor,omitempty"` + Volumes []*Volume `protobuf:"bytes,24,rep,name=volumes,proto3" json:"volumes,omitempty"` + Deps *Dependencies `protobuf:"bytes,25,opt,name=deps,proto3" json:"deps,omitempty"` + DynamicAllocation *DynamicAllocation `protobuf:"bytes,26,opt,name=dynamic_allocation,json=dynamicAllocation,proto3" json:"dynamic_allocation,omitempty"` + PythonVersion string `protobuf:"bytes,27,opt,name=python_version,json=pythonVersion,proto3" json:"python_version,omitempty"` + SparkVersion string `protobuf:"bytes,28,opt,name=spark_version,json=sparkVersion,proto3" json:"spark_version,omitempty"` + RestartPolicy *RestartPolicy `protobuf:"bytes,29,opt,name=restart_policy,json=restartPolicy,proto3" json:"restart_policy,omitempty"` + SparkUiConfiguration *SparkUIConfiguration `protobuf:"bytes,30,opt,name=spark_ui_configuration,json=sparkUiConfiguration,proto3" json:"spark_ui_configuration,omitempty"` + DriverIngressConfiguration []*DriverIngressConfiguration `protobuf:"bytes,31,rep,name=driver_ingress_configuration,json=driverIngressConfiguration,proto3" json:"driver_ingress_configuration,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SparkApplicationSpec) Reset() { + *x = SparkApplicationSpec{} + mi := &file_proto_spark_submit_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SparkApplicationSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SparkApplicationSpec) ProtoMessage() {} + +func (x *SparkApplicationSpec) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SparkApplicationSpec.ProtoReflect.Descriptor instead. +func (*SparkApplicationSpec) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{0} +} + +func (x *SparkApplicationSpec) GetType() SparkApplicationType { + if x != nil { + return x.Type + } + return SparkApplicationType_SPARK_APPLICATION_TYPE_UNSPECIFIED +} + +func (x *SparkApplicationSpec) GetMode() DeployMode { + if x != nil { + return x.Mode + } + return DeployMode_DEPLOY_MODE_UNSPECIFIED +} + +func (x *SparkApplicationSpec) GetImage() *wrapperspb.StringValue { + if x != nil { + return x.Image + } + return nil +} + +func (x *SparkApplicationSpec) GetImagePullPolicy() *wrapperspb.StringValue { + if x != nil { + return x.ImagePullPolicy + } + return nil +} + +func (x *SparkApplicationSpec) GetImagePullSecrets() []string { + if x != nil { + return x.ImagePullSecrets + } + return nil +} + +func (x *SparkApplicationSpec) GetSparkConf() map[string]string { + if x != nil { + return x.SparkConf + } + return nil +} + +func (x *SparkApplicationSpec) GetHadoopConf() map[string]string { + if x != nil { + return x.HadoopConf + } + return nil +} + +func (x *SparkApplicationSpec) GetSparkConfigMap() *wrapperspb.StringValue { + if x != nil { + return x.SparkConfigMap + } + return nil +} + +func (x *SparkApplicationSpec) GetHadoopConfigMap() *wrapperspb.StringValue { + if x != nil { + return x.HadoopConfigMap + } + return nil +} + +func (x *SparkApplicationSpec) GetArguments() []string { + if x != nil { + return x.Arguments + } + return nil +} + +func (x *SparkApplicationSpec) GetMainClass() *wrapperspb.StringValue { + if x != nil { + return x.MainClass + } + return nil +} + +func (x *SparkApplicationSpec) GetMainApplicationFile() *wrapperspb.StringValue { + if x != nil { + return x.MainApplicationFile + } + return nil +} + +func (x *SparkApplicationSpec) GetProxyUser() *wrapperspb.StringValue { + if x != nil { + return x.ProxyUser + } + return nil +} + +func (x *SparkApplicationSpec) GetFailureRetries() *wrapperspb.Int32Value { + if x != nil { + return x.FailureRetries + } + return nil +} + +func (x *SparkApplicationSpec) GetRetryInterval() *wrapperspb.Int64Value { + if x != nil { + return x.RetryInterval + } + return nil +} + +func (x *SparkApplicationSpec) GetMemoryOverheadFactor() *wrapperspb.StringValue { + if x != nil { + return x.MemoryOverheadFactor + } + return nil +} + +func (x *SparkApplicationSpec) GetMonitoring() *MonitoringSpec { + if x != nil { + return x.Monitoring + } + return nil +} + +func (x *SparkApplicationSpec) GetBatchScheduler() *wrapperspb.StringValue { + if x != nil { + return x.BatchScheduler + } + return nil +} + +func (x *SparkApplicationSpec) GetTimeToLiveSeconds() *wrapperspb.Int64Value { + if x != nil { + return x.TimeToLiveSeconds + } + return nil +} + +func (x *SparkApplicationSpec) GetBatchSchedulerConfiguration() *BatchSchedulerConfiguration { + if x != nil { + return x.BatchSchedulerConfiguration + } + return nil +} + +func (x *SparkApplicationSpec) GetDriver() *DriverSpec { + if x != nil { + return x.Driver + } + return nil +} + +func (x *SparkApplicationSpec) GetExecutor() *ExecutorSpec { + if x != nil { + return x.Executor + } + return nil +} + +func (x *SparkApplicationSpec) GetVolumes() []*Volume { + if x != nil { + return x.Volumes + } + return nil +} + +func (x *SparkApplicationSpec) GetDeps() *Dependencies { + if x != nil { + return x.Deps + } + return nil +} + +func (x *SparkApplicationSpec) GetDynamicAllocation() *DynamicAllocation { + if x != nil { + return x.DynamicAllocation + } + return nil +} + +func (x *SparkApplicationSpec) GetPythonVersion() string { + if x != nil { + return x.PythonVersion + } + return "" +} + +func (x *SparkApplicationSpec) GetSparkVersion() string { + if x != nil { + return x.SparkVersion + } + return "" +} + +func (x *SparkApplicationSpec) GetRestartPolicy() *RestartPolicy { + if x != nil { + return x.RestartPolicy + } + return nil +} + +func (x *SparkApplicationSpec) GetSparkUiConfiguration() *SparkUIConfiguration { + if x != nil { + return x.SparkUiConfiguration + } + return nil +} + +func (x *SparkApplicationSpec) GetDriverIngressConfiguration() []*DriverIngressConfiguration { + if x != nil { + return x.DriverIngressConfiguration + } + return nil +} + +// Metadata for Kubernetes objects +type ObjectMeta struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + GenerateName string `protobuf:"bytes,2,opt,name=generate_name,json=generateName,proto3" json:"generate_name,omitempty"` + Namespace string `protobuf:"bytes,3,opt,name=namespace,proto3" json:"namespace,omitempty"` + SelfLink string `protobuf:"bytes,4,opt,name=self_link,json=selfLink,proto3" json:"self_link,omitempty"` + Uid string `protobuf:"bytes,5,opt,name=uid,proto3" json:"uid,omitempty"` + ResourceVersion string `protobuf:"bytes,6,opt,name=resource_version,json=resourceVersion,proto3" json:"resource_version,omitempty"` + Generation int64 `protobuf:"varint,7,opt,name=generation,proto3" json:"generation,omitempty"` + CreationTimestamp *timestamppb.Timestamp `protobuf:"bytes,8,opt,name=creation_timestamp,json=creationTimestamp,proto3" json:"creation_timestamp,omitempty"` + DeletionTimestamp *timestamppb.Timestamp `protobuf:"bytes,9,opt,name=deletion_timestamp,json=deletionTimestamp,proto3" json:"deletion_timestamp,omitempty"` + DeletionGracePeriodSeconds *wrapperspb.Int64Value `protobuf:"bytes,10,opt,name=deletion_grace_period_seconds,json=deletionGracePeriodSeconds,proto3" json:"deletion_grace_period_seconds,omitempty"` + Labels map[string]string `protobuf:"bytes,11,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + Annotations map[string]string `protobuf:"bytes,12,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + OwnerReferences []*OwnerReference `protobuf:"bytes,13,rep,name=owner_references,json=ownerReferences,proto3" json:"owner_references,omitempty"` + Finalizers []string `protobuf:"bytes,14,rep,name=finalizers,proto3" json:"finalizers,omitempty"` + ManagedFields []*ManagedFieldsEntry `protobuf:"bytes,15,rep,name=managed_fields,json=managedFields,proto3" json:"managed_fields,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ObjectMeta) Reset() { + *x = ObjectMeta{} + mi := &file_proto_spark_submit_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ObjectMeta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ObjectMeta) ProtoMessage() {} + +func (x *ObjectMeta) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ObjectMeta.ProtoReflect.Descriptor instead. +func (*ObjectMeta) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{1} +} + +func (x *ObjectMeta) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *ObjectMeta) GetGenerateName() string { + if x != nil { + return x.GenerateName + } + return "" +} + +func (x *ObjectMeta) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *ObjectMeta) GetSelfLink() string { + if x != nil { + return x.SelfLink + } + return "" +} + +func (x *ObjectMeta) GetUid() string { + if x != nil { + return x.Uid + } + return "" +} + +func (x *ObjectMeta) GetResourceVersion() string { + if x != nil { + return x.ResourceVersion + } + return "" +} + +func (x *ObjectMeta) GetGeneration() int64 { + if x != nil { + return x.Generation + } + return 0 +} + +func (x *ObjectMeta) GetCreationTimestamp() *timestamppb.Timestamp { + if x != nil { + return x.CreationTimestamp + } + return nil +} + +func (x *ObjectMeta) GetDeletionTimestamp() *timestamppb.Timestamp { + if x != nil { + return x.DeletionTimestamp + } + return nil +} + +func (x *ObjectMeta) GetDeletionGracePeriodSeconds() *wrapperspb.Int64Value { + if x != nil { + return x.DeletionGracePeriodSeconds + } + return nil +} + +func (x *ObjectMeta) GetLabels() map[string]string { + if x != nil { + return x.Labels + } + return nil +} + +func (x *ObjectMeta) GetAnnotations() map[string]string { + if x != nil { + return x.Annotations + } + return nil +} + +func (x *ObjectMeta) GetOwnerReferences() []*OwnerReference { + if x != nil { + return x.OwnerReferences + } + return nil +} + +func (x *ObjectMeta) GetFinalizers() []string { + if x != nil { + return x.Finalizers + } + return nil +} + +func (x *ObjectMeta) GetManagedFields() []*ManagedFieldsEntry { + if x != nil { + return x.ManagedFields + } + return nil +} + +type FieldsV1 struct { + state protoimpl.MessageState `protogen:"open.v1"` + Raw string `protobuf:"bytes,1,opt,name=raw,proto3" json:"raw,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *FieldsV1) Reset() { + *x = FieldsV1{} + mi := &file_proto_spark_submit_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *FieldsV1) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FieldsV1) ProtoMessage() {} + +func (x *FieldsV1) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FieldsV1.ProtoReflect.Descriptor instead. +func (*FieldsV1) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{2} +} + +func (x *FieldsV1) GetRaw() string { + if x != nil { + return x.Raw + } + return "" +} + +type ManagedFieldsEntry struct { + state protoimpl.MessageState `protogen:"open.v1"` + Manager string `protobuf:"bytes,1,opt,name=manager,proto3" json:"manager,omitempty"` + Operation ManagedFieldsOperationType `protobuf:"varint,2,opt,name=operation,proto3,enum=spark.ManagedFieldsOperationType" json:"operation,omitempty"` + ApiVersion string `protobuf:"bytes,3,opt,name=api_version,json=apiVersion,proto3" json:"api_version,omitempty"` + MyTime *timestamppb.Timestamp `protobuf:"bytes,4,opt,name=my_time,json=myTime,proto3" json:"my_time,omitempty"` + FieldsType string `protobuf:"bytes,5,opt,name=fields_type,json=fieldsType,proto3" json:"fields_type,omitempty"` + FieldsV1 *FieldsV1 `protobuf:"bytes,6,opt,name=fields_v1,json=fieldsV1,proto3" json:"fields_v1,omitempty"` + SubResource string `protobuf:"bytes,7,opt,name=sub_resource,json=subResource,proto3" json:"sub_resource,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ManagedFieldsEntry) Reset() { + *x = ManagedFieldsEntry{} + mi := &file_proto_spark_submit_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ManagedFieldsEntry) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ManagedFieldsEntry) ProtoMessage() {} + +func (x *ManagedFieldsEntry) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ManagedFieldsEntry.ProtoReflect.Descriptor instead. +func (*ManagedFieldsEntry) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{3} +} + +func (x *ManagedFieldsEntry) GetManager() string { + if x != nil { + return x.Manager + } + return "" +} + +func (x *ManagedFieldsEntry) GetOperation() ManagedFieldsOperationType { + if x != nil { + return x.Operation + } + return ManagedFieldsOperationType_MANAGED_FIELDS_OPERATION_TYPE_UNSPECIFIED +} + +func (x *ManagedFieldsEntry) GetApiVersion() string { + if x != nil { + return x.ApiVersion + } + return "" +} + +func (x *ManagedFieldsEntry) GetMyTime() *timestamppb.Timestamp { + if x != nil { + return x.MyTime + } + return nil +} + +func (x *ManagedFieldsEntry) GetFieldsType() string { + if x != nil { + return x.FieldsType + } + return "" +} + +func (x *ManagedFieldsEntry) GetFieldsV1() *FieldsV1 { + if x != nil { + return x.FieldsV1 + } + return nil +} + +func (x *ManagedFieldsEntry) GetSubResource() string { + if x != nil { + return x.SubResource + } + return "" +} + +type OwnerReference struct { + state protoimpl.MessageState `protogen:"open.v1"` + ApiVersion string `protobuf:"bytes,1,opt,name=api_version,json=apiVersion,proto3" json:"api_version,omitempty"` + Kind string `protobuf:"bytes,2,opt,name=kind,proto3" json:"kind,omitempty"` + Name string `protobuf:"bytes,3,opt,name=name,proto3" json:"name,omitempty"` + Uid string `protobuf:"bytes,4,opt,name=uid,proto3" json:"uid,omitempty"` + Controller *wrapperspb.BoolValue `protobuf:"bytes,5,opt,name=controller,proto3" json:"controller,omitempty"` + BlockOwnerDeletion *wrapperspb.BoolValue `protobuf:"bytes,6,opt,name=block_owner_deletion,json=blockOwnerDeletion,proto3" json:"block_owner_deletion,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *OwnerReference) Reset() { + *x = OwnerReference{} + mi := &file_proto_spark_submit_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *OwnerReference) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*OwnerReference) ProtoMessage() {} + +func (x *OwnerReference) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[4] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use OwnerReference.ProtoReflect.Descriptor instead. +func (*OwnerReference) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{4} +} + +func (x *OwnerReference) GetApiVersion() string { + if x != nil { + return x.ApiVersion + } + return "" +} + +func (x *OwnerReference) GetKind() string { + if x != nil { + return x.Kind + } + return "" +} + +func (x *OwnerReference) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *OwnerReference) GetUid() string { + if x != nil { + return x.Uid + } + return "" +} + +func (x *OwnerReference) GetController() *wrapperspb.BoolValue { + if x != nil { + return x.Controller + } + return nil +} + +func (x *OwnerReference) GetBlockOwnerDeletion() *wrapperspb.BoolValue { + if x != nil { + return x.BlockOwnerDeletion + } + return nil +} + +type DriverIngressConfiguration struct { + state protoimpl.MessageState `protogen:"open.v1"` + ServicePort *wrapperspb.Int32Value `protobuf:"bytes,1,opt,name=service_port,json=servicePort,proto3" json:"service_port,omitempty"` + ServicePortName *wrapperspb.StringValue `protobuf:"bytes,2,opt,name=service_port_name,json=servicePortName,proto3" json:"service_port_name,omitempty"` + ServiceType ServiceType `protobuf:"varint,3,opt,name=service_type,json=serviceType,proto3,enum=spark.ServiceType" json:"service_type,omitempty"` + ServiceAnnotations map[string]string `protobuf:"bytes,4,rep,name=service_annotations,json=serviceAnnotations,proto3" json:"service_annotations,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + ServiceLabels map[string]string `protobuf:"bytes,5,rep,name=service_labels,json=serviceLabels,proto3" json:"service_labels,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + IngressUrlFormat string `protobuf:"bytes,6,opt,name=ingress_url_format,json=ingressUrlFormat,proto3" json:"ingress_url_format,omitempty"` + IngressAnnotations map[string]string `protobuf:"bytes,7,rep,name=ingress_annotations,json=ingressAnnotations,proto3" json:"ingress_annotations,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + IngressTls []*IngressTLS `protobuf:"bytes,8,rep,name=ingress_tls,json=ingressTls,proto3" json:"ingress_tls,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DriverIngressConfiguration) Reset() { + *x = DriverIngressConfiguration{} + mi := &file_proto_spark_submit_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DriverIngressConfiguration) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DriverIngressConfiguration) ProtoMessage() {} + +func (x *DriverIngressConfiguration) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[5] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DriverIngressConfiguration.ProtoReflect.Descriptor instead. +func (*DriverIngressConfiguration) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{5} +} + +func (x *DriverIngressConfiguration) GetServicePort() *wrapperspb.Int32Value { + if x != nil { + return x.ServicePort + } + return nil +} + +func (x *DriverIngressConfiguration) GetServicePortName() *wrapperspb.StringValue { + if x != nil { + return x.ServicePortName + } + return nil +} + +func (x *DriverIngressConfiguration) GetServiceType() ServiceType { + if x != nil { + return x.ServiceType + } + return ServiceType_SERVICE_TYPE_UNSPECIFIED +} + +func (x *DriverIngressConfiguration) GetServiceAnnotations() map[string]string { + if x != nil { + return x.ServiceAnnotations + } + return nil +} + +func (x *DriverIngressConfiguration) GetServiceLabels() map[string]string { + if x != nil { + return x.ServiceLabels + } + return nil +} + +func (x *DriverIngressConfiguration) GetIngressUrlFormat() string { + if x != nil { + return x.IngressUrlFormat + } + return "" +} + +func (x *DriverIngressConfiguration) GetIngressAnnotations() map[string]string { + if x != nil { + return x.IngressAnnotations + } + return nil +} + +func (x *DriverIngressConfiguration) GetIngressTls() []*IngressTLS { + if x != nil { + return x.IngressTls + } + return nil +} + +type SparkUIConfiguration struct { + state protoimpl.MessageState `protogen:"open.v1"` + ServicePort *wrapperspb.Int32Value `protobuf:"bytes,1,opt,name=service_port,json=servicePort,proto3" json:"service_port,omitempty"` + ServicePortName *wrapperspb.StringValue `protobuf:"bytes,2,opt,name=service_port_name,json=servicePortName,proto3" json:"service_port_name,omitempty"` + ServiceType ServiceType `protobuf:"varint,3,opt,name=service_type,json=serviceType,proto3,enum=spark.ServiceType" json:"service_type,omitempty"` + ServiceAnnotations map[string]string `protobuf:"bytes,4,rep,name=service_annotations,json=serviceAnnotations,proto3" json:"service_annotations,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + ServiceLabels map[string]string `protobuf:"bytes,5,rep,name=service_labels,json=serviceLabels,proto3" json:"service_labels,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + IngressAnnotations map[string]string `protobuf:"bytes,6,rep,name=ingress_annotations,json=ingressAnnotations,proto3" json:"ingress_annotations,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + IngressTls []*IngressTLS `protobuf:"bytes,7,rep,name=ingress_tls,json=ingressTls,proto3" json:"ingress_tls,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SparkUIConfiguration) Reset() { + *x = SparkUIConfiguration{} + mi := &file_proto_spark_submit_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SparkUIConfiguration) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SparkUIConfiguration) ProtoMessage() {} + +func (x *SparkUIConfiguration) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[6] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SparkUIConfiguration.ProtoReflect.Descriptor instead. +func (*SparkUIConfiguration) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{6} +} + +func (x *SparkUIConfiguration) GetServicePort() *wrapperspb.Int32Value { + if x != nil { + return x.ServicePort + } + return nil +} + +func (x *SparkUIConfiguration) GetServicePortName() *wrapperspb.StringValue { + if x != nil { + return x.ServicePortName + } + return nil +} + +func (x *SparkUIConfiguration) GetServiceType() ServiceType { + if x != nil { + return x.ServiceType + } + return ServiceType_SERVICE_TYPE_UNSPECIFIED +} + +func (x *SparkUIConfiguration) GetServiceAnnotations() map[string]string { + if x != nil { + return x.ServiceAnnotations + } + return nil +} + +func (x *SparkUIConfiguration) GetServiceLabels() map[string]string { + if x != nil { + return x.ServiceLabels + } + return nil +} + +func (x *SparkUIConfiguration) GetIngressAnnotations() map[string]string { + if x != nil { + return x.IngressAnnotations + } + return nil +} + +func (x *SparkUIConfiguration) GetIngressTls() []*IngressTLS { + if x != nil { + return x.IngressTls + } + return nil +} + +type IngressTLS struct { + state protoimpl.MessageState `protogen:"open.v1"` + Hosts []string `protobuf:"bytes,1,rep,name=hosts,proto3" json:"hosts,omitempty"` + SecretName string `protobuf:"bytes,2,opt,name=secret_name,json=secretName,proto3" json:"secret_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *IngressTLS) Reset() { + *x = IngressTLS{} + mi := &file_proto_spark_submit_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *IngressTLS) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IngressTLS) ProtoMessage() {} + +func (x *IngressTLS) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[7] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use IngressTLS.ProtoReflect.Descriptor instead. +func (*IngressTLS) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{7} +} + +func (x *IngressTLS) GetHosts() []string { + if x != nil { + return x.Hosts + } + return nil +} + +func (x *IngressTLS) GetSecretName() string { + if x != nil { + return x.SecretName + } + return "" +} + +type BatchSchedulerConfiguration struct { + state protoimpl.MessageState `protogen:"open.v1"` + Queue *wrapperspb.StringValue `protobuf:"bytes,1,opt,name=queue,proto3" json:"queue,omitempty"` + PriorityClassName *wrapperspb.StringValue `protobuf:"bytes,2,opt,name=priority_class_name,json=priorityClassName,proto3" json:"priority_class_name,omitempty"` + Resources map[string]*Quantity `protobuf:"bytes,3,rep,name=resources,proto3" json:"resources,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BatchSchedulerConfiguration) Reset() { + *x = BatchSchedulerConfiguration{} + mi := &file_proto_spark_submit_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BatchSchedulerConfiguration) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BatchSchedulerConfiguration) ProtoMessage() {} + +func (x *BatchSchedulerConfiguration) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[8] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BatchSchedulerConfiguration.ProtoReflect.Descriptor instead. +func (*BatchSchedulerConfiguration) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{8} +} + +func (x *BatchSchedulerConfiguration) GetQueue() *wrapperspb.StringValue { + if x != nil { + return x.Queue + } + return nil +} + +func (x *BatchSchedulerConfiguration) GetPriorityClassName() *wrapperspb.StringValue { + if x != nil { + return x.PriorityClassName + } + return nil +} + +func (x *BatchSchedulerConfiguration) GetResources() map[string]*Quantity { + if x != nil { + return x.Resources + } + return nil +} + +type MonitoringSpec struct { + state protoimpl.MessageState `protogen:"open.v1"` + ExposeDriverMetrics bool `protobuf:"varint,1,opt,name=expose_driver_metrics,json=exposeDriverMetrics,proto3" json:"expose_driver_metrics,omitempty"` + ExposeExecutorMetrics bool `protobuf:"varint,2,opt,name=expose_executor_metrics,json=exposeExecutorMetrics,proto3" json:"expose_executor_metrics,omitempty"` + MetricsProperties *wrapperspb.StringValue `protobuf:"bytes,3,opt,name=metrics_properties,json=metricsProperties,proto3" json:"metrics_properties,omitempty"` + MetricsPropertiesFile *wrapperspb.StringValue `protobuf:"bytes,4,opt,name=metrics_properties_file,json=metricsPropertiesFile,proto3" json:"metrics_properties_file,omitempty"` + Prometheus *PrometheusSpec `protobuf:"bytes,5,opt,name=prometheus,proto3" json:"prometheus,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *MonitoringSpec) Reset() { + *x = MonitoringSpec{} + mi := &file_proto_spark_submit_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *MonitoringSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*MonitoringSpec) ProtoMessage() {} + +func (x *MonitoringSpec) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[9] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use MonitoringSpec.ProtoReflect.Descriptor instead. +func (*MonitoringSpec) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{9} +} + +func (x *MonitoringSpec) GetExposeDriverMetrics() bool { + if x != nil { + return x.ExposeDriverMetrics + } + return false +} + +func (x *MonitoringSpec) GetExposeExecutorMetrics() bool { + if x != nil { + return x.ExposeExecutorMetrics + } + return false +} + +func (x *MonitoringSpec) GetMetricsProperties() *wrapperspb.StringValue { + if x != nil { + return x.MetricsProperties + } + return nil +} + +func (x *MonitoringSpec) GetMetricsPropertiesFile() *wrapperspb.StringValue { + if x != nil { + return x.MetricsPropertiesFile + } + return nil +} + +func (x *MonitoringSpec) GetPrometheus() *PrometheusSpec { + if x != nil { + return x.Prometheus + } + return nil +} + +type PrometheusSpec struct { + state protoimpl.MessageState `protogen:"open.v1"` + JmxExporterJar string `protobuf:"bytes,1,opt,name=jmx_exporter_jar,json=jmxExporterJar,proto3" json:"jmx_exporter_jar,omitempty"` + Port *wrapperspb.Int32Value `protobuf:"bytes,2,opt,name=port,proto3" json:"port,omitempty"` + PortName *wrapperspb.StringValue `protobuf:"bytes,3,opt,name=port_name,json=portName,proto3" json:"port_name,omitempty"` + ConfigFile *wrapperspb.StringValue `protobuf:"bytes,4,opt,name=config_file,json=configFile,proto3" json:"config_file,omitempty"` + Configuration *wrapperspb.StringValue `protobuf:"bytes,5,opt,name=configuration,proto3" json:"configuration,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PrometheusSpec) Reset() { + *x = PrometheusSpec{} + mi := &file_proto_spark_submit_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PrometheusSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PrometheusSpec) ProtoMessage() {} + +func (x *PrometheusSpec) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[10] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PrometheusSpec.ProtoReflect.Descriptor instead. +func (*PrometheusSpec) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{10} +} + +func (x *PrometheusSpec) GetJmxExporterJar() string { + if x != nil { + return x.JmxExporterJar + } + return "" +} + +func (x *PrometheusSpec) GetPort() *wrapperspb.Int32Value { + if x != nil { + return x.Port + } + return nil +} + +func (x *PrometheusSpec) GetPortName() *wrapperspb.StringValue { + if x != nil { + return x.PortName + } + return nil +} + +func (x *PrometheusSpec) GetConfigFile() *wrapperspb.StringValue { + if x != nil { + return x.ConfigFile + } + return nil +} + +func (x *PrometheusSpec) GetConfiguration() *wrapperspb.StringValue { + if x != nil { + return x.Configuration + } + return nil +} + +// RestartPolicy +type RestartPolicy struct { + state protoimpl.MessageState `protogen:"open.v1"` + Type string `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RestartPolicy) Reset() { + *x = RestartPolicy{} + mi := &file_proto_spark_submit_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RestartPolicy) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RestartPolicy) ProtoMessage() {} + +func (x *RestartPolicy) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[11] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RestartPolicy.ProtoReflect.Descriptor instead. +func (*RestartPolicy) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{11} +} + +func (x *RestartPolicy) GetType() string { + if x != nil { + return x.Type + } + return "" +} + +// DriverSpec and ExecutorSpec +type DriverSpec struct { + state protoimpl.MessageState `protogen:"open.v1"` + SparkPodSpec *SparkPodSpec `protobuf:"bytes,1,opt,name=spark_pod_spec,json=sparkPodSpec,proto3" json:"spark_pod_spec,omitempty"` + PodName *wrapperspb.StringValue `protobuf:"bytes,2,opt,name=pod_name,json=podName,proto3" json:"pod_name,omitempty"` + CoreRequest *wrapperspb.StringValue `protobuf:"bytes,3,opt,name=core_request,json=coreRequest,proto3" json:"core_request,omitempty"` + JavaOptions *wrapperspb.StringValue `protobuf:"bytes,4,opt,name=java_options,json=javaOptions,proto3" json:"java_options,omitempty"` + LifeCycle *Lifecycle `protobuf:"bytes,5,opt,name=life_cycle,json=lifeCycle,proto3" json:"life_cycle,omitempty"` + KubernetesMaster *wrapperspb.StringValue `protobuf:"bytes,6,opt,name=kubernetes_master,json=kubernetesMaster,proto3" json:"kubernetes_master,omitempty"` + ServiceAnnotations map[string]string `protobuf:"bytes,7,rep,name=service_annotations,json=serviceAnnotations,proto3" json:"service_annotations,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + ServiceLabels map[string]string `protobuf:"bytes,8,rep,name=service_labels,json=serviceLabels,proto3" json:"service_labels,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + Ports []*Ports `protobuf:"bytes,9,rep,name=ports,proto3" json:"ports,omitempty"` + PriorityClassName *wrapperspb.StringValue `protobuf:"bytes,10,opt,name=priority_class_name,json=priorityClassName,proto3" json:"priority_class_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DriverSpec) Reset() { + *x = DriverSpec{} + mi := &file_proto_spark_submit_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DriverSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DriverSpec) ProtoMessage() {} + +func (x *DriverSpec) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[12] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DriverSpec.ProtoReflect.Descriptor instead. +func (*DriverSpec) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{12} +} + +func (x *DriverSpec) GetSparkPodSpec() *SparkPodSpec { + if x != nil { + return x.SparkPodSpec + } + return nil +} + +func (x *DriverSpec) GetPodName() *wrapperspb.StringValue { + if x != nil { + return x.PodName + } + return nil +} + +func (x *DriverSpec) GetCoreRequest() *wrapperspb.StringValue { + if x != nil { + return x.CoreRequest + } + return nil +} + +func (x *DriverSpec) GetJavaOptions() *wrapperspb.StringValue { + if x != nil { + return x.JavaOptions + } + return nil +} + +func (x *DriverSpec) GetLifeCycle() *Lifecycle { + if x != nil { + return x.LifeCycle + } + return nil +} + +func (x *DriverSpec) GetKubernetesMaster() *wrapperspb.StringValue { + if x != nil { + return x.KubernetesMaster + } + return nil +} + +func (x *DriverSpec) GetServiceAnnotations() map[string]string { + if x != nil { + return x.ServiceAnnotations + } + return nil +} + +func (x *DriverSpec) GetServiceLabels() map[string]string { + if x != nil { + return x.ServiceLabels + } + return nil +} + +func (x *DriverSpec) GetPorts() []*Ports { + if x != nil { + return x.Ports + } + return nil +} + +func (x *DriverSpec) GetPriorityClassName() *wrapperspb.StringValue { + if x != nil { + return x.PriorityClassName + } + return nil +} + +type SparkPodSpec struct { + state protoimpl.MessageState `protogen:"open.v1"` + Template *PodTemplateSpec `protobuf:"bytes,1,opt,name=template,proto3" json:"template,omitempty"` + Cores *wrapperspb.Int32Value `protobuf:"bytes,2,opt,name=cores,proto3" json:"cores,omitempty"` + CoreLimit string `protobuf:"bytes,3,opt,name=core_limit,json=coreLimit,proto3" json:"core_limit,omitempty"` + Memory string `protobuf:"bytes,4,opt,name=memory,proto3" json:"memory,omitempty"` + MemoryOverhead string `protobuf:"bytes,5,opt,name=memory_overhead,json=memoryOverhead,proto3" json:"memory_overhead,omitempty"` + Gpu *GPUSpec `protobuf:"bytes,6,opt,name=gpu,proto3" json:"gpu,omitempty"` + Image string `protobuf:"bytes,7,opt,name=image,proto3" json:"image,omitempty"` + Configmaps []*NamePath `protobuf:"bytes,8,rep,name=configmaps,proto3" json:"configmaps,omitempty"` + Secrets []*SecretInfo `protobuf:"bytes,9,rep,name=secrets,proto3" json:"secrets,omitempty"` + Env []*EnvVar `protobuf:"bytes,10,rep,name=env,proto3" json:"env,omitempty"` + EnvVars map[string]string `protobuf:"bytes,11,rep,name=env_vars,json=envVars,proto3" json:"env_vars,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + EnvFrom []*EnvFromSource `protobuf:"bytes,12,rep,name=env_from,json=envFrom,proto3" json:"env_from,omitempty"` + Labels map[string]string `protobuf:"bytes,13,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + Annotations map[string]string `protobuf:"bytes,14,rep,name=annotations,proto3" json:"annotations,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + VolumeMounts []*VolumeMount `protobuf:"bytes,15,rep,name=volume_mounts,json=volumeMounts,proto3" json:"volume_mounts,omitempty"` + Affinity *Affinity `protobuf:"bytes,16,opt,name=affinity,proto3" json:"affinity,omitempty"` + Tolerations []*Toleration `protobuf:"bytes,17,rep,name=tolerations,proto3" json:"tolerations,omitempty"` + PodSecurityContext *PodSecurityContext `protobuf:"bytes,18,opt,name=pod_security_context,json=podSecurityContext,proto3" json:"pod_security_context,omitempty"` + SecurityContext *SecurityContext `protobuf:"bytes,19,opt,name=security_context,json=securityContext,proto3" json:"security_context,omitempty"` + SchedulerName *wrapperspb.StringValue `protobuf:"bytes,20,opt,name=scheduler_name,json=schedulerName,proto3" json:"scheduler_name,omitempty"` + Sidecars []*Container `protobuf:"bytes,21,rep,name=sidecars,proto3" json:"sidecars,omitempty"` + InitContainers []*Container `protobuf:"bytes,22,rep,name=init_containers,json=initContainers,proto3" json:"init_containers,omitempty"` + HostNetwork *wrapperspb.BoolValue `protobuf:"bytes,23,opt,name=host_network,json=hostNetwork,proto3" json:"host_network,omitempty"` + NodeSelector map[string]string `protobuf:"bytes,24,rep,name=node_selector,json=nodeSelector,proto3" json:"node_selector,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + DnsConfig *PodDNSConfig `protobuf:"bytes,25,opt,name=dns_config,json=dnsConfig,proto3" json:"dns_config,omitempty"` + TerminationGracePeriodSeconds int64 `protobuf:"varint,26,opt,name=termination_grace_period_seconds,json=terminationGracePeriodSeconds,proto3" json:"termination_grace_period_seconds,omitempty"` + ServiceAccount *wrapperspb.StringValue `protobuf:"bytes,27,opt,name=service_account,json=serviceAccount,proto3" json:"service_account,omitempty"` + HostAliases []*HostAlias `protobuf:"bytes,28,rep,name=host_aliases,json=hostAliases,proto3" json:"host_aliases,omitempty"` + ShareProcessNamespace *wrapperspb.BoolValue `protobuf:"bytes,29,opt,name=share_process_namespace,json=shareProcessNamespace,proto3" json:"share_process_namespace,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SparkPodSpec) Reset() { + *x = SparkPodSpec{} + mi := &file_proto_spark_submit_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SparkPodSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SparkPodSpec) ProtoMessage() {} + +func (x *SparkPodSpec) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[13] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SparkPodSpec.ProtoReflect.Descriptor instead. +func (*SparkPodSpec) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{13} +} + +func (x *SparkPodSpec) GetTemplate() *PodTemplateSpec { + if x != nil { + return x.Template + } + return nil +} + +func (x *SparkPodSpec) GetCores() *wrapperspb.Int32Value { + if x != nil { + return x.Cores + } + return nil +} + +func (x *SparkPodSpec) GetCoreLimit() string { + if x != nil { + return x.CoreLimit + } + return "" +} + +func (x *SparkPodSpec) GetMemory() string { + if x != nil { + return x.Memory + } + return "" +} + +func (x *SparkPodSpec) GetMemoryOverhead() string { + if x != nil { + return x.MemoryOverhead + } + return "" +} + +func (x *SparkPodSpec) GetGpu() *GPUSpec { + if x != nil { + return x.Gpu + } + return nil +} + +func (x *SparkPodSpec) GetImage() string { + if x != nil { + return x.Image + } + return "" +} + +func (x *SparkPodSpec) GetConfigmaps() []*NamePath { + if x != nil { + return x.Configmaps + } + return nil +} + +func (x *SparkPodSpec) GetSecrets() []*SecretInfo { + if x != nil { + return x.Secrets + } + return nil +} + +func (x *SparkPodSpec) GetEnv() []*EnvVar { + if x != nil { + return x.Env + } + return nil +} + +func (x *SparkPodSpec) GetEnvVars() map[string]string { + if x != nil { + return x.EnvVars + } + return nil +} + +func (x *SparkPodSpec) GetEnvFrom() []*EnvFromSource { + if x != nil { + return x.EnvFrom + } + return nil +} + +func (x *SparkPodSpec) GetLabels() map[string]string { + if x != nil { + return x.Labels + } + return nil +} + +func (x *SparkPodSpec) GetAnnotations() map[string]string { + if x != nil { + return x.Annotations + } + return nil +} + +func (x *SparkPodSpec) GetVolumeMounts() []*VolumeMount { + if x != nil { + return x.VolumeMounts + } + return nil +} + +func (x *SparkPodSpec) GetAffinity() *Affinity { + if x != nil { + return x.Affinity + } + return nil +} + +func (x *SparkPodSpec) GetTolerations() []*Toleration { + if x != nil { + return x.Tolerations + } + return nil +} + +func (x *SparkPodSpec) GetPodSecurityContext() *PodSecurityContext { + if x != nil { + return x.PodSecurityContext + } + return nil +} + +func (x *SparkPodSpec) GetSecurityContext() *SecurityContext { + if x != nil { + return x.SecurityContext + } + return nil +} + +func (x *SparkPodSpec) GetSchedulerName() *wrapperspb.StringValue { + if x != nil { + return x.SchedulerName + } + return nil +} + +func (x *SparkPodSpec) GetSidecars() []*Container { + if x != nil { + return x.Sidecars + } + return nil +} + +func (x *SparkPodSpec) GetInitContainers() []*Container { + if x != nil { + return x.InitContainers + } + return nil +} + +func (x *SparkPodSpec) GetHostNetwork() *wrapperspb.BoolValue { + if x != nil { + return x.HostNetwork + } + return nil +} + +func (x *SparkPodSpec) GetNodeSelector() map[string]string { + if x != nil { + return x.NodeSelector + } + return nil +} + +func (x *SparkPodSpec) GetDnsConfig() *PodDNSConfig { + if x != nil { + return x.DnsConfig + } + return nil +} + +func (x *SparkPodSpec) GetTerminationGracePeriodSeconds() int64 { + if x != nil { + return x.TerminationGracePeriodSeconds + } + return 0 +} + +func (x *SparkPodSpec) GetServiceAccount() *wrapperspb.StringValue { + if x != nil { + return x.ServiceAccount + } + return nil +} + +func (x *SparkPodSpec) GetHostAliases() []*HostAlias { + if x != nil { + return x.HostAliases + } + return nil +} + +func (x *SparkPodSpec) GetShareProcessNamespace() *wrapperspb.BoolValue { + if x != nil { + return x.ShareProcessNamespace + } + return nil +} + +type PodTemplateSpec struct { + state protoimpl.MessageState `protogen:"open.v1"` + ObjectMeta *ObjectMeta `protobuf:"bytes,1,opt,name=object_meta,json=objectMeta,proto3" json:"object_meta,omitempty"` + PodSpec *PodSpec `protobuf:"bytes,2,opt,name=pod_spec,json=podSpec,proto3" json:"pod_spec,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PodTemplateSpec) Reset() { + *x = PodTemplateSpec{} + mi := &file_proto_spark_submit_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PodTemplateSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PodTemplateSpec) ProtoMessage() {} + +func (x *PodTemplateSpec) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[14] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PodTemplateSpec.ProtoReflect.Descriptor instead. +func (*PodTemplateSpec) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{14} +} + +func (x *PodTemplateSpec) GetObjectMeta() *ObjectMeta { + if x != nil { + return x.ObjectMeta + } + return nil +} + +func (x *PodTemplateSpec) GetPodSpec() *PodSpec { + if x != nil { + return x.PodSpec + } + return nil +} + +type PodSpec struct { + state protoimpl.MessageState `protogen:"open.v1"` + Volumes []*Volume `protobuf:"bytes,1,rep,name=volumes,proto3" json:"volumes,omitempty"` + Containers []*Container `protobuf:"bytes,2,rep,name=containers,proto3" json:"containers,omitempty"` + EphemeralContainers []*EphemeralContainer `protobuf:"bytes,3,rep,name=ephemeral_containers,json=ephemeralContainers,proto3" json:"ephemeral_containers,omitempty"` + RestartPolicy *RestartPolicy `protobuf:"bytes,4,opt,name=restart_policy,json=restartPolicy,proto3" json:"restart_policy,omitempty"` + TerminationGracePeriodSeconds *wrapperspb.Int64Value `protobuf:"bytes,5,opt,name=termination_grace_period_seconds,json=terminationGracePeriodSeconds,proto3" json:"termination_grace_period_seconds,omitempty"` + ActiveDeadlineSeconds *wrapperspb.Int64Value `protobuf:"bytes,6,opt,name=active_deadline_seconds,json=activeDeadlineSeconds,proto3" json:"active_deadline_seconds,omitempty"` + DnsPolicy DNSPolicy `protobuf:"varint,7,opt,name=dns_policy,json=dnsPolicy,proto3,enum=spark.DNSPolicy" json:"dns_policy,omitempty"` + NodeSelector map[string]string `protobuf:"bytes,8,rep,name=node_selector,json=nodeSelector,proto3" json:"node_selector,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + ServiceAccountName string `protobuf:"bytes,9,opt,name=service_account_name,json=serviceAccountName,proto3" json:"service_account_name,omitempty"` + DeprecatedServiceAccount string `protobuf:"bytes,10,opt,name=deprecated_service_account,json=deprecatedServiceAccount,proto3" json:"deprecated_service_account,omitempty"` + AutoMountServiceAccountToken *wrapperspb.BoolValue `protobuf:"bytes,11,opt,name=auto_mount_service_account_token,json=autoMountServiceAccountToken,proto3" json:"auto_mount_service_account_token,omitempty"` + NodeName string `protobuf:"bytes,12,opt,name=node_name,json=nodeName,proto3" json:"node_name,omitempty"` + HostNetwork bool `protobuf:"varint,13,opt,name=host_network,json=hostNetwork,proto3" json:"host_network,omitempty"` + HostPid bool `protobuf:"varint,14,opt,name=host_pid,json=hostPid,proto3" json:"host_pid,omitempty"` + HostIpc bool `protobuf:"varint,15,opt,name=host_ipc,json=hostIpc,proto3" json:"host_ipc,omitempty"` + ShareProcessName *wrapperspb.BoolValue `protobuf:"bytes,16,opt,name=share_process_name,json=shareProcessName,proto3" json:"share_process_name,omitempty"` + SecurityContext *PodSecurityContext `protobuf:"bytes,17,opt,name=security_context,json=securityContext,proto3" json:"security_context,omitempty"` + ImagePullSecrets []*LocalObjectReference `protobuf:"bytes,18,rep,name=image_pull_secrets,json=imagePullSecrets,proto3" json:"image_pull_secrets,omitempty"` + HostName string `protobuf:"bytes,19,opt,name=host_name,json=hostName,proto3" json:"host_name,omitempty"` + SubDomain string `protobuf:"bytes,20,opt,name=sub_domain,json=subDomain,proto3" json:"sub_domain,omitempty"` + Affinity *Affinity `protobuf:"bytes,21,opt,name=affinity,proto3" json:"affinity,omitempty"` + SchedulerName string `protobuf:"bytes,22,opt,name=scheduler_name,json=schedulerName,proto3" json:"scheduler_name,omitempty"` + Tolerations []*Toleration `protobuf:"bytes,23,rep,name=tolerations,proto3" json:"tolerations,omitempty"` + HostAliases []*HostAlias `protobuf:"bytes,24,rep,name=host_aliases,json=hostAliases,proto3" json:"host_aliases,omitempty"` + PriorityClassName string `protobuf:"bytes,25,opt,name=priority_class_name,json=priorityClassName,proto3" json:"priority_class_name,omitempty"` + Priority *wrapperspb.Int32Value `protobuf:"bytes,26,opt,name=priority,proto3" json:"priority,omitempty"` + DnsConfig *PodDNSConfig `protobuf:"bytes,27,opt,name=dns_config,json=dnsConfig,proto3" json:"dns_config,omitempty"` + ReadinessGates []*PodReadinessGate `protobuf:"bytes,28,rep,name=readiness_gates,json=readinessGates,proto3" json:"readiness_gates,omitempty"` + RuntimeClassName *wrapperspb.StringValue `protobuf:"bytes,29,opt,name=runtime_class_name,json=runtimeClassName,proto3" json:"runtime_class_name,omitempty"` + EnableServiceLinks *wrapperspb.BoolValue `protobuf:"bytes,30,opt,name=enable_service_links,json=enableServiceLinks,proto3" json:"enable_service_links,omitempty"` + Overhead map[string]*Quantity `protobuf:"bytes,31,rep,name=overhead,proto3" json:"overhead,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + TopologySpreadConstraints []*TopologySpreadConstraint `protobuf:"bytes,32,rep,name=topology_spread_constraints,json=topologySpreadConstraints,proto3" json:"topology_spread_constraints,omitempty"` + SetHostNameAsFqdn *wrapperspb.BoolValue `protobuf:"bytes,33,opt,name=set_host_name_as_fqdn,json=setHostNameAsFqdn,proto3" json:"set_host_name_as_fqdn,omitempty"` + Os *PodOS `protobuf:"bytes,34,opt,name=os,proto3" json:"os,omitempty"` + HostUsers *wrapperspb.BoolValue `protobuf:"bytes,35,opt,name=host_users,json=hostUsers,proto3" json:"host_users,omitempty"` + SchedulingGates []*PodSchedulingGate `protobuf:"bytes,36,rep,name=scheduling_gates,json=schedulingGates,proto3" json:"scheduling_gates,omitempty"` + ResourceClaims []*PodResourceClaim `protobuf:"bytes,37,rep,name=resource_claims,json=resourceClaims,proto3" json:"resource_claims,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PodSpec) Reset() { + *x = PodSpec{} + mi := &file_proto_spark_submit_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PodSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PodSpec) ProtoMessage() {} + +func (x *PodSpec) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[15] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PodSpec.ProtoReflect.Descriptor instead. +func (*PodSpec) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{15} +} + +func (x *PodSpec) GetVolumes() []*Volume { + if x != nil { + return x.Volumes + } + return nil +} + +func (x *PodSpec) GetContainers() []*Container { + if x != nil { + return x.Containers + } + return nil +} + +func (x *PodSpec) GetEphemeralContainers() []*EphemeralContainer { + if x != nil { + return x.EphemeralContainers + } + return nil +} + +func (x *PodSpec) GetRestartPolicy() *RestartPolicy { + if x != nil { + return x.RestartPolicy + } + return nil +} + +func (x *PodSpec) GetTerminationGracePeriodSeconds() *wrapperspb.Int64Value { + if x != nil { + return x.TerminationGracePeriodSeconds + } + return nil +} + +func (x *PodSpec) GetActiveDeadlineSeconds() *wrapperspb.Int64Value { + if x != nil { + return x.ActiveDeadlineSeconds + } + return nil +} + +func (x *PodSpec) GetDnsPolicy() DNSPolicy { + if x != nil { + return x.DnsPolicy + } + return DNSPolicy_DNS_POLICY_UNSPECIFIED +} + +func (x *PodSpec) GetNodeSelector() map[string]string { + if x != nil { + return x.NodeSelector + } + return nil +} + +func (x *PodSpec) GetServiceAccountName() string { + if x != nil { + return x.ServiceAccountName + } + return "" +} + +func (x *PodSpec) GetDeprecatedServiceAccount() string { + if x != nil { + return x.DeprecatedServiceAccount + } + return "" +} + +func (x *PodSpec) GetAutoMountServiceAccountToken() *wrapperspb.BoolValue { + if x != nil { + return x.AutoMountServiceAccountToken + } + return nil +} + +func (x *PodSpec) GetNodeName() string { + if x != nil { + return x.NodeName + } + return "" +} + +func (x *PodSpec) GetHostNetwork() bool { + if x != nil { + return x.HostNetwork + } + return false +} + +func (x *PodSpec) GetHostPid() bool { + if x != nil { + return x.HostPid + } + return false +} + +func (x *PodSpec) GetHostIpc() bool { + if x != nil { + return x.HostIpc + } + return false +} + +func (x *PodSpec) GetShareProcessName() *wrapperspb.BoolValue { + if x != nil { + return x.ShareProcessName + } + return nil +} + +func (x *PodSpec) GetSecurityContext() *PodSecurityContext { + if x != nil { + return x.SecurityContext + } + return nil +} + +func (x *PodSpec) GetImagePullSecrets() []*LocalObjectReference { + if x != nil { + return x.ImagePullSecrets + } + return nil +} + +func (x *PodSpec) GetHostName() string { + if x != nil { + return x.HostName + } + return "" +} + +func (x *PodSpec) GetSubDomain() string { + if x != nil { + return x.SubDomain + } + return "" +} + +func (x *PodSpec) GetAffinity() *Affinity { + if x != nil { + return x.Affinity + } + return nil +} + +func (x *PodSpec) GetSchedulerName() string { + if x != nil { + return x.SchedulerName + } + return "" +} + +func (x *PodSpec) GetTolerations() []*Toleration { + if x != nil { + return x.Tolerations + } + return nil +} + +func (x *PodSpec) GetHostAliases() []*HostAlias { + if x != nil { + return x.HostAliases + } + return nil +} + +func (x *PodSpec) GetPriorityClassName() string { + if x != nil { + return x.PriorityClassName + } + return "" +} + +func (x *PodSpec) GetPriority() *wrapperspb.Int32Value { + if x != nil { + return x.Priority + } + return nil +} + +func (x *PodSpec) GetDnsConfig() *PodDNSConfig { + if x != nil { + return x.DnsConfig + } + return nil +} + +func (x *PodSpec) GetReadinessGates() []*PodReadinessGate { + if x != nil { + return x.ReadinessGates + } + return nil +} + +func (x *PodSpec) GetRuntimeClassName() *wrapperspb.StringValue { + if x != nil { + return x.RuntimeClassName + } + return nil +} + +func (x *PodSpec) GetEnableServiceLinks() *wrapperspb.BoolValue { + if x != nil { + return x.EnableServiceLinks + } + return nil +} + +func (x *PodSpec) GetOverhead() map[string]*Quantity { + if x != nil { + return x.Overhead + } + return nil +} + +func (x *PodSpec) GetTopologySpreadConstraints() []*TopologySpreadConstraint { + if x != nil { + return x.TopologySpreadConstraints + } + return nil +} + +func (x *PodSpec) GetSetHostNameAsFqdn() *wrapperspb.BoolValue { + if x != nil { + return x.SetHostNameAsFqdn + } + return nil +} + +func (x *PodSpec) GetOs() *PodOS { + if x != nil { + return x.Os + } + return nil +} + +func (x *PodSpec) GetHostUsers() *wrapperspb.BoolValue { + if x != nil { + return x.HostUsers + } + return nil +} + +func (x *PodSpec) GetSchedulingGates() []*PodSchedulingGate { + if x != nil { + return x.SchedulingGates + } + return nil +} + +func (x *PodSpec) GetResourceClaims() []*PodResourceClaim { + if x != nil { + return x.ResourceClaims + } + return nil +} + +type EphemeralContainer struct { + state protoimpl.MessageState `protogen:"open.v1"` + EphemeralContainerCommon *EphemeralContainerCommon `protobuf:"bytes,1,opt,name=ephemeral_container_common,json=ephemeralContainerCommon,proto3" json:"ephemeral_container_common,omitempty"` + TargetContainerName string `protobuf:"bytes,2,opt,name=target_container_name,json=targetContainerName,proto3" json:"target_container_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *EphemeralContainer) Reset() { + *x = EphemeralContainer{} + mi := &file_proto_spark_submit_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *EphemeralContainer) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EphemeralContainer) ProtoMessage() {} + +func (x *EphemeralContainer) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[16] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EphemeralContainer.ProtoReflect.Descriptor instead. +func (*EphemeralContainer) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{16} +} + +func (x *EphemeralContainer) GetEphemeralContainerCommon() *EphemeralContainerCommon { + if x != nil { + return x.EphemeralContainerCommon + } + return nil +} + +func (x *EphemeralContainer) GetTargetContainerName() string { + if x != nil { + return x.TargetContainerName + } + return "" +} + +type EphemeralContainerCommon struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Image string `protobuf:"bytes,2,opt,name=image,proto3" json:"image,omitempty"` + Command []string `protobuf:"bytes,3,rep,name=command,proto3" json:"command,omitempty"` + Args []string `protobuf:"bytes,4,rep,name=args,proto3" json:"args,omitempty"` + WorkingDir string `protobuf:"bytes,5,opt,name=working_dir,json=workingDir,proto3" json:"working_dir,omitempty"` + Ports []*ContainerPort `protobuf:"bytes,6,rep,name=ports,proto3" json:"ports,omitempty"` + EnvFrom []*EnvFromSource `protobuf:"bytes,7,rep,name=env_from,json=envFrom,proto3" json:"env_from,omitempty"` + Env []*EnvVar `protobuf:"bytes,8,rep,name=env,proto3" json:"env,omitempty"` + Resources *ResourceRequirements `protobuf:"bytes,9,opt,name=resources,proto3" json:"resources,omitempty"` + ResizePolicy []*ContainerResizePolicy `protobuf:"bytes,10,rep,name=resize_policy,json=resizePolicy,proto3" json:"resize_policy,omitempty"` + RestartPolicy ContainerRestartPolicy `protobuf:"varint,11,opt,name=restart_policy,json=restartPolicy,proto3,enum=spark.ContainerRestartPolicy" json:"restart_policy,omitempty"` + VolumeMounts []*VolumeMount `protobuf:"bytes,12,rep,name=volume_mounts,json=volumeMounts,proto3" json:"volume_mounts,omitempty"` + VolumeDevices []*VolumeDevice `protobuf:"bytes,13,rep,name=volume_devices,json=volumeDevices,proto3" json:"volume_devices,omitempty"` + ReadinessProbe *Probe `protobuf:"bytes,14,opt,name=readiness_probe,json=readinessProbe,proto3" json:"readiness_probe,omitempty"` + LifeCycle *Lifecycle `protobuf:"bytes,15,opt,name=life_cycle,json=lifeCycle,proto3" json:"life_cycle,omitempty"` + TerminationMessagePath string `protobuf:"bytes,16,opt,name=termination_message_path,json=terminationMessagePath,proto3" json:"termination_message_path,omitempty"` + TerminationMessagePolicy TerminationMessagePolicy `protobuf:"varint,17,opt,name=termination_message_policy,json=terminationMessagePolicy,proto3,enum=spark.TerminationMessagePolicy" json:"termination_message_policy,omitempty"` + ImagePullPolicy PullPolicy `protobuf:"varint,18,opt,name=image_pull_policy,json=imagePullPolicy,proto3,enum=spark.PullPolicy" json:"image_pull_policy,omitempty"` + SecurityContext *SecurityContext `protobuf:"bytes,19,opt,name=security_context,json=securityContext,proto3" json:"security_context,omitempty"` + Stdin bool `protobuf:"varint,20,opt,name=stdin,proto3" json:"stdin,omitempty"` + StdinOnce bool `protobuf:"varint,21,opt,name=stdin_once,json=stdinOnce,proto3" json:"stdin_once,omitempty"` + Tty bool `protobuf:"varint,22,opt,name=tty,proto3" json:"tty,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *EphemeralContainerCommon) Reset() { + *x = EphemeralContainerCommon{} + mi := &file_proto_spark_submit_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *EphemeralContainerCommon) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EphemeralContainerCommon) ProtoMessage() {} + +func (x *EphemeralContainerCommon) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[17] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EphemeralContainerCommon.ProtoReflect.Descriptor instead. +func (*EphemeralContainerCommon) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{17} +} + +func (x *EphemeralContainerCommon) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *EphemeralContainerCommon) GetImage() string { + if x != nil { + return x.Image + } + return "" +} + +func (x *EphemeralContainerCommon) GetCommand() []string { + if x != nil { + return x.Command + } + return nil +} + +func (x *EphemeralContainerCommon) GetArgs() []string { + if x != nil { + return x.Args + } + return nil +} + +func (x *EphemeralContainerCommon) GetWorkingDir() string { + if x != nil { + return x.WorkingDir + } + return "" +} + +func (x *EphemeralContainerCommon) GetPorts() []*ContainerPort { + if x != nil { + return x.Ports + } + return nil +} + +func (x *EphemeralContainerCommon) GetEnvFrom() []*EnvFromSource { + if x != nil { + return x.EnvFrom + } + return nil +} + +func (x *EphemeralContainerCommon) GetEnv() []*EnvVar { + if x != nil { + return x.Env + } + return nil +} + +func (x *EphemeralContainerCommon) GetResources() *ResourceRequirements { + if x != nil { + return x.Resources + } + return nil +} + +func (x *EphemeralContainerCommon) GetResizePolicy() []*ContainerResizePolicy { + if x != nil { + return x.ResizePolicy + } + return nil +} + +func (x *EphemeralContainerCommon) GetRestartPolicy() ContainerRestartPolicy { + if x != nil { + return x.RestartPolicy + } + return ContainerRestartPolicy_CONTAINER_RESTART_POLICY_UNSPECIFIED +} + +func (x *EphemeralContainerCommon) GetVolumeMounts() []*VolumeMount { + if x != nil { + return x.VolumeMounts + } + return nil +} + +func (x *EphemeralContainerCommon) GetVolumeDevices() []*VolumeDevice { + if x != nil { + return x.VolumeDevices + } + return nil +} + +func (x *EphemeralContainerCommon) GetReadinessProbe() *Probe { + if x != nil { + return x.ReadinessProbe + } + return nil +} + +func (x *EphemeralContainerCommon) GetLifeCycle() *Lifecycle { + if x != nil { + return x.LifeCycle + } + return nil +} + +func (x *EphemeralContainerCommon) GetTerminationMessagePath() string { + if x != nil { + return x.TerminationMessagePath + } + return "" +} + +func (x *EphemeralContainerCommon) GetTerminationMessagePolicy() TerminationMessagePolicy { + if x != nil { + return x.TerminationMessagePolicy + } + return TerminationMessagePolicy_TERMINATION_MESSAGE_POLICY_UNSPECIFIED +} + +func (x *EphemeralContainerCommon) GetImagePullPolicy() PullPolicy { + if x != nil { + return x.ImagePullPolicy + } + return PullPolicy_PULL_POLICY_UNSPECIFIED +} + +func (x *EphemeralContainerCommon) GetSecurityContext() *SecurityContext { + if x != nil { + return x.SecurityContext + } + return nil +} + +func (x *EphemeralContainerCommon) GetStdin() bool { + if x != nil { + return x.Stdin + } + return false +} + +func (x *EphemeralContainerCommon) GetStdinOnce() bool { + if x != nil { + return x.StdinOnce + } + return false +} + +func (x *EphemeralContainerCommon) GetTty() bool { + if x != nil { + return x.Tty + } + return false +} + +type PodReadinessGate struct { + state protoimpl.MessageState `protogen:"open.v1"` + ConditionType PodConditionType `protobuf:"varint,1,opt,name=condition_type,json=conditionType,proto3,enum=spark.PodConditionType" json:"condition_type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PodReadinessGate) Reset() { + *x = PodReadinessGate{} + mi := &file_proto_spark_submit_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PodReadinessGate) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PodReadinessGate) ProtoMessage() {} + +func (x *PodReadinessGate) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[18] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PodReadinessGate.ProtoReflect.Descriptor instead. +func (*PodReadinessGate) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{18} +} + +func (x *PodReadinessGate) GetConditionType() PodConditionType { + if x != nil { + return x.ConditionType + } + return PodConditionType_POD_CONDITION_TYPE_UNSPECIFIED +} + +type TopologySpreadConstraint struct { + state protoimpl.MessageState `protogen:"open.v1"` + MaxSkew int32 `protobuf:"varint,1,opt,name=max_skew,json=maxSkew,proto3" json:"max_skew,omitempty"` + TopologyKey string `protobuf:"bytes,2,opt,name=topology_key,json=topologyKey,proto3" json:"topology_key,omitempty"` + WhenUnsatisfiable UnsatisfiableConstraintAction `protobuf:"varint,3,opt,name=when_unsatisfiable,json=whenUnsatisfiable,proto3,enum=spark.UnsatisfiableConstraintAction" json:"when_unsatisfiable,omitempty"` + LabelSelector *LabelSelector `protobuf:"bytes,4,opt,name=label_selector,json=labelSelector,proto3" json:"label_selector,omitempty"` + MinDomains *wrapperspb.Int32Value `protobuf:"bytes,5,opt,name=min_domains,json=minDomains,proto3" json:"min_domains,omitempty"` + NodeAffinityPolicy NodeInclusionPolicy `protobuf:"varint,6,opt,name=node_affinity_policy,json=nodeAffinityPolicy,proto3,enum=spark.NodeInclusionPolicy" json:"node_affinity_policy,omitempty"` + NodeTaintsPolicy NodeInclusionPolicy `protobuf:"varint,7,opt,name=node_taints_policy,json=nodeTaintsPolicy,proto3,enum=spark.NodeInclusionPolicy" json:"node_taints_policy,omitempty"` + MatchLabelKeys []string `protobuf:"bytes,8,rep,name=match_label_keys,json=matchLabelKeys,proto3" json:"match_label_keys,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TopologySpreadConstraint) Reset() { + *x = TopologySpreadConstraint{} + mi := &file_proto_spark_submit_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TopologySpreadConstraint) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TopologySpreadConstraint) ProtoMessage() {} + +func (x *TopologySpreadConstraint) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[19] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TopologySpreadConstraint.ProtoReflect.Descriptor instead. +func (*TopologySpreadConstraint) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{19} +} + +func (x *TopologySpreadConstraint) GetMaxSkew() int32 { + if x != nil { + return x.MaxSkew + } + return 0 +} + +func (x *TopologySpreadConstraint) GetTopologyKey() string { + if x != nil { + return x.TopologyKey + } + return "" +} + +func (x *TopologySpreadConstraint) GetWhenUnsatisfiable() UnsatisfiableConstraintAction { + if x != nil { + return x.WhenUnsatisfiable + } + return UnsatisfiableConstraintAction_UNSATISFIABLE_CONSTRAINT_ACTION_UNSPECIFIED +} + +func (x *TopologySpreadConstraint) GetLabelSelector() *LabelSelector { + if x != nil { + return x.LabelSelector + } + return nil +} + +func (x *TopologySpreadConstraint) GetMinDomains() *wrapperspb.Int32Value { + if x != nil { + return x.MinDomains + } + return nil +} + +func (x *TopologySpreadConstraint) GetNodeAffinityPolicy() NodeInclusionPolicy { + if x != nil { + return x.NodeAffinityPolicy + } + return NodeInclusionPolicy_NODE_INCLUSION_POLICY_UNSPECIFIED +} + +func (x *TopologySpreadConstraint) GetNodeTaintsPolicy() NodeInclusionPolicy { + if x != nil { + return x.NodeTaintsPolicy + } + return NodeInclusionPolicy_NODE_INCLUSION_POLICY_UNSPECIFIED +} + +func (x *TopologySpreadConstraint) GetMatchLabelKeys() []string { + if x != nil { + return x.MatchLabelKeys + } + return nil +} + +type PodSchedulingGate struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PodSchedulingGate) Reset() { + *x = PodSchedulingGate{} + mi := &file_proto_spark_submit_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PodSchedulingGate) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PodSchedulingGate) ProtoMessage() {} + +func (x *PodSchedulingGate) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[20] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PodSchedulingGate.ProtoReflect.Descriptor instead. +func (*PodSchedulingGate) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{20} +} + +func (x *PodSchedulingGate) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type PodOS struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PodOS) Reset() { + *x = PodOS{} + mi := &file_proto_spark_submit_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PodOS) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PodOS) ProtoMessage() {} + +func (x *PodOS) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[21] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PodOS.ProtoReflect.Descriptor instead. +func (*PodOS) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{21} +} + +func (x *PodOS) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type PodResourceClaim struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Source *ClaimSource `protobuf:"bytes,2,opt,name=source,proto3" json:"source,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PodResourceClaim) Reset() { + *x = PodResourceClaim{} + mi := &file_proto_spark_submit_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PodResourceClaim) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PodResourceClaim) ProtoMessage() {} + +func (x *PodResourceClaim) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[22] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PodResourceClaim.ProtoReflect.Descriptor instead. +func (*PodResourceClaim) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{22} +} + +func (x *PodResourceClaim) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *PodResourceClaim) GetSource() *ClaimSource { + if x != nil { + return x.Source + } + return nil +} + +type ClaimSource struct { + state protoimpl.MessageState `protogen:"open.v1"` + ResourceClaimName *wrapperspb.StringValue `protobuf:"bytes,1,opt,name=resource_claim_name,json=resourceClaimName,proto3" json:"resource_claim_name,omitempty"` + ResourceClaimTemplateName *wrapperspb.StringValue `protobuf:"bytes,2,opt,name=resource_claim_template_name,json=resourceClaimTemplateName,proto3" json:"resource_claim_template_name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ClaimSource) Reset() { + *x = ClaimSource{} + mi := &file_proto_spark_submit_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ClaimSource) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ClaimSource) ProtoMessage() {} + +func (x *ClaimSource) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[23] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ClaimSource.ProtoReflect.Descriptor instead. +func (*ClaimSource) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{23} +} + +func (x *ClaimSource) GetResourceClaimName() *wrapperspb.StringValue { + if x != nil { + return x.ResourceClaimName + } + return nil +} + +func (x *ClaimSource) GetResourceClaimTemplateName() *wrapperspb.StringValue { + if x != nil { + return x.ResourceClaimTemplateName + } + return nil +} + +type GPUSpec struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Quantity int64 `protobuf:"varint,2,opt,name=quantity,proto3" json:"quantity,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *GPUSpec) Reset() { + *x = GPUSpec{} + mi := &file_proto_spark_submit_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *GPUSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GPUSpec) ProtoMessage() {} + +func (x *GPUSpec) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[24] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use GPUSpec.ProtoReflect.Descriptor instead. +func (*GPUSpec) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{24} +} + +func (x *GPUSpec) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *GPUSpec) GetQuantity() int64 { + if x != nil { + return x.Quantity + } + return 0 +} + +type NamePath struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *NamePath) Reset() { + *x = NamePath{} + mi := &file_proto_spark_submit_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *NamePath) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NamePath) ProtoMessage() {} + +func (x *NamePath) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[25] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NamePath.ProtoReflect.Descriptor instead. +func (*NamePath) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{25} +} + +func (x *NamePath) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *NamePath) GetPath() string { + if x != nil { + return x.Path + } + return "" +} + +type SecretInfo struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` + Type SecretType `protobuf:"varint,3,opt,name=type,proto3,enum=spark.SecretType" json:"type,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SecretInfo) Reset() { + *x = SecretInfo{} + mi := &file_proto_spark_submit_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SecretInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SecretInfo) ProtoMessage() {} + +func (x *SecretInfo) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[26] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SecretInfo.ProtoReflect.Descriptor instead. +func (*SecretInfo) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{26} +} + +func (x *SecretInfo) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *SecretInfo) GetPath() string { + if x != nil { + return x.Path + } + return "" +} + +func (x *SecretInfo) GetType() SecretType { + if x != nil { + return x.Type + } + return SecretType_SECRET_TYPE_UNSPECIFIED +} + +type Affinity struct { + state protoimpl.MessageState `protogen:"open.v1"` + NodeAffinity *NodeAffinity `protobuf:"bytes,1,opt,name=node_affinity,json=nodeAffinity,proto3" json:"node_affinity,omitempty"` + PodAffinity *PodAffinity `protobuf:"bytes,2,opt,name=pod_affinity,json=podAffinity,proto3" json:"pod_affinity,omitempty"` + PodAntiAffinity *PodAntiAffinity `protobuf:"bytes,3,opt,name=pod_anti_affinity,json=podAntiAffinity,proto3" json:"pod_anti_affinity,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Affinity) Reset() { + *x = Affinity{} + mi := &file_proto_spark_submit_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Affinity) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Affinity) ProtoMessage() {} + +func (x *Affinity) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[27] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Affinity.ProtoReflect.Descriptor instead. +func (*Affinity) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{27} +} + +func (x *Affinity) GetNodeAffinity() *NodeAffinity { + if x != nil { + return x.NodeAffinity + } + return nil +} + +func (x *Affinity) GetPodAffinity() *PodAffinity { + if x != nil { + return x.PodAffinity + } + return nil +} + +func (x *Affinity) GetPodAntiAffinity() *PodAntiAffinity { + if x != nil { + return x.PodAntiAffinity + } + return nil +} + +type PodAntiAffinity struct { + state protoimpl.MessageState `protogen:"open.v1"` + LabelSelector *LabelSelector `protobuf:"bytes,1,opt,name=label_selector,json=labelSelector,proto3" json:"label_selector,omitempty"` + Namespaces []string `protobuf:"bytes,2,rep,name=namespaces,proto3" json:"namespaces,omitempty"` + TopologyKey string `protobuf:"bytes,3,opt,name=topology_key,json=topologyKey,proto3" json:"topology_key,omitempty"` + NamespaceSelector *LabelSelector `protobuf:"bytes,4,opt,name=namespace_selector,json=namespaceSelector,proto3" json:"namespace_selector,omitempty"` + MatchLabelKeys []string `protobuf:"bytes,5,rep,name=match_label_keys,json=matchLabelKeys,proto3" json:"match_label_keys,omitempty"` + MismatchLabelKeys []string `protobuf:"bytes,6,rep,name=mismatch_label_keys,json=mismatchLabelKeys,proto3" json:"mismatch_label_keys,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PodAntiAffinity) Reset() { + *x = PodAntiAffinity{} + mi := &file_proto_spark_submit_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PodAntiAffinity) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PodAntiAffinity) ProtoMessage() {} + +func (x *PodAntiAffinity) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[28] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PodAntiAffinity.ProtoReflect.Descriptor instead. +func (*PodAntiAffinity) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{28} +} + +func (x *PodAntiAffinity) GetLabelSelector() *LabelSelector { + if x != nil { + return x.LabelSelector + } + return nil +} + +func (x *PodAntiAffinity) GetNamespaces() []string { + if x != nil { + return x.Namespaces + } + return nil +} + +func (x *PodAntiAffinity) GetTopologyKey() string { + if x != nil { + return x.TopologyKey + } + return "" +} + +func (x *PodAntiAffinity) GetNamespaceSelector() *LabelSelector { + if x != nil { + return x.NamespaceSelector + } + return nil +} + +func (x *PodAntiAffinity) GetMatchLabelKeys() []string { + if x != nil { + return x.MatchLabelKeys + } + return nil +} + +func (x *PodAntiAffinity) GetMismatchLabelKeys() []string { + if x != nil { + return x.MismatchLabelKeys + } + return nil +} + +type PodAffinity struct { + state protoimpl.MessageState `protogen:"open.v1"` + RequiredDuringSchedulingIgnoredDuringExecution []*PodAffinityTerm `protobuf:"bytes,1,rep,name=required_during_scheduling_ignored_during_execution,json=requiredDuringSchedulingIgnoredDuringExecution,proto3" json:"required_during_scheduling_ignored_during_execution,omitempty"` + PreferredDuringSchedulingIgnoredDuringExecution []*WeightedPodAffinityTerm `protobuf:"bytes,2,rep,name=preferred_during_scheduling_ignored_during_execution,json=preferredDuringSchedulingIgnoredDuringExecution,proto3" json:"preferred_during_scheduling_ignored_during_execution,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PodAffinity) Reset() { + *x = PodAffinity{} + mi := &file_proto_spark_submit_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PodAffinity) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PodAffinity) ProtoMessage() {} + +func (x *PodAffinity) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[29] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PodAffinity.ProtoReflect.Descriptor instead. +func (*PodAffinity) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{29} +} + +func (x *PodAffinity) GetRequiredDuringSchedulingIgnoredDuringExecution() []*PodAffinityTerm { + if x != nil { + return x.RequiredDuringSchedulingIgnoredDuringExecution + } + return nil +} + +func (x *PodAffinity) GetPreferredDuringSchedulingIgnoredDuringExecution() []*WeightedPodAffinityTerm { + if x != nil { + return x.PreferredDuringSchedulingIgnoredDuringExecution + } + return nil +} + +type WeightedPodAffinityTerm struct { + state protoimpl.MessageState `protogen:"open.v1"` + Weight int32 `protobuf:"varint,1,opt,name=weight,proto3" json:"weight,omitempty"` + PodAffinityTerm *PodAffinityTerm `protobuf:"bytes,2,opt,name=pod_affinity_term,json=podAffinityTerm,proto3" json:"pod_affinity_term,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *WeightedPodAffinityTerm) Reset() { + *x = WeightedPodAffinityTerm{} + mi := &file_proto_spark_submit_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *WeightedPodAffinityTerm) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WeightedPodAffinityTerm) ProtoMessage() {} + +func (x *WeightedPodAffinityTerm) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[30] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WeightedPodAffinityTerm.ProtoReflect.Descriptor instead. +func (*WeightedPodAffinityTerm) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{30} +} + +func (x *WeightedPodAffinityTerm) GetWeight() int32 { + if x != nil { + return x.Weight + } + return 0 +} + +func (x *WeightedPodAffinityTerm) GetPodAffinityTerm() *PodAffinityTerm { + if x != nil { + return x.PodAffinityTerm + } + return nil +} + +type PodAffinityTerm struct { + state protoimpl.MessageState `protogen:"open.v1"` + LabelSelector *LabelSelector `protobuf:"bytes,1,opt,name=label_selector,json=labelSelector,proto3" json:"label_selector,omitempty"` + Namespaces []string `protobuf:"bytes,2,rep,name=namespaces,proto3" json:"namespaces,omitempty"` + TopologyKey string `protobuf:"bytes,3,opt,name=topology_key,json=topologyKey,proto3" json:"topology_key,omitempty"` + NamespaceSelector *LabelSelector `protobuf:"bytes,4,opt,name=namespace_selector,json=namespaceSelector,proto3" json:"namespace_selector,omitempty"` + MatchLabelKeys []string `protobuf:"bytes,5,rep,name=match_label_keys,json=matchLabelKeys,proto3" json:"match_label_keys,omitempty"` + MismatchLabelKeys []string `protobuf:"bytes,6,rep,name=mismatch_label_keys,json=mismatchLabelKeys,proto3" json:"mismatch_label_keys,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PodAffinityTerm) Reset() { + *x = PodAffinityTerm{} + mi := &file_proto_spark_submit_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PodAffinityTerm) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PodAffinityTerm) ProtoMessage() {} + +func (x *PodAffinityTerm) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[31] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PodAffinityTerm.ProtoReflect.Descriptor instead. +func (*PodAffinityTerm) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{31} +} + +func (x *PodAffinityTerm) GetLabelSelector() *LabelSelector { + if x != nil { + return x.LabelSelector + } + return nil +} + +func (x *PodAffinityTerm) GetNamespaces() []string { + if x != nil { + return x.Namespaces + } + return nil +} + +func (x *PodAffinityTerm) GetTopologyKey() string { + if x != nil { + return x.TopologyKey + } + return "" +} + +func (x *PodAffinityTerm) GetNamespaceSelector() *LabelSelector { + if x != nil { + return x.NamespaceSelector + } + return nil +} + +func (x *PodAffinityTerm) GetMatchLabelKeys() []string { + if x != nil { + return x.MatchLabelKeys + } + return nil +} + +func (x *PodAffinityTerm) GetMismatchLabelKeys() []string { + if x != nil { + return x.MismatchLabelKeys + } + return nil +} + +type LabelSelector struct { + state protoimpl.MessageState `protogen:"open.v1"` + MatchLabels map[string]string `protobuf:"bytes,1,rep,name=match_labels,json=matchLabels,proto3" json:"match_labels,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + MatchExpressions *LabelSelectorRequirement `protobuf:"bytes,2,opt,name=match_expressions,json=matchExpressions,proto3" json:"match_expressions,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LabelSelector) Reset() { + *x = LabelSelector{} + mi := &file_proto_spark_submit_proto_msgTypes[32] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LabelSelector) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LabelSelector) ProtoMessage() {} + +func (x *LabelSelector) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[32] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LabelSelector.ProtoReflect.Descriptor instead. +func (*LabelSelector) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{32} +} + +func (x *LabelSelector) GetMatchLabels() map[string]string { + if x != nil { + return x.MatchLabels + } + return nil +} + +func (x *LabelSelector) GetMatchExpressions() *LabelSelectorRequirement { + if x != nil { + return x.MatchExpressions + } + return nil +} + +type LabelSelectorRequirement struct { + state protoimpl.MessageState `protogen:"open.v1"` + Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Operator LabelSelectorOperator `protobuf:"varint,2,opt,name=operator,proto3,enum=spark.LabelSelectorOperator" json:"operator,omitempty"` + Values []string `protobuf:"bytes,3,rep,name=values,proto3" json:"values,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LabelSelectorRequirement) Reset() { + *x = LabelSelectorRequirement{} + mi := &file_proto_spark_submit_proto_msgTypes[33] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LabelSelectorRequirement) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LabelSelectorRequirement) ProtoMessage() {} + +func (x *LabelSelectorRequirement) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[33] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LabelSelectorRequirement.ProtoReflect.Descriptor instead. +func (*LabelSelectorRequirement) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{33} +} + +func (x *LabelSelectorRequirement) GetKey() string { + if x != nil { + return x.Key + } + return "" +} + +func (x *LabelSelectorRequirement) GetOperator() LabelSelectorOperator { + if x != nil { + return x.Operator + } + return LabelSelectorOperator_LABEL_SELECTOR_OPERATOR_UNSPECIFIED +} + +func (x *LabelSelectorRequirement) GetValues() []string { + if x != nil { + return x.Values + } + return nil +} + +type NodeAffinity struct { + state protoimpl.MessageState `protogen:"open.v1"` + RequiredDuringSchedulingIgnoredDuringExecution *NodeSelector `protobuf:"bytes,1,opt,name=required_during_scheduling_ignored_during_execution,json=requiredDuringSchedulingIgnoredDuringExecution,proto3" json:"required_during_scheduling_ignored_during_execution,omitempty"` + PreferredDuringSchedulingIgnoredDuringExecution []*PreferredSchedulingTerm `protobuf:"bytes,2,rep,name=preferred_during_scheduling_ignored_during_execution,json=preferredDuringSchedulingIgnoredDuringExecution,proto3" json:"preferred_during_scheduling_ignored_during_execution,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *NodeAffinity) Reset() { + *x = NodeAffinity{} + mi := &file_proto_spark_submit_proto_msgTypes[34] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *NodeAffinity) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NodeAffinity) ProtoMessage() {} + +func (x *NodeAffinity) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[34] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NodeAffinity.ProtoReflect.Descriptor instead. +func (*NodeAffinity) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{34} +} + +func (x *NodeAffinity) GetRequiredDuringSchedulingIgnoredDuringExecution() *NodeSelector { + if x != nil { + return x.RequiredDuringSchedulingIgnoredDuringExecution + } + return nil +} + +func (x *NodeAffinity) GetPreferredDuringSchedulingIgnoredDuringExecution() []*PreferredSchedulingTerm { + if x != nil { + return x.PreferredDuringSchedulingIgnoredDuringExecution + } + return nil +} + +type PreferredSchedulingTerm struct { + state protoimpl.MessageState `protogen:"open.v1"` + Weight int32 `protobuf:"varint,1,opt,name=weight,proto3" json:"weight,omitempty"` + Preference *NodeSelectorTerm `protobuf:"bytes,2,opt,name=preference,proto3" json:"preference,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PreferredSchedulingTerm) Reset() { + *x = PreferredSchedulingTerm{} + mi := &file_proto_spark_submit_proto_msgTypes[35] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PreferredSchedulingTerm) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PreferredSchedulingTerm) ProtoMessage() {} + +func (x *PreferredSchedulingTerm) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[35] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PreferredSchedulingTerm.ProtoReflect.Descriptor instead. +func (*PreferredSchedulingTerm) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{35} +} + +func (x *PreferredSchedulingTerm) GetWeight() int32 { + if x != nil { + return x.Weight + } + return 0 +} + +func (x *PreferredSchedulingTerm) GetPreference() *NodeSelectorTerm { + if x != nil { + return x.Preference + } + return nil +} + +type NodeSelector struct { + state protoimpl.MessageState `protogen:"open.v1"` + NodeSelectorTerms []*NodeSelectorTerm `protobuf:"bytes,1,rep,name=node_selector_terms,json=nodeSelectorTerms,proto3" json:"node_selector_terms,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *NodeSelector) Reset() { + *x = NodeSelector{} + mi := &file_proto_spark_submit_proto_msgTypes[36] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *NodeSelector) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NodeSelector) ProtoMessage() {} + +func (x *NodeSelector) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[36] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NodeSelector.ProtoReflect.Descriptor instead. +func (*NodeSelector) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{36} +} + +func (x *NodeSelector) GetNodeSelectorTerms() []*NodeSelectorTerm { + if x != nil { + return x.NodeSelectorTerms + } + return nil +} + +type NodeSelectorTerm struct { + state protoimpl.MessageState `protogen:"open.v1"` + MatchExpressions []*NodeSelectorRequirement `protobuf:"bytes,1,rep,name=match_expressions,json=matchExpressions,proto3" json:"match_expressions,omitempty"` + MatchFields []*NodeSelectorRequirement `protobuf:"bytes,2,rep,name=match_fields,json=matchFields,proto3" json:"match_fields,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *NodeSelectorTerm) Reset() { + *x = NodeSelectorTerm{} + mi := &file_proto_spark_submit_proto_msgTypes[37] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *NodeSelectorTerm) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NodeSelectorTerm) ProtoMessage() {} + +func (x *NodeSelectorTerm) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[37] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NodeSelectorTerm.ProtoReflect.Descriptor instead. +func (*NodeSelectorTerm) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{37} +} + +func (x *NodeSelectorTerm) GetMatchExpressions() []*NodeSelectorRequirement { + if x != nil { + return x.MatchExpressions + } + return nil +} + +func (x *NodeSelectorTerm) GetMatchFields() []*NodeSelectorRequirement { + if x != nil { + return x.MatchFields + } + return nil +} + +type NodeSelectorRequirement struct { + state protoimpl.MessageState `protogen:"open.v1"` + Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Operator NodeSelectorOperator `protobuf:"varint,2,opt,name=operator,proto3,enum=spark.NodeSelectorOperator" json:"operator,omitempty"` + Values []string `protobuf:"bytes,3,rep,name=values,proto3" json:"values,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *NodeSelectorRequirement) Reset() { + *x = NodeSelectorRequirement{} + mi := &file_proto_spark_submit_proto_msgTypes[38] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *NodeSelectorRequirement) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NodeSelectorRequirement) ProtoMessage() {} + +func (x *NodeSelectorRequirement) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[38] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NodeSelectorRequirement.ProtoReflect.Descriptor instead. +func (*NodeSelectorRequirement) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{38} +} + +func (x *NodeSelectorRequirement) GetKey() string { + if x != nil { + return x.Key + } + return "" +} + +func (x *NodeSelectorRequirement) GetOperator() NodeSelectorOperator { + if x != nil { + return x.Operator + } + return NodeSelectorOperator_NODE_SELECTOR_OPERATOR_UNSPECIFIED +} + +func (x *NodeSelectorRequirement) GetValues() []string { + if x != nil { + return x.Values + } + return nil +} + +type Toleration struct { + state protoimpl.MessageState `protogen:"open.v1"` + Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Operator TolerationOperator `protobuf:"varint,2,opt,name=operator,proto3,enum=spark.TolerationOperator" json:"operator,omitempty"` + Value string `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` + Effect TaintEffect `protobuf:"varint,4,opt,name=effect,proto3,enum=spark.TaintEffect" json:"effect,omitempty"` + TolerationSeconds *wrapperspb.Int64Value `protobuf:"bytes,5,opt,name=toleration_seconds,json=tolerationSeconds,proto3" json:"toleration_seconds,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Toleration) Reset() { + *x = Toleration{} + mi := &file_proto_spark_submit_proto_msgTypes[39] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Toleration) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Toleration) ProtoMessage() {} + +func (x *Toleration) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[39] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Toleration.ProtoReflect.Descriptor instead. +func (*Toleration) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{39} +} + +func (x *Toleration) GetKey() string { + if x != nil { + return x.Key + } + return "" +} + +func (x *Toleration) GetOperator() TolerationOperator { + if x != nil { + return x.Operator + } + return TolerationOperator_TOLERATION_OPERATOR_UNSPECIFIED +} + +func (x *Toleration) GetValue() string { + if x != nil { + return x.Value + } + return "" +} + +func (x *Toleration) GetEffect() TaintEffect { + if x != nil { + return x.Effect + } + return TaintEffect_TAINT_EFFECT_UNSPECIFIED +} + +func (x *Toleration) GetTolerationSeconds() *wrapperspb.Int64Value { + if x != nil { + return x.TolerationSeconds + } + return nil +} + +type PodSecurityContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + SeLinuxOptions *SELinuxOptions `protobuf:"bytes,1,opt,name=se_linux_options,json=seLinuxOptions,proto3" json:"se_linux_options,omitempty"` + WindowsSecurityContextOptions *WindowsSecurityContextOptions `protobuf:"bytes,2,opt,name=windows_security_context_options,json=windowsSecurityContextOptions,proto3" json:"windows_security_context_options,omitempty"` + RunAsUser *wrapperspb.Int64Value `protobuf:"bytes,3,opt,name=run_as_user,json=runAsUser,proto3" json:"run_as_user,omitempty"` + RunAsGroup *wrapperspb.Int64Value `protobuf:"bytes,4,opt,name=run_as_group,json=runAsGroup,proto3" json:"run_as_group,omitempty"` + RunAsNonroot *wrapperspb.BoolValue `protobuf:"bytes,5,opt,name=run_as_nonroot,json=runAsNonroot,proto3" json:"run_as_nonroot,omitempty"` + SupplementalGroups []int64 `protobuf:"varint,6,rep,packed,name=supplemental_groups,json=supplementalGroups,proto3" json:"supplemental_groups,omitempty"` + FsGroup *wrapperspb.Int64Value `protobuf:"bytes,7,opt,name=fs_group,json=fsGroup,proto3" json:"fs_group,omitempty"` + SysCtl []*Sysctl `protobuf:"bytes,8,rep,name=sys_ctl,json=sysCtl,proto3" json:"sys_ctl,omitempty"` + FsGroupChangePolicy PodFSGroupChangePolicy `protobuf:"varint,9,opt,name=fs_group_change_policy,json=fsGroupChangePolicy,proto3,enum=spark.PodFSGroupChangePolicy" json:"fs_group_change_policy,omitempty"` + SecCompProfile *SeccompProfile `protobuf:"bytes,10,opt,name=sec_comp_profile,json=secCompProfile,proto3" json:"sec_comp_profile,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PodSecurityContext) Reset() { + *x = PodSecurityContext{} + mi := &file_proto_spark_submit_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PodSecurityContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PodSecurityContext) ProtoMessage() {} + +func (x *PodSecurityContext) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[40] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PodSecurityContext.ProtoReflect.Descriptor instead. +func (*PodSecurityContext) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{40} +} + +func (x *PodSecurityContext) GetSeLinuxOptions() *SELinuxOptions { + if x != nil { + return x.SeLinuxOptions + } + return nil +} + +func (x *PodSecurityContext) GetWindowsSecurityContextOptions() *WindowsSecurityContextOptions { + if x != nil { + return x.WindowsSecurityContextOptions + } + return nil +} + +func (x *PodSecurityContext) GetRunAsUser() *wrapperspb.Int64Value { + if x != nil { + return x.RunAsUser + } + return nil +} + +func (x *PodSecurityContext) GetRunAsGroup() *wrapperspb.Int64Value { + if x != nil { + return x.RunAsGroup + } + return nil +} + +func (x *PodSecurityContext) GetRunAsNonroot() *wrapperspb.BoolValue { + if x != nil { + return x.RunAsNonroot + } + return nil +} + +func (x *PodSecurityContext) GetSupplementalGroups() []int64 { + if x != nil { + return x.SupplementalGroups + } + return nil +} + +func (x *PodSecurityContext) GetFsGroup() *wrapperspb.Int64Value { + if x != nil { + return x.FsGroup + } + return nil +} + +func (x *PodSecurityContext) GetSysCtl() []*Sysctl { + if x != nil { + return x.SysCtl + } + return nil +} + +func (x *PodSecurityContext) GetFsGroupChangePolicy() PodFSGroupChangePolicy { + if x != nil { + return x.FsGroupChangePolicy + } + return PodFSGroupChangePolicy_POD_FS_GROUP_CHANGE_POLICY_UNSPECIFIED +} + +func (x *PodSecurityContext) GetSecCompProfile() *SeccompProfile { + if x != nil { + return x.SecCompProfile + } + return nil +} + +type Sysctl struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Sysctl) Reset() { + *x = Sysctl{} + mi := &file_proto_spark_submit_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Sysctl) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Sysctl) ProtoMessage() {} + +func (x *Sysctl) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[41] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Sysctl.ProtoReflect.Descriptor instead. +func (*Sysctl) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{41} +} + +func (x *Sysctl) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Sysctl) GetValue() string { + if x != nil { + return x.Value + } + return "" +} + +type Container struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Image string `protobuf:"bytes,2,opt,name=image,proto3" json:"image,omitempty"` + Command []string `protobuf:"bytes,3,rep,name=command,proto3" json:"command,omitempty"` + Args []string `protobuf:"bytes,4,rep,name=args,proto3" json:"args,omitempty"` + WorkingDir string `protobuf:"bytes,5,opt,name=working_dir,json=workingDir,proto3" json:"working_dir,omitempty"` + Ports []*ContainerPort `protobuf:"bytes,6,rep,name=ports,proto3" json:"ports,omitempty"` + EnvFrom []*EnvFromSource `protobuf:"bytes,7,rep,name=env_from,json=envFrom,proto3" json:"env_from,omitempty"` + Env []*EnvVar `protobuf:"bytes,8,rep,name=env,proto3" json:"env,omitempty"` + Resources *ResourceRequirements `protobuf:"bytes,9,opt,name=resources,proto3" json:"resources,omitempty"` + ResizePolicy []*ContainerResizePolicy `protobuf:"bytes,10,rep,name=resize_policy,json=resizePolicy,proto3" json:"resize_policy,omitempty"` + RestartPolicy ContainerRestartPolicy `protobuf:"varint,11,opt,name=restart_policy,json=restartPolicy,proto3,enum=spark.ContainerRestartPolicy" json:"restart_policy,omitempty"` + VolumeMounts []*VolumeMount `protobuf:"bytes,12,rep,name=volume_mounts,json=volumeMounts,proto3" json:"volume_mounts,omitempty"` + VolumeDevices []*VolumeDevice `protobuf:"bytes,13,rep,name=volume_devices,json=volumeDevices,proto3" json:"volume_devices,omitempty"` + LivenessProbe *Probe `protobuf:"bytes,14,opt,name=liveness_probe,json=livenessProbe,proto3" json:"liveness_probe,omitempty"` + ReadinessProbe *Probe `protobuf:"bytes,15,opt,name=readiness_probe,json=readinessProbe,proto3" json:"readiness_probe,omitempty"` + StartupProbe *Probe `protobuf:"bytes,16,opt,name=startup_probe,json=startupProbe,proto3" json:"startup_probe,omitempty"` + LifeCycle *Lifecycle `protobuf:"bytes,17,opt,name=life_cycle,json=lifeCycle,proto3" json:"life_cycle,omitempty"` + TerminationMessagePath string `protobuf:"bytes,18,opt,name=termination_message_path,json=terminationMessagePath,proto3" json:"termination_message_path,omitempty"` + TerminationMessagePolicy TerminationMessagePolicy `protobuf:"varint,19,opt,name=termination_message_policy,json=terminationMessagePolicy,proto3,enum=spark.TerminationMessagePolicy" json:"termination_message_policy,omitempty"` + ImagePullPolicy PullPolicy `protobuf:"varint,20,opt,name=image_pull_policy,json=imagePullPolicy,proto3,enum=spark.PullPolicy" json:"image_pull_policy,omitempty"` + SecurityContext *SecurityContext `protobuf:"bytes,21,opt,name=security_context,json=securityContext,proto3" json:"security_context,omitempty"` + Stdin bool `protobuf:"varint,22,opt,name=stdin,proto3" json:"stdin,omitempty"` + StdinOnce bool `protobuf:"varint,23,opt,name=stdin_once,json=stdinOnce,proto3" json:"stdin_once,omitempty"` + Tty bool `protobuf:"varint,24,opt,name=tty,proto3" json:"tty,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Container) Reset() { + *x = Container{} + mi := &file_proto_spark_submit_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Container) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Container) ProtoMessage() {} + +func (x *Container) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[42] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Container.ProtoReflect.Descriptor instead. +func (*Container) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{42} +} + +func (x *Container) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Container) GetImage() string { + if x != nil { + return x.Image + } + return "" +} + +func (x *Container) GetCommand() []string { + if x != nil { + return x.Command + } + return nil +} + +func (x *Container) GetArgs() []string { + if x != nil { + return x.Args + } + return nil +} + +func (x *Container) GetWorkingDir() string { + if x != nil { + return x.WorkingDir + } + return "" +} + +func (x *Container) GetPorts() []*ContainerPort { + if x != nil { + return x.Ports + } + return nil +} + +func (x *Container) GetEnvFrom() []*EnvFromSource { + if x != nil { + return x.EnvFrom + } + return nil +} + +func (x *Container) GetEnv() []*EnvVar { + if x != nil { + return x.Env + } + return nil +} + +func (x *Container) GetResources() *ResourceRequirements { + if x != nil { + return x.Resources + } + return nil +} + +func (x *Container) GetResizePolicy() []*ContainerResizePolicy { + if x != nil { + return x.ResizePolicy + } + return nil +} + +func (x *Container) GetRestartPolicy() ContainerRestartPolicy { + if x != nil { + return x.RestartPolicy + } + return ContainerRestartPolicy_CONTAINER_RESTART_POLICY_UNSPECIFIED +} + +func (x *Container) GetVolumeMounts() []*VolumeMount { + if x != nil { + return x.VolumeMounts + } + return nil +} + +func (x *Container) GetVolumeDevices() []*VolumeDevice { + if x != nil { + return x.VolumeDevices + } + return nil +} + +func (x *Container) GetLivenessProbe() *Probe { + if x != nil { + return x.LivenessProbe + } + return nil +} + +func (x *Container) GetReadinessProbe() *Probe { + if x != nil { + return x.ReadinessProbe + } + return nil +} + +func (x *Container) GetStartupProbe() *Probe { + if x != nil { + return x.StartupProbe + } + return nil +} + +func (x *Container) GetLifeCycle() *Lifecycle { + if x != nil { + return x.LifeCycle + } + return nil +} + +func (x *Container) GetTerminationMessagePath() string { + if x != nil { + return x.TerminationMessagePath + } + return "" +} + +func (x *Container) GetTerminationMessagePolicy() TerminationMessagePolicy { + if x != nil { + return x.TerminationMessagePolicy + } + return TerminationMessagePolicy_TERMINATION_MESSAGE_POLICY_UNSPECIFIED +} + +func (x *Container) GetImagePullPolicy() PullPolicy { + if x != nil { + return x.ImagePullPolicy + } + return PullPolicy_PULL_POLICY_UNSPECIFIED +} + +func (x *Container) GetSecurityContext() *SecurityContext { + if x != nil { + return x.SecurityContext + } + return nil +} + +func (x *Container) GetStdin() bool { + if x != nil { + return x.Stdin + } + return false +} + +func (x *Container) GetStdinOnce() bool { + if x != nil { + return x.StdinOnce + } + return false +} + +func (x *Container) GetTty() bool { + if x != nil { + return x.Tty + } + return false +} + +type ContainerPort struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + HostPort int32 `protobuf:"varint,2,opt,name=host_port,json=hostPort,proto3" json:"host_port,omitempty"` + ContainerPort int32 `protobuf:"varint,3,opt,name=container_port,json=containerPort,proto3" json:"container_port,omitempty"` + Protocol Protocol `protobuf:"varint,4,opt,name=protocol,proto3,enum=spark.Protocol" json:"protocol,omitempty"` + HostIp string `protobuf:"bytes,5,opt,name=host_ip,json=hostIp,proto3" json:"host_ip,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ContainerPort) Reset() { + *x = ContainerPort{} + mi := &file_proto_spark_submit_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ContainerPort) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ContainerPort) ProtoMessage() {} + +func (x *ContainerPort) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[43] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ContainerPort.ProtoReflect.Descriptor instead. +func (*ContainerPort) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{43} +} + +func (x *ContainerPort) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *ContainerPort) GetHostPort() int32 { + if x != nil { + return x.HostPort + } + return 0 +} + +func (x *ContainerPort) GetContainerPort() int32 { + if x != nil { + return x.ContainerPort + } + return 0 +} + +func (x *ContainerPort) GetProtocol() Protocol { + if x != nil { + return x.Protocol + } + return Protocol_PROTOCOL_UNSPECIFIED +} + +func (x *ContainerPort) GetHostIp() string { + if x != nil { + return x.HostIp + } + return "" +} + +type ConfigMapEnvSource struct { + state protoimpl.MessageState `protogen:"open.v1"` + LocalObjectReference *LocalObjectReference `protobuf:"bytes,1,opt,name=local_object_reference,json=localObjectReference,proto3" json:"local_object_reference,omitempty"` + Optional *wrapperspb.BoolValue `protobuf:"bytes,2,opt,name=optional,proto3" json:"optional,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ConfigMapEnvSource) Reset() { + *x = ConfigMapEnvSource{} + mi := &file_proto_spark_submit_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ConfigMapEnvSource) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ConfigMapEnvSource) ProtoMessage() {} + +func (x *ConfigMapEnvSource) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[44] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ConfigMapEnvSource.ProtoReflect.Descriptor instead. +func (*ConfigMapEnvSource) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{44} +} + +func (x *ConfigMapEnvSource) GetLocalObjectReference() *LocalObjectReference { + if x != nil { + return x.LocalObjectReference + } + return nil +} + +func (x *ConfigMapEnvSource) GetOptional() *wrapperspb.BoolValue { + if x != nil { + return x.Optional + } + return nil +} + +type EnvFromSource struct { + state protoimpl.MessageState `protogen:"open.v1"` + Prefix string `protobuf:"bytes,1,opt,name=prefix,proto3" json:"prefix,omitempty"` + ConfigMapRef *ConfigMapEnvSource `protobuf:"bytes,2,opt,name=config_map_ref,json=configMapRef,proto3" json:"config_map_ref,omitempty"` + SecretRef *SecretEnvSource `protobuf:"bytes,3,opt,name=secret_ref,json=secretRef,proto3" json:"secret_ref,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *EnvFromSource) Reset() { + *x = EnvFromSource{} + mi := &file_proto_spark_submit_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *EnvFromSource) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EnvFromSource) ProtoMessage() {} + +func (x *EnvFromSource) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[45] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EnvFromSource.ProtoReflect.Descriptor instead. +func (*EnvFromSource) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{45} +} + +func (x *EnvFromSource) GetPrefix() string { + if x != nil { + return x.Prefix + } + return "" +} + +func (x *EnvFromSource) GetConfigMapRef() *ConfigMapEnvSource { + if x != nil { + return x.ConfigMapRef + } + return nil +} + +func (x *EnvFromSource) GetSecretRef() *SecretEnvSource { + if x != nil { + return x.SecretRef + } + return nil +} + +type SecretEnvSource struct { + state protoimpl.MessageState `protogen:"open.v1"` + LocalObjectReference *LocalObjectReference `protobuf:"bytes,1,opt,name=local_object_reference,json=localObjectReference,proto3" json:"local_object_reference,omitempty"` + Optional *wrapperspb.BoolValue `protobuf:"bytes,3,opt,name=optional,proto3" json:"optional,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SecretEnvSource) Reset() { + *x = SecretEnvSource{} + mi := &file_proto_spark_submit_proto_msgTypes[46] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SecretEnvSource) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SecretEnvSource) ProtoMessage() {} + +func (x *SecretEnvSource) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[46] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SecretEnvSource.ProtoReflect.Descriptor instead. +func (*SecretEnvSource) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{46} +} + +func (x *SecretEnvSource) GetLocalObjectReference() *LocalObjectReference { + if x != nil { + return x.LocalObjectReference + } + return nil +} + +func (x *SecretEnvSource) GetOptional() *wrapperspb.BoolValue { + if x != nil { + return x.Optional + } + return nil +} + +type EnvVar struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + ValueFrom *EnvVarSource `protobuf:"bytes,3,opt,name=value_from,json=valueFrom,proto3" json:"value_from,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *EnvVar) Reset() { + *x = EnvVar{} + mi := &file_proto_spark_submit_proto_msgTypes[47] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *EnvVar) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EnvVar) ProtoMessage() {} + +func (x *EnvVar) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[47] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EnvVar.ProtoReflect.Descriptor instead. +func (*EnvVar) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{47} +} + +func (x *EnvVar) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *EnvVar) GetValue() string { + if x != nil { + return x.Value + } + return "" +} + +func (x *EnvVar) GetValueFrom() *EnvVarSource { + if x != nil { + return x.ValueFrom + } + return nil +} + +type EnvVarSource struct { + state protoimpl.MessageState `protogen:"open.v1"` + FieldRef *ObjectFieldSelector `protobuf:"bytes,1,opt,name=field_ref,json=fieldRef,proto3" json:"field_ref,omitempty"` + ResourceFieldRef *ResourceFieldSelector `protobuf:"bytes,2,opt,name=resource_field_ref,json=resourceFieldRef,proto3" json:"resource_field_ref,omitempty"` + ConfigMapKeyRef *ConfigMapKeySelector `protobuf:"bytes,3,opt,name=config_map_key_ref,json=configMapKeyRef,proto3" json:"config_map_key_ref,omitempty"` + SecretKeyRef *SecretKeySelector `protobuf:"bytes,4,opt,name=secret_key_ref,json=secretKeyRef,proto3" json:"secret_key_ref,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *EnvVarSource) Reset() { + *x = EnvVarSource{} + mi := &file_proto_spark_submit_proto_msgTypes[48] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *EnvVarSource) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EnvVarSource) ProtoMessage() {} + +func (x *EnvVarSource) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[48] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EnvVarSource.ProtoReflect.Descriptor instead. +func (*EnvVarSource) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{48} +} + +func (x *EnvVarSource) GetFieldRef() *ObjectFieldSelector { + if x != nil { + return x.FieldRef + } + return nil +} + +func (x *EnvVarSource) GetResourceFieldRef() *ResourceFieldSelector { + if x != nil { + return x.ResourceFieldRef + } + return nil +} + +func (x *EnvVarSource) GetConfigMapKeyRef() *ConfigMapKeySelector { + if x != nil { + return x.ConfigMapKeyRef + } + return nil +} + +func (x *EnvVarSource) GetSecretKeyRef() *SecretKeySelector { + if x != nil { + return x.SecretKeyRef + } + return nil +} + +type SecretKeySelector struct { + state protoimpl.MessageState `protogen:"open.v1"` + LocalObjectReference *LocalObjectReference `protobuf:"bytes,1,opt,name=local_object_reference,json=localObjectReference,proto3" json:"local_object_reference,omitempty"` + Key string `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` + Optional *wrapperspb.BoolValue `protobuf:"bytes,3,opt,name=optional,proto3" json:"optional,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SecretKeySelector) Reset() { + *x = SecretKeySelector{} + mi := &file_proto_spark_submit_proto_msgTypes[49] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SecretKeySelector) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SecretKeySelector) ProtoMessage() {} + +func (x *SecretKeySelector) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[49] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SecretKeySelector.ProtoReflect.Descriptor instead. +func (*SecretKeySelector) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{49} +} + +func (x *SecretKeySelector) GetLocalObjectReference() *LocalObjectReference { + if x != nil { + return x.LocalObjectReference + } + return nil +} + +func (x *SecretKeySelector) GetKey() string { + if x != nil { + return x.Key + } + return "" +} + +func (x *SecretKeySelector) GetOptional() *wrapperspb.BoolValue { + if x != nil { + return x.Optional + } + return nil +} + +type ConfigMapKeySelector struct { + state protoimpl.MessageState `protogen:"open.v1"` + LocalObjectReference *LocalObjectReference `protobuf:"bytes,1,opt,name=local_object_reference,json=localObjectReference,proto3" json:"local_object_reference,omitempty"` + Key string `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` + Optional *wrapperspb.BoolValue `protobuf:"bytes,3,opt,name=optional,proto3" json:"optional,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ConfigMapKeySelector) Reset() { + *x = ConfigMapKeySelector{} + mi := &file_proto_spark_submit_proto_msgTypes[50] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ConfigMapKeySelector) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ConfigMapKeySelector) ProtoMessage() {} + +func (x *ConfigMapKeySelector) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[50] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ConfigMapKeySelector.ProtoReflect.Descriptor instead. +func (*ConfigMapKeySelector) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{50} +} + +func (x *ConfigMapKeySelector) GetLocalObjectReference() *LocalObjectReference { + if x != nil { + return x.LocalObjectReference + } + return nil +} + +func (x *ConfigMapKeySelector) GetKey() string { + if x != nil { + return x.Key + } + return "" +} + +func (x *ConfigMapKeySelector) GetOptional() *wrapperspb.BoolValue { + if x != nil { + return x.Optional + } + return nil +} + +type LocalObjectReference struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LocalObjectReference) Reset() { + *x = LocalObjectReference{} + mi := &file_proto_spark_submit_proto_msgTypes[51] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LocalObjectReference) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LocalObjectReference) ProtoMessage() {} + +func (x *LocalObjectReference) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[51] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LocalObjectReference.ProtoReflect.Descriptor instead. +func (*LocalObjectReference) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{51} +} + +func (x *LocalObjectReference) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type ResourceFieldSelector struct { + state protoimpl.MessageState `protogen:"open.v1"` + ContainerName string `protobuf:"bytes,1,opt,name=container_name,json=containerName,proto3" json:"container_name,omitempty"` + Resource string `protobuf:"bytes,2,opt,name=resource,proto3" json:"resource,omitempty"` + Divisor *Quantity `protobuf:"bytes,3,opt,name=divisor,proto3" json:"divisor,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResourceFieldSelector) Reset() { + *x = ResourceFieldSelector{} + mi := &file_proto_spark_submit_proto_msgTypes[52] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResourceFieldSelector) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResourceFieldSelector) ProtoMessage() {} + +func (x *ResourceFieldSelector) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[52] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResourceFieldSelector.ProtoReflect.Descriptor instead. +func (*ResourceFieldSelector) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{52} +} + +func (x *ResourceFieldSelector) GetContainerName() string { + if x != nil { + return x.ContainerName + } + return "" +} + +func (x *ResourceFieldSelector) GetResource() string { + if x != nil { + return x.Resource + } + return "" +} + +func (x *ResourceFieldSelector) GetDivisor() *Quantity { + if x != nil { + return x.Divisor + } + return nil +} + +type ObjectFieldSelector struct { + state protoimpl.MessageState `protogen:"open.v1"` + ApiVersion string `protobuf:"bytes,1,opt,name=api_version,json=apiVersion,proto3" json:"api_version,omitempty"` + FieldPath string `protobuf:"bytes,2,opt,name=field_path,json=fieldPath,proto3" json:"field_path,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ObjectFieldSelector) Reset() { + *x = ObjectFieldSelector{} + mi := &file_proto_spark_submit_proto_msgTypes[53] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ObjectFieldSelector) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ObjectFieldSelector) ProtoMessage() {} + +func (x *ObjectFieldSelector) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[53] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ObjectFieldSelector.ProtoReflect.Descriptor instead. +func (*ObjectFieldSelector) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{53} +} + +func (x *ObjectFieldSelector) GetApiVersion() string { + if x != nil { + return x.ApiVersion + } + return "" +} + +func (x *ObjectFieldSelector) GetFieldPath() string { + if x != nil { + return x.FieldPath + } + return "" +} + +type ResourceRequirements struct { + state protoimpl.MessageState `protogen:"open.v1"` + Limits map[string]*Quantity `protobuf:"bytes,1,rep,name=limits,proto3" json:"limits,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + Requests map[string]*Quantity `protobuf:"bytes,2,rep,name=requests,proto3" json:"requests,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + Claims []*ResourceClaim `protobuf:"bytes,3,rep,name=claims,proto3" json:"claims,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResourceRequirements) Reset() { + *x = ResourceRequirements{} + mi := &file_proto_spark_submit_proto_msgTypes[54] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResourceRequirements) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResourceRequirements) ProtoMessage() {} + +func (x *ResourceRequirements) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[54] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResourceRequirements.ProtoReflect.Descriptor instead. +func (*ResourceRequirements) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{54} +} + +func (x *ResourceRequirements) GetLimits() map[string]*Quantity { + if x != nil { + return x.Limits + } + return nil +} + +func (x *ResourceRequirements) GetRequests() map[string]*Quantity { + if x != nil { + return x.Requests + } + return nil +} + +func (x *ResourceRequirements) GetClaims() []*ResourceClaim { + if x != nil { + return x.Claims + } + return nil +} + +type ResourceClaim struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResourceClaim) Reset() { + *x = ResourceClaim{} + mi := &file_proto_spark_submit_proto_msgTypes[55] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResourceClaim) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResourceClaim) ProtoMessage() {} + +func (x *ResourceClaim) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[55] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResourceClaim.ProtoReflect.Descriptor instead. +func (*ResourceClaim) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{55} +} + +func (x *ResourceClaim) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +type ResourceListEntry struct { + state protoimpl.MessageState `protogen:"open.v1"` + ResourceName string `protobuf:"bytes,1,opt,name=resource_name,json=resourceName,proto3" json:"resource_name,omitempty"` + Quantity *Quantity `protobuf:"bytes,2,opt,name=quantity,proto3" json:"quantity,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ResourceListEntry) Reset() { + *x = ResourceListEntry{} + mi := &file_proto_spark_submit_proto_msgTypes[56] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ResourceListEntry) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResourceListEntry) ProtoMessage() {} + +func (x *ResourceListEntry) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[56] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResourceListEntry.ProtoReflect.Descriptor instead. +func (*ResourceListEntry) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{56} +} + +func (x *ResourceListEntry) GetResourceName() string { + if x != nil { + return x.ResourceName + } + return "" +} + +func (x *ResourceListEntry) GetQuantity() *Quantity { + if x != nil { + return x.Quantity + } + return nil +} + +type Quantity struct { + state protoimpl.MessageState `protogen:"open.v1"` + I *Int64Amount `protobuf:"bytes,1,opt,name=i,proto3" json:"i,omitempty"` + D *InfDecAmount `protobuf:"bytes,2,opt,name=d,proto3" json:"d,omitempty"` + S string `protobuf:"bytes,3,opt,name=s,proto3" json:"s,omitempty"` + Format Format `protobuf:"varint,4,opt,name=format,proto3,enum=spark.Format" json:"format,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Quantity) Reset() { + *x = Quantity{} + mi := &file_proto_spark_submit_proto_msgTypes[57] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Quantity) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Quantity) ProtoMessage() {} + +func (x *Quantity) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[57] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Quantity.ProtoReflect.Descriptor instead. +func (*Quantity) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{57} +} + +func (x *Quantity) GetI() *Int64Amount { + if x != nil { + return x.I + } + return nil +} + +func (x *Quantity) GetD() *InfDecAmount { + if x != nil { + return x.D + } + return nil +} + +func (x *Quantity) GetS() string { + if x != nil { + return x.S + } + return "" +} + +func (x *Quantity) GetFormat() Format { + if x != nil { + return x.Format + } + return Format_FORMAT_UNSPECIFIED +} + +type InfDecAmount struct { + state protoimpl.MessageState `protogen:"open.v1"` + Dec string `protobuf:"bytes,1,opt,name=dec,proto3" json:"dec,omitempty"` // Representing *inf.Dec as a string + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *InfDecAmount) Reset() { + *x = InfDecAmount{} + mi := &file_proto_spark_submit_proto_msgTypes[58] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *InfDecAmount) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*InfDecAmount) ProtoMessage() {} + +func (x *InfDecAmount) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[58] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use InfDecAmount.ProtoReflect.Descriptor instead. +func (*InfDecAmount) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{58} +} + +func (x *InfDecAmount) GetDec() string { + if x != nil { + return x.Dec + } + return "" +} + +type Int64Amount struct { + state protoimpl.MessageState `protogen:"open.v1"` + Value int64 `protobuf:"varint,1,opt,name=value,proto3" json:"value,omitempty"` + Scale *Scale `protobuf:"bytes,2,opt,name=scale,proto3" json:"scale,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Int64Amount) Reset() { + *x = Int64Amount{} + mi := &file_proto_spark_submit_proto_msgTypes[59] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Int64Amount) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Int64Amount) ProtoMessage() {} + +func (x *Int64Amount) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[59] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Int64Amount.ProtoReflect.Descriptor instead. +func (*Int64Amount) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{59} +} + +func (x *Int64Amount) GetValue() int64 { + if x != nil { + return x.Value + } + return 0 +} + +func (x *Int64Amount) GetScale() *Scale { + if x != nil { + return x.Scale + } + return nil +} + +type Scale struct { + state protoimpl.MessageState `protogen:"open.v1"` + Value int32 `protobuf:"varint,1,opt,name=value,proto3" json:"value,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Scale) Reset() { + *x = Scale{} + mi := &file_proto_spark_submit_proto_msgTypes[60] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Scale) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Scale) ProtoMessage() {} + +func (x *Scale) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[60] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Scale.ProtoReflect.Descriptor instead. +func (*Scale) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{60} +} + +func (x *Scale) GetValue() int32 { + if x != nil { + return x.Value + } + return 0 +} + +type ContainerResizePolicy struct { + state protoimpl.MessageState `protogen:"open.v1"` + ResourceName string `protobuf:"bytes,1,opt,name=resource_name,json=resourceName,proto3" json:"resource_name,omitempty"` + RestartPolicy ResourceResizeRestartPolicy `protobuf:"varint,2,opt,name=restart_policy,json=restartPolicy,proto3,enum=spark.ResourceResizeRestartPolicy" json:"restart_policy,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ContainerResizePolicy) Reset() { + *x = ContainerResizePolicy{} + mi := &file_proto_spark_submit_proto_msgTypes[61] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ContainerResizePolicy) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ContainerResizePolicy) ProtoMessage() {} + +func (x *ContainerResizePolicy) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[61] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ContainerResizePolicy.ProtoReflect.Descriptor instead. +func (*ContainerResizePolicy) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{61} +} + +func (x *ContainerResizePolicy) GetResourceName() string { + if x != nil { + return x.ResourceName + } + return "" +} + +func (x *ContainerResizePolicy) GetRestartPolicy() ResourceResizeRestartPolicy { + if x != nil { + return x.RestartPolicy + } + return ResourceResizeRestartPolicy_RESOURCE_RESIZE_RESTART_POLICY_UNSPECIFIED +} + +type VolumeDevice struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + DevicePath string `protobuf:"bytes,2,opt,name=device_path,json=devicePath,proto3" json:"device_path,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *VolumeDevice) Reset() { + *x = VolumeDevice{} + mi := &file_proto_spark_submit_proto_msgTypes[62] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *VolumeDevice) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VolumeDevice) ProtoMessage() {} + +func (x *VolumeDevice) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[62] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use VolumeDevice.ProtoReflect.Descriptor instead. +func (*VolumeDevice) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{62} +} + +func (x *VolumeDevice) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *VolumeDevice) GetDevicePath() string { + if x != nil { + return x.DevicePath + } + return "" +} + +type ProbeHandler struct { + state protoimpl.MessageState `protogen:"open.v1"` + Exec *ExecAction `protobuf:"bytes,1,opt,name=exec,proto3" json:"exec,omitempty"` + HttpGet *HTTPGetAction `protobuf:"bytes,2,opt,name=http_get,json=httpGet,proto3" json:"http_get,omitempty"` + TcpSocket *TCPSocketAction `protobuf:"bytes,3,opt,name=tcp_socket,json=tcpSocket,proto3" json:"tcp_socket,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ProbeHandler) Reset() { + *x = ProbeHandler{} + mi := &file_proto_spark_submit_proto_msgTypes[63] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ProbeHandler) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ProbeHandler) ProtoMessage() {} + +func (x *ProbeHandler) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[63] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ProbeHandler.ProtoReflect.Descriptor instead. +func (*ProbeHandler) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{63} +} + +func (x *ProbeHandler) GetExec() *ExecAction { + if x != nil { + return x.Exec + } + return nil +} + +func (x *ProbeHandler) GetHttpGet() *HTTPGetAction { + if x != nil { + return x.HttpGet + } + return nil +} + +func (x *ProbeHandler) GetTcpSocket() *TCPSocketAction { + if x != nil { + return x.TcpSocket + } + return nil +} + +type Probe struct { + state protoimpl.MessageState `protogen:"open.v1"` + ProbeHandler *ProbeHandler `protobuf:"bytes,1,opt,name=probe_handler,json=probeHandler,proto3" json:"probe_handler,omitempty"` + InitialDelaySeconds int32 `protobuf:"varint,2,opt,name=initial_delay_seconds,json=initialDelaySeconds,proto3" json:"initial_delay_seconds,omitempty"` + TimeoutSeconds int32 `protobuf:"varint,3,opt,name=timeout_seconds,json=timeoutSeconds,proto3" json:"timeout_seconds,omitempty"` + PeriodSeconds int32 `protobuf:"varint,4,opt,name=period_seconds,json=periodSeconds,proto3" json:"period_seconds,omitempty"` + SuccessThreshold int32 `protobuf:"varint,5,opt,name=success_threshold,json=successThreshold,proto3" json:"success_threshold,omitempty"` + FailureThreshold int32 `protobuf:"varint,6,opt,name=failure_threshold,json=failureThreshold,proto3" json:"failure_threshold,omitempty"` + TerminationGracePeriodSeconds *wrapperspb.Int64Value `protobuf:"bytes,7,opt,name=termination_grace_period_seconds,json=terminationGracePeriodSeconds,proto3" json:"termination_grace_period_seconds,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Probe) Reset() { + *x = Probe{} + mi := &file_proto_spark_submit_proto_msgTypes[64] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Probe) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Probe) ProtoMessage() {} + +func (x *Probe) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[64] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Probe.ProtoReflect.Descriptor instead. +func (*Probe) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{64} +} + +func (x *Probe) GetProbeHandler() *ProbeHandler { + if x != nil { + return x.ProbeHandler + } + return nil +} + +func (x *Probe) GetInitialDelaySeconds() int32 { + if x != nil { + return x.InitialDelaySeconds + } + return 0 +} + +func (x *Probe) GetTimeoutSeconds() int32 { + if x != nil { + return x.TimeoutSeconds + } + return 0 +} + +func (x *Probe) GetPeriodSeconds() int32 { + if x != nil { + return x.PeriodSeconds + } + return 0 +} + +func (x *Probe) GetSuccessThreshold() int32 { + if x != nil { + return x.SuccessThreshold + } + return 0 +} + +func (x *Probe) GetFailureThreshold() int32 { + if x != nil { + return x.FailureThreshold + } + return 0 +} + +func (x *Probe) GetTerminationGracePeriodSeconds() *wrapperspb.Int64Value { + if x != nil { + return x.TerminationGracePeriodSeconds + } + return nil +} + +type SecurityContext struct { + state protoimpl.MessageState `protogen:"open.v1"` + Capabilities *Capabilities `protobuf:"bytes,1,opt,name=capabilities,proto3" json:"capabilities,omitempty"` + Privileged *wrapperspb.BoolValue `protobuf:"bytes,2,opt,name=privileged,proto3" json:"privileged,omitempty"` + SeLinuxOptions *SELinuxOptions `protobuf:"bytes,3,opt,name=se_linux_options,json=seLinuxOptions,proto3" json:"se_linux_options,omitempty"` + WindowsSecurityContextOptions *WindowsSecurityContextOptions `protobuf:"bytes,4,opt,name=windows_security_context_options,json=windowsSecurityContextOptions,proto3" json:"windows_security_context_options,omitempty"` + RunAsUser *wrapperspb.Int64Value `protobuf:"bytes,5,opt,name=run_as_user,json=runAsUser,proto3" json:"run_as_user,omitempty"` + RunAsGroup *wrapperspb.Int64Value `protobuf:"bytes,6,opt,name=run_as_group,json=runAsGroup,proto3" json:"run_as_group,omitempty"` + RunAsNonRoot *wrapperspb.BoolValue `protobuf:"bytes,7,opt,name=run_as_non_root,json=runAsNonRoot,proto3" json:"run_as_non_root,omitempty"` + ReadOnlyFileSystem *wrapperspb.BoolValue `protobuf:"bytes,8,opt,name=read_only_file_system,json=readOnlyFileSystem,proto3" json:"read_only_file_system,omitempty"` + AllowPrivilegeEscalation *wrapperspb.BoolValue `protobuf:"bytes,9,opt,name=allow_privilege_escalation,json=allowPrivilegeEscalation,proto3" json:"allow_privilege_escalation,omitempty"` + ProcMount ProcMountType `protobuf:"varint,10,opt,name=proc_mount,json=procMount,proto3,enum=spark.ProcMountType" json:"proc_mount,omitempty"` + SecCompProfile *SeccompProfile `protobuf:"bytes,11,opt,name=sec_comp_profile,json=secCompProfile,proto3" json:"sec_comp_profile,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SecurityContext) Reset() { + *x = SecurityContext{} + mi := &file_proto_spark_submit_proto_msgTypes[65] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SecurityContext) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SecurityContext) ProtoMessage() {} + +func (x *SecurityContext) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[65] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SecurityContext.ProtoReflect.Descriptor instead. +func (*SecurityContext) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{65} +} + +func (x *SecurityContext) GetCapabilities() *Capabilities { + if x != nil { + return x.Capabilities + } + return nil +} + +func (x *SecurityContext) GetPrivileged() *wrapperspb.BoolValue { + if x != nil { + return x.Privileged + } + return nil +} + +func (x *SecurityContext) GetSeLinuxOptions() *SELinuxOptions { + if x != nil { + return x.SeLinuxOptions + } + return nil +} + +func (x *SecurityContext) GetWindowsSecurityContextOptions() *WindowsSecurityContextOptions { + if x != nil { + return x.WindowsSecurityContextOptions + } + return nil +} + +func (x *SecurityContext) GetRunAsUser() *wrapperspb.Int64Value { + if x != nil { + return x.RunAsUser + } + return nil +} + +func (x *SecurityContext) GetRunAsGroup() *wrapperspb.Int64Value { + if x != nil { + return x.RunAsGroup + } + return nil +} + +func (x *SecurityContext) GetRunAsNonRoot() *wrapperspb.BoolValue { + if x != nil { + return x.RunAsNonRoot + } + return nil +} + +func (x *SecurityContext) GetReadOnlyFileSystem() *wrapperspb.BoolValue { + if x != nil { + return x.ReadOnlyFileSystem + } + return nil +} + +func (x *SecurityContext) GetAllowPrivilegeEscalation() *wrapperspb.BoolValue { + if x != nil { + return x.AllowPrivilegeEscalation + } + return nil +} + +func (x *SecurityContext) GetProcMount() ProcMountType { + if x != nil { + return x.ProcMount + } + return ProcMountType_PROC_MOUNT_TYPE_UNSPECIFIED +} + +func (x *SecurityContext) GetSecCompProfile() *SeccompProfile { + if x != nil { + return x.SecCompProfile + } + return nil +} + +type Capabilities struct { + state protoimpl.MessageState `protogen:"open.v1"` + Add []string `protobuf:"bytes,1,rep,name=add,proto3" json:"add,omitempty"` + Drop []string `protobuf:"bytes,2,rep,name=drop,proto3" json:"drop,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Capabilities) Reset() { + *x = Capabilities{} + mi := &file_proto_spark_submit_proto_msgTypes[66] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Capabilities) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Capabilities) ProtoMessage() {} + +func (x *Capabilities) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[66] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Capabilities.ProtoReflect.Descriptor instead. +func (*Capabilities) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{66} +} + +func (x *Capabilities) GetAdd() []string { + if x != nil { + return x.Add + } + return nil +} + +func (x *Capabilities) GetDrop() []string { + if x != nil { + return x.Drop + } + return nil +} + +type SELinuxOptions struct { + state protoimpl.MessageState `protogen:"open.v1"` + User string `protobuf:"bytes,1,opt,name=user,proto3" json:"user,omitempty"` + Role string `protobuf:"bytes,2,opt,name=role,proto3" json:"role,omitempty"` + Type string `protobuf:"bytes,3,opt,name=type,proto3" json:"type,omitempty"` + Level string `protobuf:"bytes,4,opt,name=level,proto3" json:"level,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SELinuxOptions) Reset() { + *x = SELinuxOptions{} + mi := &file_proto_spark_submit_proto_msgTypes[67] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SELinuxOptions) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SELinuxOptions) ProtoMessage() {} + +func (x *SELinuxOptions) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[67] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SELinuxOptions.ProtoReflect.Descriptor instead. +func (*SELinuxOptions) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{67} +} + +func (x *SELinuxOptions) GetUser() string { + if x != nil { + return x.User + } + return "" +} + +func (x *SELinuxOptions) GetRole() string { + if x != nil { + return x.Role + } + return "" +} + +func (x *SELinuxOptions) GetType() string { + if x != nil { + return x.Type + } + return "" +} + +func (x *SELinuxOptions) GetLevel() string { + if x != nil { + return x.Level + } + return "" +} + +type WindowsSecurityContextOptions struct { + state protoimpl.MessageState `protogen:"open.v1"` + GmsaCredentialSpecName *wrapperspb.StringValue `protobuf:"bytes,1,opt,name=gmsa_credential_spec_name,json=gmsaCredentialSpecName,proto3" json:"gmsa_credential_spec_name,omitempty"` + GmsaCredentialSpec *wrapperspb.StringValue `protobuf:"bytes,2,opt,name=gmsa_credential_spec,json=gmsaCredentialSpec,proto3" json:"gmsa_credential_spec,omitempty"` + RunAsUserName *wrapperspb.StringValue `protobuf:"bytes,3,opt,name=run_as_user_name,json=runAsUserName,proto3" json:"run_as_user_name,omitempty"` + HostProcess *wrapperspb.BoolValue `protobuf:"bytes,4,opt,name=host_process,json=hostProcess,proto3" json:"host_process,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *WindowsSecurityContextOptions) Reset() { + *x = WindowsSecurityContextOptions{} + mi := &file_proto_spark_submit_proto_msgTypes[68] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *WindowsSecurityContextOptions) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WindowsSecurityContextOptions) ProtoMessage() {} + +func (x *WindowsSecurityContextOptions) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[68] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WindowsSecurityContextOptions.ProtoReflect.Descriptor instead. +func (*WindowsSecurityContextOptions) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{68} +} + +func (x *WindowsSecurityContextOptions) GetGmsaCredentialSpecName() *wrapperspb.StringValue { + if x != nil { + return x.GmsaCredentialSpecName + } + return nil +} + +func (x *WindowsSecurityContextOptions) GetGmsaCredentialSpec() *wrapperspb.StringValue { + if x != nil { + return x.GmsaCredentialSpec + } + return nil +} + +func (x *WindowsSecurityContextOptions) GetRunAsUserName() *wrapperspb.StringValue { + if x != nil { + return x.RunAsUserName + } + return nil +} + +func (x *WindowsSecurityContextOptions) GetHostProcess() *wrapperspb.BoolValue { + if x != nil { + return x.HostProcess + } + return nil +} + +type SeccompProfile struct { + state protoimpl.MessageState `protogen:"open.v1"` + Type SeccompProfileType `protobuf:"varint,1,opt,name=type,proto3,enum=spark.SeccompProfileType" json:"type,omitempty"` + LocalHostProfile *wrapperspb.StringValue `protobuf:"bytes,2,opt,name=local_host_profile,json=localHostProfile,proto3" json:"local_host_profile,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SeccompProfile) Reset() { + *x = SeccompProfile{} + mi := &file_proto_spark_submit_proto_msgTypes[69] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SeccompProfile) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SeccompProfile) ProtoMessage() {} + +func (x *SeccompProfile) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[69] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SeccompProfile.ProtoReflect.Descriptor instead. +func (*SeccompProfile) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{69} +} + +func (x *SeccompProfile) GetType() SeccompProfileType { + if x != nil { + return x.Type + } + return SeccompProfileType_SECCOMP_PROFILE_TYPE_UNSPECIFIED +} + +func (x *SeccompProfile) GetLocalHostProfile() *wrapperspb.StringValue { + if x != nil { + return x.LocalHostProfile + } + return nil +} + +type PodDNSConfig struct { + state protoimpl.MessageState `protogen:"open.v1"` + NameServers []string `protobuf:"bytes,1,rep,name=name_servers,json=nameServers,proto3" json:"name_servers,omitempty"` + Searches []string `protobuf:"bytes,2,rep,name=searches,proto3" json:"searches,omitempty"` + Options []*PodDNSConfigOption `protobuf:"bytes,3,rep,name=options,proto3" json:"options,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PodDNSConfig) Reset() { + *x = PodDNSConfig{} + mi := &file_proto_spark_submit_proto_msgTypes[70] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PodDNSConfig) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PodDNSConfig) ProtoMessage() {} + +func (x *PodDNSConfig) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[70] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PodDNSConfig.ProtoReflect.Descriptor instead. +func (*PodDNSConfig) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{70} +} + +func (x *PodDNSConfig) GetNameServers() []string { + if x != nil { + return x.NameServers + } + return nil +} + +func (x *PodDNSConfig) GetSearches() []string { + if x != nil { + return x.Searches + } + return nil +} + +func (x *PodDNSConfig) GetOptions() []*PodDNSConfigOption { + if x != nil { + return x.Options + } + return nil +} + +type PodDNSConfigOption struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *PodDNSConfigOption) Reset() { + *x = PodDNSConfigOption{} + mi := &file_proto_spark_submit_proto_msgTypes[71] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *PodDNSConfigOption) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PodDNSConfigOption) ProtoMessage() {} + +func (x *PodDNSConfigOption) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[71] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PodDNSConfigOption.ProtoReflect.Descriptor instead. +func (*PodDNSConfigOption) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{71} +} + +func (x *PodDNSConfigOption) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *PodDNSConfigOption) GetValue() string { + if x != nil { + return x.Value + } + return "" +} + +type HostAlias struct { + state protoimpl.MessageState `protogen:"open.v1"` + Ip string `protobuf:"bytes,1,opt,name=ip,proto3" json:"ip,omitempty"` + HostNames []string `protobuf:"bytes,2,rep,name=host_names,json=hostNames,proto3" json:"host_names,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *HostAlias) Reset() { + *x = HostAlias{} + mi := &file_proto_spark_submit_proto_msgTypes[72] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *HostAlias) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HostAlias) ProtoMessage() {} + +func (x *HostAlias) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[72] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use HostAlias.ProtoReflect.Descriptor instead. +func (*HostAlias) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{72} +} + +func (x *HostAlias) GetIp() string { + if x != nil { + return x.Ip + } + return "" +} + +func (x *HostAlias) GetHostNames() []string { + if x != nil { + return x.HostNames + } + return nil +} + +type Lifecycle struct { + state protoimpl.MessageState `protogen:"open.v1"` + PostStart *LifecycleHandler `protobuf:"bytes,1,opt,name=post_start,json=postStart,proto3" json:"post_start,omitempty"` + PreStop *LifecycleHandler `protobuf:"bytes,2,opt,name=pre_stop,json=preStop,proto3" json:"pre_stop,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Lifecycle) Reset() { + *x = Lifecycle{} + mi := &file_proto_spark_submit_proto_msgTypes[73] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Lifecycle) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Lifecycle) ProtoMessage() {} + +func (x *Lifecycle) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[73] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Lifecycle.ProtoReflect.Descriptor instead. +func (*Lifecycle) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{73} +} + +func (x *Lifecycle) GetPostStart() *LifecycleHandler { + if x != nil { + return x.PostStart + } + return nil +} + +func (x *Lifecycle) GetPreStop() *LifecycleHandler { + if x != nil { + return x.PreStop + } + return nil +} + +type LifecycleHandler struct { + state protoimpl.MessageState `protogen:"open.v1"` + Exec *ExecAction `protobuf:"bytes,1,opt,name=exec,proto3" json:"exec,omitempty"` + HttpGet *HTTPGetAction `protobuf:"bytes,2,opt,name=http_get,json=httpGet,proto3" json:"http_get,omitempty"` + TcpSocket *TCPSocketAction `protobuf:"bytes,3,opt,name=tcp_socket,json=tcpSocket,proto3" json:"tcp_socket,omitempty"` + Sleep *SleepAction `protobuf:"bytes,4,opt,name=sleep,proto3" json:"sleep,omitempty"` // Add more fields as needed + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *LifecycleHandler) Reset() { + *x = LifecycleHandler{} + mi := &file_proto_spark_submit_proto_msgTypes[74] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *LifecycleHandler) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LifecycleHandler) ProtoMessage() {} + +func (x *LifecycleHandler) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[74] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use LifecycleHandler.ProtoReflect.Descriptor instead. +func (*LifecycleHandler) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{74} +} + +func (x *LifecycleHandler) GetExec() *ExecAction { + if x != nil { + return x.Exec + } + return nil +} + +func (x *LifecycleHandler) GetHttpGet() *HTTPGetAction { + if x != nil { + return x.HttpGet + } + return nil +} + +func (x *LifecycleHandler) GetTcpSocket() *TCPSocketAction { + if x != nil { + return x.TcpSocket + } + return nil +} + +func (x *LifecycleHandler) GetSleep() *SleepAction { + if x != nil { + return x.Sleep + } + return nil +} + +type SleepAction struct { + state protoimpl.MessageState `protogen:"open.v1"` + Seconds int64 `protobuf:"varint,1,opt,name=seconds,proto3" json:"seconds,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SleepAction) Reset() { + *x = SleepAction{} + mi := &file_proto_spark_submit_proto_msgTypes[75] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SleepAction) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SleepAction) ProtoMessage() {} + +func (x *SleepAction) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[75] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SleepAction.ProtoReflect.Descriptor instead. +func (*SleepAction) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{75} +} + +func (x *SleepAction) GetSeconds() int64 { + if x != nil { + return x.Seconds + } + return 0 +} + +type TCPSocketAction struct { + state protoimpl.MessageState `protogen:"open.v1"` + Port *IntOrString `protobuf:"bytes,1,opt,name=port,proto3" json:"port,omitempty"` + Host string `protobuf:"bytes,2,opt,name=host,proto3" json:"host,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *TCPSocketAction) Reset() { + *x = TCPSocketAction{} + mi := &file_proto_spark_submit_proto_msgTypes[76] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *TCPSocketAction) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*TCPSocketAction) ProtoMessage() {} + +func (x *TCPSocketAction) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[76] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use TCPSocketAction.ProtoReflect.Descriptor instead. +func (*TCPSocketAction) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{76} +} + +func (x *TCPSocketAction) GetPort() *IntOrString { + if x != nil { + return x.Port + } + return nil +} + +func (x *TCPSocketAction) GetHost() string { + if x != nil { + return x.Host + } + return "" +} + +type ExecAction struct { + state protoimpl.MessageState `protogen:"open.v1"` + Command []string `protobuf:"bytes,1,rep,name=command,proto3" json:"command,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExecAction) Reset() { + *x = ExecAction{} + mi := &file_proto_spark_submit_proto_msgTypes[77] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExecAction) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecAction) ProtoMessage() {} + +func (x *ExecAction) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[77] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ExecAction.ProtoReflect.Descriptor instead. +func (*ExecAction) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{77} +} + +func (x *ExecAction) GetCommand() []string { + if x != nil { + return x.Command + } + return nil +} + +// Message for HTTPGetAction +type HTTPGetAction struct { + state protoimpl.MessageState `protogen:"open.v1"` + Path string `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"` + Port *IntOrString `protobuf:"bytes,2,opt,name=port,proto3" json:"port,omitempty"` + Host string `protobuf:"bytes,3,opt,name=host,proto3" json:"host,omitempty"` + Scheme URIScheme `protobuf:"varint,4,opt,name=scheme,proto3,enum=spark.URIScheme" json:"scheme,omitempty"` + HttpHeaders []*HTTPHeader `protobuf:"bytes,5,rep,name=http_headers,json=httpHeaders,proto3" json:"http_headers,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *HTTPGetAction) Reset() { + *x = HTTPGetAction{} + mi := &file_proto_spark_submit_proto_msgTypes[78] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *HTTPGetAction) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HTTPGetAction) ProtoMessage() {} + +func (x *HTTPGetAction) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[78] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use HTTPGetAction.ProtoReflect.Descriptor instead. +func (*HTTPGetAction) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{78} +} + +func (x *HTTPGetAction) GetPath() string { + if x != nil { + return x.Path + } + return "" +} + +func (x *HTTPGetAction) GetPort() *IntOrString { + if x != nil { + return x.Port + } + return nil +} + +func (x *HTTPGetAction) GetHost() string { + if x != nil { + return x.Host + } + return "" +} + +func (x *HTTPGetAction) GetScheme() URIScheme { + if x != nil { + return x.Scheme + } + return URIScheme_URISCHEME_UNSPECIFIED +} + +func (x *HTTPGetAction) GetHttpHeaders() []*HTTPHeader { + if x != nil { + return x.HttpHeaders + } + return nil +} + +// Message for HTTPHeader +type HTTPHeader struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *HTTPHeader) Reset() { + *x = HTTPHeader{} + mi := &file_proto_spark_submit_proto_msgTypes[79] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *HTTPHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HTTPHeader) ProtoMessage() {} + +func (x *HTTPHeader) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[79] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use HTTPHeader.ProtoReflect.Descriptor instead. +func (*HTTPHeader) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{79} +} + +func (x *HTTPHeader) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *HTTPHeader) GetValue() string { + if x != nil { + return x.Value + } + return "" +} + +// Message for IntOrString (since intstr.IntOrString can be either int or string) +type IntOrString struct { + state protoimpl.MessageState `protogen:"open.v1"` + // Types that are valid to be assigned to Type: + // + // *IntOrString_IntVal + // *IntOrString_StrVal + Type isIntOrString_Type `protobuf_oneof:"type"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *IntOrString) Reset() { + *x = IntOrString{} + mi := &file_proto_spark_submit_proto_msgTypes[80] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *IntOrString) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IntOrString) ProtoMessage() {} + +func (x *IntOrString) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[80] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use IntOrString.ProtoReflect.Descriptor instead. +func (*IntOrString) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{80} +} + +func (x *IntOrString) GetType() isIntOrString_Type { + if x != nil { + return x.Type + } + return nil +} + +func (x *IntOrString) GetIntVal() int32 { + if x != nil { + if x, ok := x.Type.(*IntOrString_IntVal); ok { + return x.IntVal + } + } + return 0 +} + +func (x *IntOrString) GetStrVal() string { + if x != nil { + if x, ok := x.Type.(*IntOrString_StrVal); ok { + return x.StrVal + } + } + return "" +} + +type isIntOrString_Type interface { + isIntOrString_Type() +} + +type IntOrString_IntVal struct { + IntVal int32 `protobuf:"varint,1,opt,name=int_val,json=intVal,proto3,oneof"` +} + +type IntOrString_StrVal struct { + StrVal string `protobuf:"bytes,2,opt,name=str_val,json=strVal,proto3,oneof"` +} + +func (*IntOrString_IntVal) isIntOrString_Type() {} + +func (*IntOrString_StrVal) isIntOrString_Type() {} + +type Ports struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Protocol string `protobuf:"bytes,2,opt,name=protocol,proto3" json:"protocol,omitempty"` + ContainerPort string `protobuf:"bytes,3,opt,name=container_port,json=containerPort,proto3" json:"container_port,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Ports) Reset() { + *x = Ports{} + mi := &file_proto_spark_submit_proto_msgTypes[81] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Ports) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Ports) ProtoMessage() {} + +func (x *Ports) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[81] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Ports.ProtoReflect.Descriptor instead. +func (*Ports) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{81} +} + +func (x *Ports) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Ports) GetProtocol() string { + if x != nil { + return x.Protocol + } + return "" +} + +func (x *Ports) GetContainerPort() string { + if x != nil { + return x.ContainerPort + } + return "" +} + +type ExecutorSpec struct { + state protoimpl.MessageState `protogen:"open.v1"` + SparkPodSpec *SparkPodSpec `protobuf:"bytes,1,opt,name=spark_pod_spec,json=sparkPodSpec,proto3" json:"spark_pod_spec,omitempty"` + Instances *wrapperspb.Int32Value `protobuf:"bytes,2,opt,name=instances,proto3" json:"instances,omitempty"` + CoreRequest *wrapperspb.StringValue `protobuf:"bytes,3,opt,name=core_request,json=coreRequest,proto3" json:"core_request,omitempty"` + JavaOptions *wrapperspb.StringValue `protobuf:"bytes,4,opt,name=java_options,json=javaOptions,proto3" json:"java_options,omitempty"` + LifeCycle *Lifecycle `protobuf:"bytes,5,opt,name=life_cycle,json=lifeCycle,proto3" json:"life_cycle,omitempty"` + DeleteOnTermination *wrapperspb.BoolValue `protobuf:"bytes,6,opt,name=delete_on_termination,json=deleteOnTermination,proto3" json:"delete_on_termination,omitempty"` + Ports []*Ports `protobuf:"bytes,7,rep,name=ports,proto3" json:"ports,omitempty"` + PriorityClassName *wrapperspb.StringValue `protobuf:"bytes,8,opt,name=priority_class_name,json=priorityClassName,proto3" json:"priority_class_name,omitempty"` // Add more fields as needed + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ExecutorSpec) Reset() { + *x = ExecutorSpec{} + mi := &file_proto_spark_submit_proto_msgTypes[82] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ExecutorSpec) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExecutorSpec) ProtoMessage() {} + +func (x *ExecutorSpec) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[82] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ExecutorSpec.ProtoReflect.Descriptor instead. +func (*ExecutorSpec) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{82} +} + +func (x *ExecutorSpec) GetSparkPodSpec() *SparkPodSpec { + if x != nil { + return x.SparkPodSpec + } + return nil +} + +func (x *ExecutorSpec) GetInstances() *wrapperspb.Int32Value { + if x != nil { + return x.Instances + } + return nil +} + +func (x *ExecutorSpec) GetCoreRequest() *wrapperspb.StringValue { + if x != nil { + return x.CoreRequest + } + return nil +} + +func (x *ExecutorSpec) GetJavaOptions() *wrapperspb.StringValue { + if x != nil { + return x.JavaOptions + } + return nil +} + +func (x *ExecutorSpec) GetLifeCycle() *Lifecycle { + if x != nil { + return x.LifeCycle + } + return nil +} + +func (x *ExecutorSpec) GetDeleteOnTermination() *wrapperspb.BoolValue { + if x != nil { + return x.DeleteOnTermination + } + return nil +} + +func (x *ExecutorSpec) GetPorts() []*Ports { + if x != nil { + return x.Ports + } + return nil +} + +func (x *ExecutorSpec) GetPriorityClassName() *wrapperspb.StringValue { + if x != nil { + return x.PriorityClassName + } + return nil +} + +// Volume and VolumeMount +type Volume struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` // e.g., "hostPath", "emptyDir", etc. + Path string `protobuf:"bytes,3,opt,name=path,proto3" json:"path,omitempty"` // Add more fields as needed for different volume types + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Volume) Reset() { + *x = Volume{} + mi := &file_proto_spark_submit_proto_msgTypes[83] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Volume) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Volume) ProtoMessage() {} + +func (x *Volume) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[83] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Volume.ProtoReflect.Descriptor instead. +func (*Volume) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{83} +} + +func (x *Volume) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Volume) GetType() string { + if x != nil { + return x.Type + } + return "" +} + +func (x *Volume) GetPath() string { + if x != nil { + return x.Path + } + return "" +} + +type VolumeMount struct { + state protoimpl.MessageState `protogen:"open.v1"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + MountPath string `protobuf:"bytes,2,opt,name=mount_path,json=mountPath,proto3" json:"mount_path,omitempty"` + ReadOnly bool `protobuf:"varint,3,opt,name=read_only,json=readOnly,proto3" json:"read_only,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *VolumeMount) Reset() { + *x = VolumeMount{} + mi := &file_proto_spark_submit_proto_msgTypes[84] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *VolumeMount) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*VolumeMount) ProtoMessage() {} + +func (x *VolumeMount) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[84] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use VolumeMount.ProtoReflect.Descriptor instead. +func (*VolumeMount) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{84} +} + +func (x *VolumeMount) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *VolumeMount) GetMountPath() string { + if x != nil { + return x.MountPath + } + return "" +} + +func (x *VolumeMount) GetReadOnly() bool { + if x != nil { + return x.ReadOnly + } + return false +} + +// SparkApplicationStatus (simplified) +type SparkApplicationStatus struct { + state protoimpl.MessageState `protogen:"open.v1"` + ApplicationState string `protobuf:"bytes,1,opt,name=application_state,json=applicationState,proto3" json:"application_state,omitempty"` + SparkApplicationId string `protobuf:"bytes,2,opt,name=spark_application_id,json=sparkApplicationId,proto3" json:"spark_application_id,omitempty"` + SubmissionId string `protobuf:"bytes,3,opt,name=submission_id,json=submissionId,proto3" json:"submission_id,omitempty"` // Add more fields as needed + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SparkApplicationStatus) Reset() { + *x = SparkApplicationStatus{} + mi := &file_proto_spark_submit_proto_msgTypes[85] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SparkApplicationStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SparkApplicationStatus) ProtoMessage() {} + +func (x *SparkApplicationStatus) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[85] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SparkApplicationStatus.ProtoReflect.Descriptor instead. +func (*SparkApplicationStatus) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{85} +} + +func (x *SparkApplicationStatus) GetApplicationState() string { + if x != nil { + return x.ApplicationState + } + return "" +} + +func (x *SparkApplicationStatus) GetSparkApplicationId() string { + if x != nil { + return x.SparkApplicationId + } + return "" +} + +func (x *SparkApplicationStatus) GetSubmissionId() string { + if x != nil { + return x.SubmissionId + } + return "" +} + +// The SparkApplication CRD +type SparkApplication struct { + state protoimpl.MessageState `protogen:"open.v1"` + Metadata *ObjectMeta `protobuf:"bytes,1,opt,name=metadata,proto3" json:"metadata,omitempty"` + Spec *SparkApplicationSpec `protobuf:"bytes,2,opt,name=spec,proto3" json:"spec,omitempty"` + Status *SparkApplicationStatus `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SparkApplication) Reset() { + *x = SparkApplication{} + mi := &file_proto_spark_submit_proto_msgTypes[86] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SparkApplication) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SparkApplication) ProtoMessage() {} + +func (x *SparkApplication) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[86] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SparkApplication.ProtoReflect.Descriptor instead. +func (*SparkApplication) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{86} +} + +func (x *SparkApplication) GetMetadata() *ObjectMeta { + if x != nil { + return x.Metadata + } + return nil +} + +func (x *SparkApplication) GetSpec() *SparkApplicationSpec { + if x != nil { + return x.Spec + } + return nil +} + +func (x *SparkApplication) GetStatus() *SparkApplicationStatus { + if x != nil { + return x.Status + } + return nil +} + +// The request message containing the SparkApplication and submission ID. +type RunAltSparkSubmitRequest struct { + state protoimpl.MessageState `protogen:"open.v1"` + SparkApplication *SparkApplication `protobuf:"bytes,1,opt,name=spark_application,json=sparkApplication,proto3" json:"spark_application,omitempty"` + SubmissionId string `protobuf:"bytes,2,opt,name=submission_id,json=submissionId,proto3" json:"submission_id,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RunAltSparkSubmitRequest) Reset() { + *x = RunAltSparkSubmitRequest{} + mi := &file_proto_spark_submit_proto_msgTypes[87] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RunAltSparkSubmitRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RunAltSparkSubmitRequest) ProtoMessage() {} + +func (x *RunAltSparkSubmitRequest) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[87] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RunAltSparkSubmitRequest.ProtoReflect.Descriptor instead. +func (*RunAltSparkSubmitRequest) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{87} +} + +func (x *RunAltSparkSubmitRequest) GetSparkApplication() *SparkApplication { + if x != nil { + return x.SparkApplication + } + return nil +} + +func (x *RunAltSparkSubmitRequest) GetSubmissionId() string { + if x != nil { + return x.SubmissionId + } + return "" +} + +// The response message indicating success or failure. +type RunAltSparkSubmitResponse struct { + state protoimpl.MessageState `protogen:"open.v1"` + Success bool `protobuf:"varint,1,opt,name=success,proto3" json:"success,omitempty"` + ErrorMessage string `protobuf:"bytes,2,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *RunAltSparkSubmitResponse) Reset() { + *x = RunAltSparkSubmitResponse{} + mi := &file_proto_spark_submit_proto_msgTypes[88] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *RunAltSparkSubmitResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*RunAltSparkSubmitResponse) ProtoMessage() {} + +func (x *RunAltSparkSubmitResponse) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[88] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use RunAltSparkSubmitResponse.ProtoReflect.Descriptor instead. +func (*RunAltSparkSubmitResponse) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{88} +} + +func (x *RunAltSparkSubmitResponse) GetSuccess() bool { + if x != nil { + return x.Success + } + return false +} + +func (x *RunAltSparkSubmitResponse) GetErrorMessage() string { + if x != nil { + return x.ErrorMessage + } + return "" +} + +// Dependencies specifies all possible types of dependencies of a Spark application. +type Dependencies struct { + state protoimpl.MessageState `protogen:"open.v1"` + Jars []string `protobuf:"bytes,1,rep,name=jars,proto3" json:"jars,omitempty"` + Files []string `protobuf:"bytes,2,rep,name=files,proto3" json:"files,omitempty"` + PyFiles []string `protobuf:"bytes,3,rep,name=py_files,json=pyFiles,proto3" json:"py_files,omitempty"` + Packages []string `protobuf:"bytes,4,rep,name=packages,proto3" json:"packages,omitempty"` + ExcludePackages []string `protobuf:"bytes,5,rep,name=exclude_packages,json=excludePackages,proto3" json:"exclude_packages,omitempty"` + Repositories []string `protobuf:"bytes,6,rep,name=repositories,proto3" json:"repositories,omitempty"` + Archives []string `protobuf:"bytes,7,rep,name=archives,proto3" json:"archives,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *Dependencies) Reset() { + *x = Dependencies{} + mi := &file_proto_spark_submit_proto_msgTypes[89] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Dependencies) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Dependencies) ProtoMessage() {} + +func (x *Dependencies) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[89] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Dependencies.ProtoReflect.Descriptor instead. +func (*Dependencies) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{89} +} + +func (x *Dependencies) GetJars() []string { + if x != nil { + return x.Jars + } + return nil +} + +func (x *Dependencies) GetFiles() []string { + if x != nil { + return x.Files + } + return nil +} + +func (x *Dependencies) GetPyFiles() []string { + if x != nil { + return x.PyFiles + } + return nil +} + +func (x *Dependencies) GetPackages() []string { + if x != nil { + return x.Packages + } + return nil +} + +func (x *Dependencies) GetExcludePackages() []string { + if x != nil { + return x.ExcludePackages + } + return nil +} + +func (x *Dependencies) GetRepositories() []string { + if x != nil { + return x.Repositories + } + return nil +} + +func (x *Dependencies) GetArchives() []string { + if x != nil { + return x.Archives + } + return nil +} + +type DynamicAllocation struct { + state protoimpl.MessageState `protogen:"open.v1"` + Enabled bool `protobuf:"varint,1,opt,name=enabled,proto3" json:"enabled,omitempty"` + InitialExecutors int32 `protobuf:"varint,2,opt,name=initial_executors,json=initialExecutors,proto3" json:"initial_executors,omitempty"` + MinExecutors int32 `protobuf:"varint,3,opt,name=min_executors,json=minExecutors,proto3" json:"min_executors,omitempty"` + MaxExecutors int32 `protobuf:"varint,4,opt,name=max_executors,json=maxExecutors,proto3" json:"max_executors,omitempty"` + ShuffleTrackingTimeout int64 `protobuf:"varint,5,opt,name=shuffle_tracking_timeout,json=shuffleTrackingTimeout,proto3" json:"shuffle_tracking_timeout,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *DynamicAllocation) Reset() { + *x = DynamicAllocation{} + mi := &file_proto_spark_submit_proto_msgTypes[90] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DynamicAllocation) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DynamicAllocation) ProtoMessage() {} + +func (x *DynamicAllocation) ProtoReflect() protoreflect.Message { + mi := &file_proto_spark_submit_proto_msgTypes[90] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DynamicAllocation.ProtoReflect.Descriptor instead. +func (*DynamicAllocation) Descriptor() ([]byte, []int) { + return file_proto_spark_submit_proto_rawDescGZIP(), []int{90} +} + +func (x *DynamicAllocation) GetEnabled() bool { + if x != nil { + return x.Enabled + } + return false +} + +func (x *DynamicAllocation) GetInitialExecutors() int32 { + if x != nil { + return x.InitialExecutors + } + return 0 +} + +func (x *DynamicAllocation) GetMinExecutors() int32 { + if x != nil { + return x.MinExecutors + } + return 0 +} + +func (x *DynamicAllocation) GetMaxExecutors() int32 { + if x != nil { + return x.MaxExecutors + } + return 0 +} + +func (x *DynamicAllocation) GetShuffleTrackingTimeout() int64 { + if x != nil { + return x.ShuffleTrackingTimeout + } + return 0 +} + +var File_proto_spark_submit_proto protoreflect.FileDescriptor + +const file_proto_spark_submit_proto_rawDesc = "" + + "\n" + + "\x18proto/spark_submit.proto\x12\x05spark\x1a\x1egoogle/protobuf/wrappers.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"\xf6\x0f\n" + + "\x14SparkApplicationSpec\x12/\n" + + "\x04type\x18\x01 \x01(\x0e2\x1b.spark.SparkApplicationTypeR\x04type\x12%\n" + + "\x04mode\x18\x02 \x01(\x0e2\x11.spark.DeployModeR\x04mode\x122\n" + + "\x05image\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\x05image\x12H\n" + + "\x11image_pull_policy\x18\x04 \x01(\v2\x1c.google.protobuf.StringValueR\x0fimagePullPolicy\x12,\n" + + "\x12image_pull_secrets\x18\x05 \x03(\tR\x10imagePullSecrets\x12I\n" + + "\n" + + "spark_conf\x18\x06 \x03(\v2*.spark.SparkApplicationSpec.SparkConfEntryR\tsparkConf\x12L\n" + + "\vhadoop_conf\x18\a \x03(\v2+.spark.SparkApplicationSpec.HadoopConfEntryR\n" + + "hadoopConf\x12F\n" + + "\x10spark_config_map\x18\b \x01(\v2\x1c.google.protobuf.StringValueR\x0esparkConfigMap\x12H\n" + + "\x11hadoop_config_map\x18\t \x01(\v2\x1c.google.protobuf.StringValueR\x0fhadoopConfigMap\x12\x1c\n" + + "\targuments\x18\n" + + " \x03(\tR\targuments\x12;\n" + + "\n" + + "main_class\x18\v \x01(\v2\x1c.google.protobuf.StringValueR\tmainClass\x12P\n" + + "\x15main_application_file\x18\f \x01(\v2\x1c.google.protobuf.StringValueR\x13mainApplicationFile\x12;\n" + + "\n" + + "proxy_user\x18\r \x01(\v2\x1c.google.protobuf.StringValueR\tproxyUser\x12D\n" + + "\x0ffailure_retries\x18\x0f \x01(\v2\x1b.google.protobuf.Int32ValueR\x0efailureRetries\x12B\n" + + "\x0eretry_interval\x18\x10 \x01(\v2\x1b.google.protobuf.Int64ValueR\rretryInterval\x12R\n" + + "\x16memory_overhead_factor\x18\x11 \x01(\v2\x1c.google.protobuf.StringValueR\x14memoryOverheadFactor\x125\n" + + "\n" + + "monitoring\x18\x12 \x01(\v2\x15.spark.MonitoringSpecR\n" + + "monitoring\x12E\n" + + "\x0fbatch_scheduler\x18\x13 \x01(\v2\x1c.google.protobuf.StringValueR\x0ebatchScheduler\x12L\n" + + "\x14time_to_live_seconds\x18\x14 \x01(\v2\x1b.google.protobuf.Int64ValueR\x11timeToLiveSeconds\x12f\n" + + "\x1dbatch_scheduler_configuration\x18\x15 \x01(\v2\".spark.BatchSchedulerConfigurationR\x1bbatchSchedulerConfiguration\x12)\n" + + "\x06driver\x18\x16 \x01(\v2\x11.spark.DriverSpecR\x06driver\x12/\n" + + "\bexecutor\x18\x17 \x01(\v2\x13.spark.ExecutorSpecR\bexecutor\x12'\n" + + "\avolumes\x18\x18 \x03(\v2\r.spark.VolumeR\avolumes\x12'\n" + + "\x04deps\x18\x19 \x01(\v2\x13.spark.DependenciesR\x04deps\x12G\n" + + "\x12dynamic_allocation\x18\x1a \x01(\v2\x18.spark.DynamicAllocationR\x11dynamicAllocation\x12%\n" + + "\x0epython_version\x18\x1b \x01(\tR\rpythonVersion\x12#\n" + + "\rspark_version\x18\x1c \x01(\tR\fsparkVersion\x12;\n" + + "\x0erestart_policy\x18\x1d \x01(\v2\x14.spark.RestartPolicyR\rrestartPolicy\x12Q\n" + + "\x16spark_ui_configuration\x18\x1e \x01(\v2\x1b.spark.SparkUIConfigurationR\x14sparkUiConfiguration\x12c\n" + + "\x1cdriver_ingress_configuration\x18\x1f \x03(\v2!.spark.DriverIngressConfigurationR\x1adriverIngressConfiguration\x1a<\n" + + "\x0eSparkConfEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\x1a=\n" + + "\x0fHadoopConfEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\xef\x06\n" + + "\n" + + "ObjectMeta\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12#\n" + + "\rgenerate_name\x18\x02 \x01(\tR\fgenerateName\x12\x1c\n" + + "\tnamespace\x18\x03 \x01(\tR\tnamespace\x12\x1b\n" + + "\tself_link\x18\x04 \x01(\tR\bselfLink\x12\x10\n" + + "\x03uid\x18\x05 \x01(\tR\x03uid\x12)\n" + + "\x10resource_version\x18\x06 \x01(\tR\x0fresourceVersion\x12\x1e\n" + + "\n" + + "generation\x18\a \x01(\x03R\n" + + "generation\x12I\n" + + "\x12creation_timestamp\x18\b \x01(\v2\x1a.google.protobuf.TimestampR\x11creationTimestamp\x12I\n" + + "\x12deletion_timestamp\x18\t \x01(\v2\x1a.google.protobuf.TimestampR\x11deletionTimestamp\x12^\n" + + "\x1ddeletion_grace_period_seconds\x18\n" + + " \x01(\v2\x1b.google.protobuf.Int64ValueR\x1adeletionGracePeriodSeconds\x125\n" + + "\x06labels\x18\v \x03(\v2\x1d.spark.ObjectMeta.LabelsEntryR\x06labels\x12D\n" + + "\vannotations\x18\f \x03(\v2\".spark.ObjectMeta.AnnotationsEntryR\vannotations\x12@\n" + + "\x10owner_references\x18\r \x03(\v2\x15.spark.OwnerReferenceR\x0fownerReferences\x12\x1e\n" + + "\n" + + "finalizers\x18\x0e \x03(\tR\n" + + "finalizers\x12@\n" + + "\x0emanaged_fields\x18\x0f \x03(\v2\x19.spark.ManagedFieldsEntryR\rmanagedFields\x1a9\n" + + "\vLabelsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\x1a>\n" + + "\x10AnnotationsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\x1c\n" + + "\bFieldsV1\x12\x10\n" + + "\x03raw\x18\x01 \x01(\tR\x03raw\"\xb7\x02\n" + + "\x12ManagedFieldsEntry\x12\x18\n" + + "\amanager\x18\x01 \x01(\tR\amanager\x12?\n" + + "\toperation\x18\x02 \x01(\x0e2!.spark.ManagedFieldsOperationTypeR\toperation\x12\x1f\n" + + "\vapi_version\x18\x03 \x01(\tR\n" + + "apiVersion\x123\n" + + "\amy_time\x18\x04 \x01(\v2\x1a.google.protobuf.TimestampR\x06myTime\x12\x1f\n" + + "\vfields_type\x18\x05 \x01(\tR\n" + + "fieldsType\x12,\n" + + "\tfields_v1\x18\x06 \x01(\v2\x0f.spark.FieldsV1R\bfieldsV1\x12!\n" + + "\fsub_resource\x18\a \x01(\tR\vsubResource\"\xf5\x01\n" + + "\x0eOwnerReference\x12\x1f\n" + + "\vapi_version\x18\x01 \x01(\tR\n" + + "apiVersion\x12\x12\n" + + "\x04kind\x18\x02 \x01(\tR\x04kind\x12\x12\n" + + "\x04name\x18\x03 \x01(\tR\x04name\x12\x10\n" + + "\x03uid\x18\x04 \x01(\tR\x03uid\x12:\n" + + "\n" + + "controller\x18\x05 \x01(\v2\x1a.google.protobuf.BoolValueR\n" + + "controller\x12L\n" + + "\x14block_owner_deletion\x18\x06 \x01(\v2\x1a.google.protobuf.BoolValueR\x12blockOwnerDeletion\"\xc4\x06\n" + + "\x1aDriverIngressConfiguration\x12>\n" + + "\fservice_port\x18\x01 \x01(\v2\x1b.google.protobuf.Int32ValueR\vservicePort\x12H\n" + + "\x11service_port_name\x18\x02 \x01(\v2\x1c.google.protobuf.StringValueR\x0fservicePortName\x125\n" + + "\fservice_type\x18\x03 \x01(\x0e2\x12.spark.ServiceTypeR\vserviceType\x12j\n" + + "\x13service_annotations\x18\x04 \x03(\v29.spark.DriverIngressConfiguration.ServiceAnnotationsEntryR\x12serviceAnnotations\x12[\n" + + "\x0eservice_labels\x18\x05 \x03(\v24.spark.DriverIngressConfiguration.ServiceLabelsEntryR\rserviceLabels\x12,\n" + + "\x12ingress_url_format\x18\x06 \x01(\tR\x10ingressUrlFormat\x12j\n" + + "\x13ingress_annotations\x18\a \x03(\v29.spark.DriverIngressConfiguration.IngressAnnotationsEntryR\x12ingressAnnotations\x122\n" + + "\vingress_tls\x18\b \x03(\v2\x11.spark.IngressTLSR\n" + + "ingressTls\x1aE\n" + + "\x17ServiceAnnotationsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\x1a@\n" + + "\x12ServiceLabelsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\x1aE\n" + + "\x17IngressAnnotationsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\xfe\x05\n" + + "\x14SparkUIConfiguration\x12>\n" + + "\fservice_port\x18\x01 \x01(\v2\x1b.google.protobuf.Int32ValueR\vservicePort\x12H\n" + + "\x11service_port_name\x18\x02 \x01(\v2\x1c.google.protobuf.StringValueR\x0fservicePortName\x125\n" + + "\fservice_type\x18\x03 \x01(\x0e2\x12.spark.ServiceTypeR\vserviceType\x12d\n" + + "\x13service_annotations\x18\x04 \x03(\v23.spark.SparkUIConfiguration.ServiceAnnotationsEntryR\x12serviceAnnotations\x12U\n" + + "\x0eservice_labels\x18\x05 \x03(\v2..spark.SparkUIConfiguration.ServiceLabelsEntryR\rserviceLabels\x12d\n" + + "\x13ingress_annotations\x18\x06 \x03(\v23.spark.SparkUIConfiguration.IngressAnnotationsEntryR\x12ingressAnnotations\x122\n" + + "\vingress_tls\x18\a \x03(\v2\x11.spark.IngressTLSR\n" + + "ingressTls\x1aE\n" + + "\x17ServiceAnnotationsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\x1a@\n" + + "\x12ServiceLabelsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\x1aE\n" + + "\x17IngressAnnotationsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"C\n" + + "\n" + + "IngressTLS\x12\x14\n" + + "\x05hosts\x18\x01 \x03(\tR\x05hosts\x12\x1f\n" + + "\vsecret_name\x18\x02 \x01(\tR\n" + + "secretName\"\xbf\x02\n" + + "\x1bBatchSchedulerConfiguration\x122\n" + + "\x05queue\x18\x01 \x01(\v2\x1c.google.protobuf.StringValueR\x05queue\x12L\n" + + "\x13priority_class_name\x18\x02 \x01(\v2\x1c.google.protobuf.StringValueR\x11priorityClassName\x12O\n" + + "\tresources\x18\x03 \x03(\v21.spark.BatchSchedulerConfiguration.ResourcesEntryR\tresources\x1aM\n" + + "\x0eResourcesEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12%\n" + + "\x05value\x18\x02 \x01(\v2\x0f.spark.QuantityR\x05value:\x028\x01\"\xd6\x02\n" + + "\x0eMonitoringSpec\x122\n" + + "\x15expose_driver_metrics\x18\x01 \x01(\bR\x13exposeDriverMetrics\x126\n" + + "\x17expose_executor_metrics\x18\x02 \x01(\bR\x15exposeExecutorMetrics\x12K\n" + + "\x12metrics_properties\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\x11metricsProperties\x12T\n" + + "\x17metrics_properties_file\x18\x04 \x01(\v2\x1c.google.protobuf.StringValueR\x15metricsPropertiesFile\x125\n" + + "\n" + + "prometheus\x18\x05 \x01(\v2\x15.spark.PrometheusSpecR\n" + + "prometheus\"\xa9\x02\n" + + "\x0ePrometheusSpec\x12(\n" + + "\x10jmx_exporter_jar\x18\x01 \x01(\tR\x0ejmxExporterJar\x12/\n" + + "\x04port\x18\x02 \x01(\v2\x1b.google.protobuf.Int32ValueR\x04port\x129\n" + + "\tport_name\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\bportName\x12=\n" + + "\vconfig_file\x18\x04 \x01(\v2\x1c.google.protobuf.StringValueR\n" + + "configFile\x12B\n" + + "\rconfiguration\x18\x05 \x01(\v2\x1c.google.protobuf.StringValueR\rconfiguration\"#\n" + + "\rRestartPolicy\x12\x12\n" + + "\x04type\x18\x01 \x01(\tR\x04type\"\xa2\x06\n" + + "\n" + + "DriverSpec\x129\n" + + "\x0espark_pod_spec\x18\x01 \x01(\v2\x13.spark.SparkPodSpecR\fsparkPodSpec\x127\n" + + "\bpod_name\x18\x02 \x01(\v2\x1c.google.protobuf.StringValueR\apodName\x12?\n" + + "\fcore_request\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\vcoreRequest\x12?\n" + + "\fjava_options\x18\x04 \x01(\v2\x1c.google.protobuf.StringValueR\vjavaOptions\x12/\n" + + "\n" + + "life_cycle\x18\x05 \x01(\v2\x10.spark.LifecycleR\tlifeCycle\x12I\n" + + "\x11kubernetes_master\x18\x06 \x01(\v2\x1c.google.protobuf.StringValueR\x10kubernetesMaster\x12Z\n" + + "\x13service_annotations\x18\a \x03(\v2).spark.DriverSpec.ServiceAnnotationsEntryR\x12serviceAnnotations\x12K\n" + + "\x0eservice_labels\x18\b \x03(\v2$.spark.DriverSpec.ServiceLabelsEntryR\rserviceLabels\x12\"\n" + + "\x05ports\x18\t \x03(\v2\f.spark.PortsR\x05ports\x12L\n" + + "\x13priority_class_name\x18\n" + + " \x01(\v2\x1c.google.protobuf.StringValueR\x11priorityClassName\x1aE\n" + + "\x17ServiceAnnotationsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\x1a@\n" + + "\x12ServiceLabelsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\xa4\x0e\n" + + "\fSparkPodSpec\x122\n" + + "\btemplate\x18\x01 \x01(\v2\x16.spark.PodTemplateSpecR\btemplate\x121\n" + + "\x05cores\x18\x02 \x01(\v2\x1b.google.protobuf.Int32ValueR\x05cores\x12\x1d\n" + + "\n" + + "core_limit\x18\x03 \x01(\tR\tcoreLimit\x12\x16\n" + + "\x06memory\x18\x04 \x01(\tR\x06memory\x12'\n" + + "\x0fmemory_overhead\x18\x05 \x01(\tR\x0ememoryOverhead\x12 \n" + + "\x03gpu\x18\x06 \x01(\v2\x0e.spark.GPUSpecR\x03gpu\x12\x14\n" + + "\x05image\x18\a \x01(\tR\x05image\x12/\n" + + "\n" + + "configmaps\x18\b \x03(\v2\x0f.spark.NamePathR\n" + + "configmaps\x12+\n" + + "\asecrets\x18\t \x03(\v2\x11.spark.SecretInfoR\asecrets\x12\x1f\n" + + "\x03env\x18\n" + + " \x03(\v2\r.spark.EnvVarR\x03env\x12;\n" + + "\benv_vars\x18\v \x03(\v2 .spark.SparkPodSpec.EnvVarsEntryR\aenvVars\x12/\n" + + "\benv_from\x18\f \x03(\v2\x14.spark.EnvFromSourceR\aenvFrom\x127\n" + + "\x06labels\x18\r \x03(\v2\x1f.spark.SparkPodSpec.LabelsEntryR\x06labels\x12F\n" + + "\vannotations\x18\x0e \x03(\v2$.spark.SparkPodSpec.AnnotationsEntryR\vannotations\x127\n" + + "\rvolume_mounts\x18\x0f \x03(\v2\x12.spark.VolumeMountR\fvolumeMounts\x12+\n" + + "\baffinity\x18\x10 \x01(\v2\x0f.spark.AffinityR\baffinity\x123\n" + + "\vtolerations\x18\x11 \x03(\v2\x11.spark.TolerationR\vtolerations\x12K\n" + + "\x14pod_security_context\x18\x12 \x01(\v2\x19.spark.PodSecurityContextR\x12podSecurityContext\x12A\n" + + "\x10security_context\x18\x13 \x01(\v2\x16.spark.SecurityContextR\x0fsecurityContext\x12C\n" + + "\x0escheduler_name\x18\x14 \x01(\v2\x1c.google.protobuf.StringValueR\rschedulerName\x12,\n" + + "\bsidecars\x18\x15 \x03(\v2\x10.spark.ContainerR\bsidecars\x129\n" + + "\x0finit_containers\x18\x16 \x03(\v2\x10.spark.ContainerR\x0einitContainers\x12=\n" + + "\fhost_network\x18\x17 \x01(\v2\x1a.google.protobuf.BoolValueR\vhostNetwork\x12J\n" + + "\rnode_selector\x18\x18 \x03(\v2%.spark.SparkPodSpec.NodeSelectorEntryR\fnodeSelector\x122\n" + + "\n" + + "dns_config\x18\x19 \x01(\v2\x13.spark.PodDNSConfigR\tdnsConfig\x12G\n" + + " termination_grace_period_seconds\x18\x1a \x01(\x03R\x1dterminationGracePeriodSeconds\x12E\n" + + "\x0fservice_account\x18\x1b \x01(\v2\x1c.google.protobuf.StringValueR\x0eserviceAccount\x123\n" + + "\fhost_aliases\x18\x1c \x03(\v2\x10.spark.HostAliasR\vhostAliases\x12R\n" + + "\x17share_process_namespace\x18\x1d \x01(\v2\x1a.google.protobuf.BoolValueR\x15shareProcessNamespace\x1a:\n" + + "\fEnvVarsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\x1a9\n" + + "\vLabelsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\x1a>\n" + + "\x10AnnotationsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\x1a?\n" + + "\x11NodeSelectorEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"p\n" + + "\x0fPodTemplateSpec\x122\n" + + "\vobject_meta\x18\x01 \x01(\v2\x11.spark.ObjectMetaR\n" + + "objectMeta\x12)\n" + + "\bpod_spec\x18\x02 \x01(\v2\x0e.spark.PodSpecR\apodSpec\"\x92\x12\n" + + "\aPodSpec\x12'\n" + + "\avolumes\x18\x01 \x03(\v2\r.spark.VolumeR\avolumes\x120\n" + + "\n" + + "containers\x18\x02 \x03(\v2\x10.spark.ContainerR\n" + + "containers\x12L\n" + + "\x14ephemeral_containers\x18\x03 \x03(\v2\x19.spark.EphemeralContainerR\x13ephemeralContainers\x12;\n" + + "\x0erestart_policy\x18\x04 \x01(\v2\x14.spark.RestartPolicyR\rrestartPolicy\x12d\n" + + " termination_grace_period_seconds\x18\x05 \x01(\v2\x1b.google.protobuf.Int64ValueR\x1dterminationGracePeriodSeconds\x12S\n" + + "\x17active_deadline_seconds\x18\x06 \x01(\v2\x1b.google.protobuf.Int64ValueR\x15activeDeadlineSeconds\x12/\n" + + "\n" + + "dns_policy\x18\a \x01(\x0e2\x10.spark.DNSPolicyR\tdnsPolicy\x12E\n" + + "\rnode_selector\x18\b \x03(\v2 .spark.PodSpec.NodeSelectorEntryR\fnodeSelector\x120\n" + + "\x14service_account_name\x18\t \x01(\tR\x12serviceAccountName\x12<\n" + + "\x1adeprecated_service_account\x18\n" + + " \x01(\tR\x18deprecatedServiceAccount\x12b\n" + + " auto_mount_service_account_token\x18\v \x01(\v2\x1a.google.protobuf.BoolValueR\x1cautoMountServiceAccountToken\x12\x1b\n" + + "\tnode_name\x18\f \x01(\tR\bnodeName\x12!\n" + + "\fhost_network\x18\r \x01(\bR\vhostNetwork\x12\x19\n" + + "\bhost_pid\x18\x0e \x01(\bR\ahostPid\x12\x19\n" + + "\bhost_ipc\x18\x0f \x01(\bR\ahostIpc\x12H\n" + + "\x12share_process_name\x18\x10 \x01(\v2\x1a.google.protobuf.BoolValueR\x10shareProcessName\x12D\n" + + "\x10security_context\x18\x11 \x01(\v2\x19.spark.PodSecurityContextR\x0fsecurityContext\x12I\n" + + "\x12image_pull_secrets\x18\x12 \x03(\v2\x1b.spark.LocalObjectReferenceR\x10imagePullSecrets\x12\x1b\n" + + "\thost_name\x18\x13 \x01(\tR\bhostName\x12\x1d\n" + + "\n" + + "sub_domain\x18\x14 \x01(\tR\tsubDomain\x12+\n" + + "\baffinity\x18\x15 \x01(\v2\x0f.spark.AffinityR\baffinity\x12%\n" + + "\x0escheduler_name\x18\x16 \x01(\tR\rschedulerName\x123\n" + + "\vtolerations\x18\x17 \x03(\v2\x11.spark.TolerationR\vtolerations\x123\n" + + "\fhost_aliases\x18\x18 \x03(\v2\x10.spark.HostAliasR\vhostAliases\x12.\n" + + "\x13priority_class_name\x18\x19 \x01(\tR\x11priorityClassName\x127\n" + + "\bpriority\x18\x1a \x01(\v2\x1b.google.protobuf.Int32ValueR\bpriority\x122\n" + + "\n" + + "dns_config\x18\x1b \x01(\v2\x13.spark.PodDNSConfigR\tdnsConfig\x12@\n" + + "\x0freadiness_gates\x18\x1c \x03(\v2\x17.spark.PodReadinessGateR\x0ereadinessGates\x12J\n" + + "\x12runtime_class_name\x18\x1d \x01(\v2\x1c.google.protobuf.StringValueR\x10runtimeClassName\x12L\n" + + "\x14enable_service_links\x18\x1e \x01(\v2\x1a.google.protobuf.BoolValueR\x12enableServiceLinks\x128\n" + + "\boverhead\x18\x1f \x03(\v2\x1c.spark.PodSpec.OverheadEntryR\boverhead\x12_\n" + + "\x1btopology_spread_constraints\x18 \x03(\v2\x1f.spark.TopologySpreadConstraintR\x19topologySpreadConstraints\x12L\n" + + "\x15set_host_name_as_fqdn\x18! \x01(\v2\x1a.google.protobuf.BoolValueR\x11setHostNameAsFqdn\x12\x1c\n" + + "\x02os\x18\" \x01(\v2\f.spark.PodOSR\x02os\x129\n" + + "\n" + + "host_users\x18# \x01(\v2\x1a.google.protobuf.BoolValueR\thostUsers\x12C\n" + + "\x10scheduling_gates\x18$ \x03(\v2\x18.spark.PodSchedulingGateR\x0fschedulingGates\x12@\n" + + "\x0fresource_claims\x18% \x03(\v2\x17.spark.PodResourceClaimR\x0eresourceClaims\x1a?\n" + + "\x11NodeSelectorEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\x1aL\n" + + "\rOverheadEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12%\n" + + "\x05value\x18\x02 \x01(\v2\x0f.spark.QuantityR\x05value:\x028\x01\"\xa7\x01\n" + + "\x12EphemeralContainer\x12]\n" + + "\x1aephemeral_container_common\x18\x01 \x01(\v2\x1f.spark.EphemeralContainerCommonR\x18ephemeralContainerCommon\x122\n" + + "\x15target_container_name\x18\x02 \x01(\tR\x13targetContainerName\"\x94\b\n" + + "\x18EphemeralContainerCommon\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n" + + "\x05image\x18\x02 \x01(\tR\x05image\x12\x18\n" + + "\acommand\x18\x03 \x03(\tR\acommand\x12\x12\n" + + "\x04args\x18\x04 \x03(\tR\x04args\x12\x1f\n" + + "\vworking_dir\x18\x05 \x01(\tR\n" + + "workingDir\x12*\n" + + "\x05ports\x18\x06 \x03(\v2\x14.spark.ContainerPortR\x05ports\x12/\n" + + "\benv_from\x18\a \x03(\v2\x14.spark.EnvFromSourceR\aenvFrom\x12\x1f\n" + + "\x03env\x18\b \x03(\v2\r.spark.EnvVarR\x03env\x129\n" + + "\tresources\x18\t \x01(\v2\x1b.spark.ResourceRequirementsR\tresources\x12A\n" + + "\rresize_policy\x18\n" + + " \x03(\v2\x1c.spark.ContainerResizePolicyR\fresizePolicy\x12D\n" + + "\x0erestart_policy\x18\v \x01(\x0e2\x1d.spark.ContainerRestartPolicyR\rrestartPolicy\x127\n" + + "\rvolume_mounts\x18\f \x03(\v2\x12.spark.VolumeMountR\fvolumeMounts\x12:\n" + + "\x0evolume_devices\x18\r \x03(\v2\x13.spark.VolumeDeviceR\rvolumeDevices\x125\n" + + "\x0freadiness_probe\x18\x0e \x01(\v2\f.spark.ProbeR\x0ereadinessProbe\x12/\n" + + "\n" + + "life_cycle\x18\x0f \x01(\v2\x10.spark.LifecycleR\tlifeCycle\x128\n" + + "\x18termination_message_path\x18\x10 \x01(\tR\x16terminationMessagePath\x12]\n" + + "\x1atermination_message_policy\x18\x11 \x01(\x0e2\x1f.spark.TerminationMessagePolicyR\x18terminationMessagePolicy\x12=\n" + + "\x11image_pull_policy\x18\x12 \x01(\x0e2\x11.spark.PullPolicyR\x0fimagePullPolicy\x12A\n" + + "\x10security_context\x18\x13 \x01(\v2\x16.spark.SecurityContextR\x0fsecurityContext\x12\x14\n" + + "\x05stdin\x18\x14 \x01(\bR\x05stdin\x12\x1d\n" + + "\n" + + "stdin_once\x18\x15 \x01(\bR\tstdinOnce\x12\x10\n" + + "\x03tty\x18\x16 \x01(\bR\x03tty\"R\n" + + "\x10PodReadinessGate\x12>\n" + + "\x0econdition_type\x18\x01 \x01(\x0e2\x17.spark.PodConditionTypeR\rconditionType\"\xea\x03\n" + + "\x18TopologySpreadConstraint\x12\x19\n" + + "\bmax_skew\x18\x01 \x01(\x05R\amaxSkew\x12!\n" + + "\ftopology_key\x18\x02 \x01(\tR\vtopologyKey\x12S\n" + + "\x12when_unsatisfiable\x18\x03 \x01(\x0e2$.spark.UnsatisfiableConstraintActionR\x11whenUnsatisfiable\x12;\n" + + "\x0elabel_selector\x18\x04 \x01(\v2\x14.spark.LabelSelectorR\rlabelSelector\x12<\n" + + "\vmin_domains\x18\x05 \x01(\v2\x1b.google.protobuf.Int32ValueR\n" + + "minDomains\x12L\n" + + "\x14node_affinity_policy\x18\x06 \x01(\x0e2\x1a.spark.NodeInclusionPolicyR\x12nodeAffinityPolicy\x12H\n" + + "\x12node_taints_policy\x18\a \x01(\x0e2\x1a.spark.NodeInclusionPolicyR\x10nodeTaintsPolicy\x12(\n" + + "\x10match_label_keys\x18\b \x03(\tR\x0ematchLabelKeys\"'\n" + + "\x11PodSchedulingGate\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\"\x1b\n" + + "\x05PodOS\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\"R\n" + + "\x10PodResourceClaim\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12*\n" + + "\x06source\x18\x02 \x01(\v2\x12.spark.ClaimSourceR\x06source\"\xba\x01\n" + + "\vClaimSource\x12L\n" + + "\x13resource_claim_name\x18\x01 \x01(\v2\x1c.google.protobuf.StringValueR\x11resourceClaimName\x12]\n" + + "\x1cresource_claim_template_name\x18\x02 \x01(\v2\x1c.google.protobuf.StringValueR\x19resourceClaimTemplateName\"9\n" + + "\aGPUSpec\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x1a\n" + + "\bquantity\x18\x02 \x01(\x03R\bquantity\"2\n" + + "\bNamePath\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x12\n" + + "\x04path\x18\x02 \x01(\tR\x04path\"[\n" + + "\n" + + "SecretInfo\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x12\n" + + "\x04path\x18\x02 \x01(\tR\x04path\x12%\n" + + "\x04type\x18\x03 \x01(\x0e2\x11.spark.SecretTypeR\x04type\"\xbf\x01\n" + + "\bAffinity\x128\n" + + "\rnode_affinity\x18\x01 \x01(\v2\x13.spark.NodeAffinityR\fnodeAffinity\x125\n" + + "\fpod_affinity\x18\x02 \x01(\v2\x12.spark.PodAffinityR\vpodAffinity\x12B\n" + + "\x11pod_anti_affinity\x18\x03 \x01(\v2\x16.spark.PodAntiAffinityR\x0fpodAntiAffinity\"\xb0\x02\n" + + "\x0fPodAntiAffinity\x12;\n" + + "\x0elabel_selector\x18\x01 \x01(\v2\x14.spark.LabelSelectorR\rlabelSelector\x12\x1e\n" + + "\n" + + "namespaces\x18\x02 \x03(\tR\n" + + "namespaces\x12!\n" + + "\ftopology_key\x18\x03 \x01(\tR\vtopologyKey\x12C\n" + + "\x12namespace_selector\x18\x04 \x01(\v2\x14.spark.LabelSelectorR\x11namespaceSelector\x12(\n" + + "\x10match_label_keys\x18\x05 \x03(\tR\x0ematchLabelKeys\x12.\n" + + "\x13mismatch_label_keys\x18\x06 \x03(\tR\x11mismatchLabelKeys\"\xa3\x02\n" + + "\vPodAffinity\x12\x83\x01\n" + + "3required_during_scheduling_ignored_during_execution\x18\x01 \x03(\v2\x16.spark.PodAffinityTermR.requiredDuringSchedulingIgnoredDuringExecution\x12\x8d\x01\n" + + "4preferred_during_scheduling_ignored_during_execution\x18\x02 \x03(\v2\x1e.spark.WeightedPodAffinityTermR/preferredDuringSchedulingIgnoredDuringExecution\"u\n" + + "\x17WeightedPodAffinityTerm\x12\x16\n" + + "\x06weight\x18\x01 \x01(\x05R\x06weight\x12B\n" + + "\x11pod_affinity_term\x18\x02 \x01(\v2\x16.spark.PodAffinityTermR\x0fpodAffinityTerm\"\xb0\x02\n" + + "\x0fPodAffinityTerm\x12;\n" + + "\x0elabel_selector\x18\x01 \x01(\v2\x14.spark.LabelSelectorR\rlabelSelector\x12\x1e\n" + + "\n" + + "namespaces\x18\x02 \x03(\tR\n" + + "namespaces\x12!\n" + + "\ftopology_key\x18\x03 \x01(\tR\vtopologyKey\x12C\n" + + "\x12namespace_selector\x18\x04 \x01(\v2\x14.spark.LabelSelectorR\x11namespaceSelector\x12(\n" + + "\x10match_label_keys\x18\x05 \x03(\tR\x0ematchLabelKeys\x12.\n" + + "\x13mismatch_label_keys\x18\x06 \x03(\tR\x11mismatchLabelKeys\"\xe7\x01\n" + + "\rLabelSelector\x12H\n" + + "\fmatch_labels\x18\x01 \x03(\v2%.spark.LabelSelector.MatchLabelsEntryR\vmatchLabels\x12L\n" + + "\x11match_expressions\x18\x02 \x01(\v2\x1f.spark.LabelSelectorRequirementR\x10matchExpressions\x1a>\n" + + "\x10MatchLabelsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"~\n" + + "\x18LabelSelectorRequirement\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x128\n" + + "\boperator\x18\x02 \x01(\x0e2\x1c.spark.LabelSelectorOperatorR\boperator\x12\x16\n" + + "\x06values\x18\x03 \x03(\tR\x06values\"\xa1\x02\n" + + "\fNodeAffinity\x12\x80\x01\n" + + "3required_during_scheduling_ignored_during_execution\x18\x01 \x01(\v2\x13.spark.NodeSelectorR.requiredDuringSchedulingIgnoredDuringExecution\x12\x8d\x01\n" + + "4preferred_during_scheduling_ignored_during_execution\x18\x02 \x03(\v2\x1e.spark.PreferredSchedulingTermR/preferredDuringSchedulingIgnoredDuringExecution\"j\n" + + "\x17PreferredSchedulingTerm\x12\x16\n" + + "\x06weight\x18\x01 \x01(\x05R\x06weight\x127\n" + + "\n" + + "preference\x18\x02 \x01(\v2\x17.spark.NodeSelectorTermR\n" + + "preference\"W\n" + + "\fNodeSelector\x12G\n" + + "\x13node_selector_terms\x18\x01 \x03(\v2\x17.spark.NodeSelectorTermR\x11nodeSelectorTerms\"\xa2\x01\n" + + "\x10NodeSelectorTerm\x12K\n" + + "\x11match_expressions\x18\x01 \x03(\v2\x1e.spark.NodeSelectorRequirementR\x10matchExpressions\x12A\n" + + "\fmatch_fields\x18\x02 \x03(\v2\x1e.spark.NodeSelectorRequirementR\vmatchFields\"|\n" + + "\x17NodeSelectorRequirement\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x127\n" + + "\boperator\x18\x02 \x01(\x0e2\x1b.spark.NodeSelectorOperatorR\boperator\x12\x16\n" + + "\x06values\x18\x03 \x03(\tR\x06values\"\xe3\x01\n" + + "\n" + + "Toleration\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x125\n" + + "\boperator\x18\x02 \x01(\x0e2\x19.spark.TolerationOperatorR\boperator\x12\x14\n" + + "\x05value\x18\x03 \x01(\tR\x05value\x12*\n" + + "\x06effect\x18\x04 \x01(\x0e2\x12.spark.TaintEffectR\x06effect\x12J\n" + + "\x12toleration_seconds\x18\x05 \x01(\v2\x1b.google.protobuf.Int64ValueR\x11tolerationSeconds\"\xa8\x05\n" + + "\x12PodSecurityContext\x12?\n" + + "\x10se_linux_options\x18\x01 \x01(\v2\x15.spark.SELinuxOptionsR\x0eseLinuxOptions\x12m\n" + + " windows_security_context_options\x18\x02 \x01(\v2$.spark.WindowsSecurityContextOptionsR\x1dwindowsSecurityContextOptions\x12;\n" + + "\vrun_as_user\x18\x03 \x01(\v2\x1b.google.protobuf.Int64ValueR\trunAsUser\x12=\n" + + "\frun_as_group\x18\x04 \x01(\v2\x1b.google.protobuf.Int64ValueR\n" + + "runAsGroup\x12@\n" + + "\x0erun_as_nonroot\x18\x05 \x01(\v2\x1a.google.protobuf.BoolValueR\frunAsNonroot\x12/\n" + + "\x13supplemental_groups\x18\x06 \x03(\x03R\x12supplementalGroups\x126\n" + + "\bfs_group\x18\a \x01(\v2\x1b.google.protobuf.Int64ValueR\afsGroup\x12&\n" + + "\asys_ctl\x18\b \x03(\v2\r.spark.SysctlR\x06sysCtl\x12R\n" + + "\x16fs_group_change_policy\x18\t \x01(\x0e2\x1d.spark.PodFSGroupChangePolicyR\x13fsGroupChangePolicy\x12?\n" + + "\x10sec_comp_profile\x18\n" + + " \x01(\v2\x15.spark.SeccompProfileR\x0esecCompProfile\"2\n" + + "\x06Sysctl\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value\"\xed\b\n" + + "\tContainer\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n" + + "\x05image\x18\x02 \x01(\tR\x05image\x12\x18\n" + + "\acommand\x18\x03 \x03(\tR\acommand\x12\x12\n" + + "\x04args\x18\x04 \x03(\tR\x04args\x12\x1f\n" + + "\vworking_dir\x18\x05 \x01(\tR\n" + + "workingDir\x12*\n" + + "\x05ports\x18\x06 \x03(\v2\x14.spark.ContainerPortR\x05ports\x12/\n" + + "\benv_from\x18\a \x03(\v2\x14.spark.EnvFromSourceR\aenvFrom\x12\x1f\n" + + "\x03env\x18\b \x03(\v2\r.spark.EnvVarR\x03env\x129\n" + + "\tresources\x18\t \x01(\v2\x1b.spark.ResourceRequirementsR\tresources\x12A\n" + + "\rresize_policy\x18\n" + + " \x03(\v2\x1c.spark.ContainerResizePolicyR\fresizePolicy\x12D\n" + + "\x0erestart_policy\x18\v \x01(\x0e2\x1d.spark.ContainerRestartPolicyR\rrestartPolicy\x127\n" + + "\rvolume_mounts\x18\f \x03(\v2\x12.spark.VolumeMountR\fvolumeMounts\x12:\n" + + "\x0evolume_devices\x18\r \x03(\v2\x13.spark.VolumeDeviceR\rvolumeDevices\x123\n" + + "\x0eliveness_probe\x18\x0e \x01(\v2\f.spark.ProbeR\rlivenessProbe\x125\n" + + "\x0freadiness_probe\x18\x0f \x01(\v2\f.spark.ProbeR\x0ereadinessProbe\x121\n" + + "\rstartup_probe\x18\x10 \x01(\v2\f.spark.ProbeR\fstartupProbe\x12/\n" + + "\n" + + "life_cycle\x18\x11 \x01(\v2\x10.spark.LifecycleR\tlifeCycle\x128\n" + + "\x18termination_message_path\x18\x12 \x01(\tR\x16terminationMessagePath\x12]\n" + + "\x1atermination_message_policy\x18\x13 \x01(\x0e2\x1f.spark.TerminationMessagePolicyR\x18terminationMessagePolicy\x12=\n" + + "\x11image_pull_policy\x18\x14 \x01(\x0e2\x11.spark.PullPolicyR\x0fimagePullPolicy\x12A\n" + + "\x10security_context\x18\x15 \x01(\v2\x16.spark.SecurityContextR\x0fsecurityContext\x12\x14\n" + + "\x05stdin\x18\x16 \x01(\bR\x05stdin\x12\x1d\n" + + "\n" + + "stdin_once\x18\x17 \x01(\bR\tstdinOnce\x12\x10\n" + + "\x03tty\x18\x18 \x01(\bR\x03tty\"\xad\x01\n" + + "\rContainerPort\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x1b\n" + + "\thost_port\x18\x02 \x01(\x05R\bhostPort\x12%\n" + + "\x0econtainer_port\x18\x03 \x01(\x05R\rcontainerPort\x12+\n" + + "\bprotocol\x18\x04 \x01(\x0e2\x0f.spark.ProtocolR\bprotocol\x12\x17\n" + + "\ahost_ip\x18\x05 \x01(\tR\x06hostIp\"\x9f\x01\n" + + "\x12ConfigMapEnvSource\x12Q\n" + + "\x16local_object_reference\x18\x01 \x01(\v2\x1b.spark.LocalObjectReferenceR\x14localObjectReference\x126\n" + + "\boptional\x18\x02 \x01(\v2\x1a.google.protobuf.BoolValueR\boptional\"\x9f\x01\n" + + "\rEnvFromSource\x12\x16\n" + + "\x06prefix\x18\x01 \x01(\tR\x06prefix\x12?\n" + + "\x0econfig_map_ref\x18\x02 \x01(\v2\x19.spark.ConfigMapEnvSourceR\fconfigMapRef\x125\n" + + "\n" + + "secret_ref\x18\x03 \x01(\v2\x16.spark.SecretEnvSourceR\tsecretRef\"\x9c\x01\n" + + "\x0fSecretEnvSource\x12Q\n" + + "\x16local_object_reference\x18\x01 \x01(\v2\x1b.spark.LocalObjectReferenceR\x14localObjectReference\x126\n" + + "\boptional\x18\x03 \x01(\v2\x1a.google.protobuf.BoolValueR\boptional\"f\n" + + "\x06EnvVar\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value\x122\n" + + "\n" + + "value_from\x18\x03 \x01(\v2\x13.spark.EnvVarSourceR\tvalueFrom\"\x9d\x02\n" + + "\fEnvVarSource\x127\n" + + "\tfield_ref\x18\x01 \x01(\v2\x1a.spark.ObjectFieldSelectorR\bfieldRef\x12J\n" + + "\x12resource_field_ref\x18\x02 \x01(\v2\x1c.spark.ResourceFieldSelectorR\x10resourceFieldRef\x12H\n" + + "\x12config_map_key_ref\x18\x03 \x01(\v2\x1b.spark.ConfigMapKeySelectorR\x0fconfigMapKeyRef\x12>\n" + + "\x0esecret_key_ref\x18\x04 \x01(\v2\x18.spark.SecretKeySelectorR\fsecretKeyRef\"\xb0\x01\n" + + "\x11SecretKeySelector\x12Q\n" + + "\x16local_object_reference\x18\x01 \x01(\v2\x1b.spark.LocalObjectReferenceR\x14localObjectReference\x12\x10\n" + + "\x03key\x18\x02 \x01(\tR\x03key\x126\n" + + "\boptional\x18\x03 \x01(\v2\x1a.google.protobuf.BoolValueR\boptional\"\xb3\x01\n" + + "\x14ConfigMapKeySelector\x12Q\n" + + "\x16local_object_reference\x18\x01 \x01(\v2\x1b.spark.LocalObjectReferenceR\x14localObjectReference\x12\x10\n" + + "\x03key\x18\x02 \x01(\tR\x03key\x126\n" + + "\boptional\x18\x03 \x01(\v2\x1a.google.protobuf.BoolValueR\boptional\"*\n" + + "\x14LocalObjectReference\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\"\x85\x01\n" + + "\x15ResourceFieldSelector\x12%\n" + + "\x0econtainer_name\x18\x01 \x01(\tR\rcontainerName\x12\x1a\n" + + "\bresource\x18\x02 \x01(\tR\bresource\x12)\n" + + "\adivisor\x18\x03 \x01(\v2\x0f.spark.QuantityR\adivisor\"U\n" + + "\x13ObjectFieldSelector\x12\x1f\n" + + "\vapi_version\x18\x01 \x01(\tR\n" + + "apiVersion\x12\x1d\n" + + "\n" + + "field_path\x18\x02 \x01(\tR\tfieldPath\"\xe6\x02\n" + + "\x14ResourceRequirements\x12?\n" + + "\x06limits\x18\x01 \x03(\v2'.spark.ResourceRequirements.LimitsEntryR\x06limits\x12E\n" + + "\brequests\x18\x02 \x03(\v2).spark.ResourceRequirements.RequestsEntryR\brequests\x12,\n" + + "\x06claims\x18\x03 \x03(\v2\x14.spark.ResourceClaimR\x06claims\x1aJ\n" + + "\vLimitsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12%\n" + + "\x05value\x18\x02 \x01(\v2\x0f.spark.QuantityR\x05value:\x028\x01\x1aL\n" + + "\rRequestsEntry\x12\x10\n" + + "\x03key\x18\x01 \x01(\tR\x03key\x12%\n" + + "\x05value\x18\x02 \x01(\v2\x0f.spark.QuantityR\x05value:\x028\x01\"#\n" + + "\rResourceClaim\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\"e\n" + + "\x11ResourceListEntry\x12#\n" + + "\rresource_name\x18\x01 \x01(\tR\fresourceName\x12+\n" + + "\bquantity\x18\x02 \x01(\v2\x0f.spark.QuantityR\bquantity\"\x84\x01\n" + + "\bQuantity\x12 \n" + + "\x01i\x18\x01 \x01(\v2\x12.spark.Int64AmountR\x01i\x12!\n" + + "\x01d\x18\x02 \x01(\v2\x13.spark.InfDecAmountR\x01d\x12\f\n" + + "\x01s\x18\x03 \x01(\tR\x01s\x12%\n" + + "\x06format\x18\x04 \x01(\x0e2\r.spark.FormatR\x06format\" \n" + + "\fInfDecAmount\x12\x10\n" + + "\x03dec\x18\x01 \x01(\tR\x03dec\"G\n" + + "\vInt64Amount\x12\x14\n" + + "\x05value\x18\x01 \x01(\x03R\x05value\x12\"\n" + + "\x05scale\x18\x02 \x01(\v2\f.spark.ScaleR\x05scale\"\x1d\n" + + "\x05Scale\x12\x14\n" + + "\x05value\x18\x01 \x01(\x05R\x05value\"\x87\x01\n" + + "\x15ContainerResizePolicy\x12#\n" + + "\rresource_name\x18\x01 \x01(\tR\fresourceName\x12I\n" + + "\x0erestart_policy\x18\x02 \x01(\x0e2\".spark.ResourceResizeRestartPolicyR\rrestartPolicy\"C\n" + + "\fVolumeDevice\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n" + + "\vdevice_path\x18\x02 \x01(\tR\n" + + "devicePath\"\x9d\x01\n" + + "\fProbeHandler\x12%\n" + + "\x04exec\x18\x01 \x01(\v2\x11.spark.ExecActionR\x04exec\x12/\n" + + "\bhttp_get\x18\x02 \x01(\v2\x14.spark.HTTPGetActionR\ahttpGet\x125\n" + + "\n" + + "tcp_socket\x18\x03 \x01(\v2\x16.spark.TCPSocketActionR\ttcpSocket\"\x85\x03\n" + + "\x05Probe\x128\n" + + "\rprobe_handler\x18\x01 \x01(\v2\x13.spark.ProbeHandlerR\fprobeHandler\x122\n" + + "\x15initial_delay_seconds\x18\x02 \x01(\x05R\x13initialDelaySeconds\x12'\n" + + "\x0ftimeout_seconds\x18\x03 \x01(\x05R\x0etimeoutSeconds\x12%\n" + + "\x0eperiod_seconds\x18\x04 \x01(\x05R\rperiodSeconds\x12+\n" + + "\x11success_threshold\x18\x05 \x01(\x05R\x10successThreshold\x12+\n" + + "\x11failure_threshold\x18\x06 \x01(\x05R\x10failureThreshold\x12d\n" + + " termination_grace_period_seconds\x18\a \x01(\v2\x1b.google.protobuf.Int64ValueR\x1dterminationGracePeriodSeconds\"\x94\x06\n" + + "\x0fSecurityContext\x127\n" + + "\fcapabilities\x18\x01 \x01(\v2\x13.spark.CapabilitiesR\fcapabilities\x12:\n" + + "\n" + + "privileged\x18\x02 \x01(\v2\x1a.google.protobuf.BoolValueR\n" + + "privileged\x12?\n" + + "\x10se_linux_options\x18\x03 \x01(\v2\x15.spark.SELinuxOptionsR\x0eseLinuxOptions\x12m\n" + + " windows_security_context_options\x18\x04 \x01(\v2$.spark.WindowsSecurityContextOptionsR\x1dwindowsSecurityContextOptions\x12;\n" + + "\vrun_as_user\x18\x05 \x01(\v2\x1b.google.protobuf.Int64ValueR\trunAsUser\x12=\n" + + "\frun_as_group\x18\x06 \x01(\v2\x1b.google.protobuf.Int64ValueR\n" + + "runAsGroup\x12A\n" + + "\x0frun_as_non_root\x18\a \x01(\v2\x1a.google.protobuf.BoolValueR\frunAsNonRoot\x12M\n" + + "\x15read_only_file_system\x18\b \x01(\v2\x1a.google.protobuf.BoolValueR\x12readOnlyFileSystem\x12X\n" + + "\x1aallow_privilege_escalation\x18\t \x01(\v2\x1a.google.protobuf.BoolValueR\x18allowPrivilegeEscalation\x123\n" + + "\n" + + "proc_mount\x18\n" + + " \x01(\x0e2\x14.spark.ProcMountTypeR\tprocMount\x12?\n" + + "\x10sec_comp_profile\x18\v \x01(\v2\x15.spark.SeccompProfileR\x0esecCompProfile\"4\n" + + "\fCapabilities\x12\x10\n" + + "\x03add\x18\x01 \x03(\tR\x03add\x12\x12\n" + + "\x04drop\x18\x02 \x03(\tR\x04drop\"b\n" + + "\x0eSELinuxOptions\x12\x12\n" + + "\x04user\x18\x01 \x01(\tR\x04user\x12\x12\n" + + "\x04role\x18\x02 \x01(\tR\x04role\x12\x12\n" + + "\x04type\x18\x03 \x01(\tR\x04type\x12\x14\n" + + "\x05level\x18\x04 \x01(\tR\x05level\"\xce\x02\n" + + "\x1dWindowsSecurityContextOptions\x12W\n" + + "\x19gmsa_credential_spec_name\x18\x01 \x01(\v2\x1c.google.protobuf.StringValueR\x16gmsaCredentialSpecName\x12N\n" + + "\x14gmsa_credential_spec\x18\x02 \x01(\v2\x1c.google.protobuf.StringValueR\x12gmsaCredentialSpec\x12E\n" + + "\x10run_as_user_name\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\rrunAsUserName\x12=\n" + + "\fhost_process\x18\x04 \x01(\v2\x1a.google.protobuf.BoolValueR\vhostProcess\"\x8b\x01\n" + + "\x0eSeccompProfile\x12-\n" + + "\x04type\x18\x01 \x01(\x0e2\x19.spark.SeccompProfileTypeR\x04type\x12J\n" + + "\x12local_host_profile\x18\x02 \x01(\v2\x1c.google.protobuf.StringValueR\x10localHostProfile\"\x82\x01\n" + + "\fPodDNSConfig\x12!\n" + + "\fname_servers\x18\x01 \x03(\tR\vnameServers\x12\x1a\n" + + "\bsearches\x18\x02 \x03(\tR\bsearches\x123\n" + + "\aoptions\x18\x03 \x03(\v2\x19.spark.PodDNSConfigOptionR\aoptions\">\n" + + "\x12PodDNSConfigOption\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value\":\n" + + "\tHostAlias\x12\x0e\n" + + "\x02ip\x18\x01 \x01(\tR\x02ip\x12\x1d\n" + + "\n" + + "host_names\x18\x02 \x03(\tR\thostNames\"w\n" + + "\tLifecycle\x126\n" + + "\n" + + "post_start\x18\x01 \x01(\v2\x17.spark.LifecycleHandlerR\tpostStart\x122\n" + + "\bpre_stop\x18\x02 \x01(\v2\x17.spark.LifecycleHandlerR\apreStop\"\xcb\x01\n" + + "\x10LifecycleHandler\x12%\n" + + "\x04exec\x18\x01 \x01(\v2\x11.spark.ExecActionR\x04exec\x12/\n" + + "\bhttp_get\x18\x02 \x01(\v2\x14.spark.HTTPGetActionR\ahttpGet\x125\n" + + "\n" + + "tcp_socket\x18\x03 \x01(\v2\x16.spark.TCPSocketActionR\ttcpSocket\x12(\n" + + "\x05sleep\x18\x04 \x01(\v2\x12.spark.SleepActionR\x05sleep\"'\n" + + "\vSleepAction\x12\x18\n" + + "\aseconds\x18\x01 \x01(\x03R\aseconds\"M\n" + + "\x0fTCPSocketAction\x12&\n" + + "\x04port\x18\x01 \x01(\v2\x12.spark.IntOrStringR\x04port\x12\x12\n" + + "\x04host\x18\x02 \x01(\tR\x04host\"&\n" + + "\n" + + "ExecAction\x12\x18\n" + + "\acommand\x18\x01 \x03(\tR\acommand\"\xbf\x01\n" + + "\rHTTPGetAction\x12\x12\n" + + "\x04path\x18\x01 \x01(\tR\x04path\x12&\n" + + "\x04port\x18\x02 \x01(\v2\x12.spark.IntOrStringR\x04port\x12\x12\n" + + "\x04host\x18\x03 \x01(\tR\x04host\x12(\n" + + "\x06scheme\x18\x04 \x01(\x0e2\x10.spark.URISchemeR\x06scheme\x124\n" + + "\fhttp_headers\x18\x05 \x03(\v2\x11.spark.HTTPHeaderR\vhttpHeaders\"6\n" + + "\n" + + "HTTPHeader\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n" + + "\x05value\x18\x02 \x01(\tR\x05value\"K\n" + + "\vIntOrString\x12\x19\n" + + "\aint_val\x18\x01 \x01(\x05H\x00R\x06intVal\x12\x19\n" + + "\astr_val\x18\x02 \x01(\tH\x00R\x06strValB\x06\n" + + "\x04type\"^\n" + + "\x05Ports\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x1a\n" + + "\bprotocol\x18\x02 \x01(\tR\bprotocol\x12%\n" + + "\x0econtainer_port\x18\x03 \x01(\tR\rcontainerPort\"\xf9\x03\n" + + "\fExecutorSpec\x129\n" + + "\x0espark_pod_spec\x18\x01 \x01(\v2\x13.spark.SparkPodSpecR\fsparkPodSpec\x129\n" + + "\tinstances\x18\x02 \x01(\v2\x1b.google.protobuf.Int32ValueR\tinstances\x12?\n" + + "\fcore_request\x18\x03 \x01(\v2\x1c.google.protobuf.StringValueR\vcoreRequest\x12?\n" + + "\fjava_options\x18\x04 \x01(\v2\x1c.google.protobuf.StringValueR\vjavaOptions\x12/\n" + + "\n" + + "life_cycle\x18\x05 \x01(\v2\x10.spark.LifecycleR\tlifeCycle\x12N\n" + + "\x15delete_on_termination\x18\x06 \x01(\v2\x1a.google.protobuf.BoolValueR\x13deleteOnTermination\x12\"\n" + + "\x05ports\x18\a \x03(\v2\f.spark.PortsR\x05ports\x12L\n" + + "\x13priority_class_name\x18\b \x01(\v2\x1c.google.protobuf.StringValueR\x11priorityClassName\"D\n" + + "\x06Volume\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x12\n" + + "\x04type\x18\x02 \x01(\tR\x04type\x12\x12\n" + + "\x04path\x18\x03 \x01(\tR\x04path\"]\n" + + "\vVolumeMount\x12\x12\n" + + "\x04name\x18\x01 \x01(\tR\x04name\x12\x1d\n" + + "\n" + + "mount_path\x18\x02 \x01(\tR\tmountPath\x12\x1b\n" + + "\tread_only\x18\x03 \x01(\bR\breadOnly\"\x9c\x01\n" + + "\x16SparkApplicationStatus\x12+\n" + + "\x11application_state\x18\x01 \x01(\tR\x10applicationState\x120\n" + + "\x14spark_application_id\x18\x02 \x01(\tR\x12sparkApplicationId\x12#\n" + + "\rsubmission_id\x18\x03 \x01(\tR\fsubmissionId\"\xa9\x01\n" + + "\x10SparkApplication\x12-\n" + + "\bmetadata\x18\x01 \x01(\v2\x11.spark.ObjectMetaR\bmetadata\x12/\n" + + "\x04spec\x18\x02 \x01(\v2\x1b.spark.SparkApplicationSpecR\x04spec\x125\n" + + "\x06status\x18\x03 \x01(\v2\x1d.spark.SparkApplicationStatusR\x06status\"\x85\x01\n" + + "\x18RunAltSparkSubmitRequest\x12D\n" + + "\x11spark_application\x18\x01 \x01(\v2\x17.spark.SparkApplicationR\x10sparkApplication\x12#\n" + + "\rsubmission_id\x18\x02 \x01(\tR\fsubmissionId\"Z\n" + + "\x19RunAltSparkSubmitResponse\x12\x18\n" + + "\asuccess\x18\x01 \x01(\bR\asuccess\x12#\n" + + "\rerror_message\x18\x02 \x01(\tR\ferrorMessage\"\xda\x01\n" + + "\fDependencies\x12\x12\n" + + "\x04jars\x18\x01 \x03(\tR\x04jars\x12\x14\n" + + "\x05files\x18\x02 \x03(\tR\x05files\x12\x19\n" + + "\bpy_files\x18\x03 \x03(\tR\apyFiles\x12\x1a\n" + + "\bpackages\x18\x04 \x03(\tR\bpackages\x12)\n" + + "\x10exclude_packages\x18\x05 \x03(\tR\x0fexcludePackages\x12\"\n" + + "\frepositories\x18\x06 \x03(\tR\frepositories\x12\x1a\n" + + "\barchives\x18\a \x03(\tR\barchives\"\xde\x01\n" + + "\x11DynamicAllocation\x12\x18\n" + + "\aenabled\x18\x01 \x01(\bR\aenabled\x12+\n" + + "\x11initial_executors\x18\x02 \x01(\x05R\x10initialExecutors\x12#\n" + + "\rmin_executors\x18\x03 \x01(\x05R\fminExecutors\x12#\n" + + "\rmax_executors\x18\x04 \x01(\x05R\fmaxExecutors\x128\n" + + "\x18shuffle_tracking_timeout\x18\x05 \x01(\x03R\x16shuffleTrackingTimeout*\x9e\x01\n" + + "\x1aManagedFieldsOperationType\x12-\n" + + ")MANAGED_FIELDS_OPERATION_TYPE_UNSPECIFIED\x10\x00\x12'\n" + + "#MANAGED_FIELDS_OPERATION_TYPE_APPLY\x10\x01\x12(\n" + + "$MANAGED_FIELDS_OPERATION_TYPE_UPDATE\x10\x02*\xc2\x01\n" + + "\x14SparkApplicationType\x12&\n" + + "\"SPARK_APPLICATION_TYPE_UNSPECIFIED\x10\x00\x12\x1f\n" + + "\x1bSPARK_APPLICATION_TYPE_JAVA\x10\x01\x12 \n" + + "\x1cSPARK_APPLICATION_TYPE_SCALA\x10\x02\x12!\n" + + "\x1dSPARK_APPLICATION_TYPE_PYTHON\x10\x03\x12\x1c\n" + + "\x18SPARK_APPLICATION_TYPE_R\x10\x04*}\n" + + "\n" + + "DeployMode\x12\x1b\n" + + "\x17DEPLOY_MODE_UNSPECIFIED\x10\x00\x12\x17\n" + + "\x13DEPLOY_MODE_CLUSTER\x10\x01\x12\x16\n" + + "\x12DEPLOY_MODE_CLIENT\x10\x02\x12!\n" + + "\x1dDEPLOY_MODE_IN_CLUSTER_CLIENT\x10\x03*\xa4\x01\n" + + "\vServiceType\x12\x1c\n" + + "\x18SERVICE_TYPE_UNSPECIFIED\x10\x00\x12\x1b\n" + + "\x17SERVICE_TYPE_CLUSTER_IP\x10\x01\x12\x1a\n" + + "\x16SERVICE_TYPE_NODE_PORT\x10\x02\x12\x1e\n" + + "\x1aSERVICE_TYPE_LOAD_BALANCER\x10\x03\x12\x1e\n" + + "\x1aSERVICE_TYPE_EXTERNAL_NAME\x10\x04*\x9e\x01\n" + + "\tDNSPolicy\x12\x1a\n" + + "\x16DNS_POLICY_UNSPECIFIED\x10\x00\x12*\n" + + "&DNS_POLICY_CLUSTER_FIRST_WITH_HOST_NET\x10\x01\x12\x1c\n" + + "\x18DNS_POLICY_CLUSTER_FIRST\x10\x02\x12\x16\n" + + "\x12DNS_POLICY_DEFAULT\x10\x03\x12\x13\n" + + "\x0fDNS_POLICY_NONE\x10\x04*\xa7\x02\n" + + "\x10PodConditionType\x12\"\n" + + "\x1ePOD_CONDITION_TYPE_UNSPECIFIED\x10\x00\x12'\n" + + "#POD_CONDITION_TYPE_CONTAINERS_READY\x10\x01\x12\"\n" + + "\x1ePOD_CONDITION_TYPE_INITIALIZED\x10\x02\x12\x1c\n" + + "\x18POD_CONDITION_TYPE_READY\x10\x03\x12$\n" + + " POD_CONDITION_TYPE_POD_SCHEDULED\x10\x04\x12(\n" + + "$POD_CONDITION_TYPE_DISRUPTION_TARGET\x10\x05\x124\n" + + "0POD_CONDITION_TYPE_POD_READY_TO_START_CONTAINERS\x10\x06*\xba\x01\n" + + "\x1dUnsatisfiableConstraintAction\x12/\n" + + "+UNSATISFIABLE_CONSTRAINT_ACTION_UNSPECIFIED\x10\x00\x123\n" + + "/UNSATISFIABLE_CONSTRAINT_ACTION_DO_NOT_SCHEDULE\x10\x01\x123\n" + + "/UNSATISFIABLE_CONSTRAINT_ACTION_SCHEDULE_ANYWAY\x10\x02*\x7f\n" + + "\x13NodeInclusionPolicy\x12%\n" + + "!NODE_INCLUSION_POLICY_UNSPECIFIED\x10\x00\x12 \n" + + "\x1cNODE_INCLUSION_POLCIY_IGNORE\x10\x01\x12\x1f\n" + + "\x1bNODE_INCLUSION_POLCIY_HONOR\x10\x02*\x90\x01\n" + + "\n" + + "SecretType\x12\x1b\n" + + "\x17SECRET_TYPE_UNSPECIFIED\x10\x00\x12#\n" + + "\x1fSECRET_TYPE_GCP_SERVICE_ACCOUNT\x10\x01\x12'\n" + + "#SECRET_TYPE_HADOOP_DELEGATION_TOKEN\x10\x02\x12\x17\n" + + "\x13SECRET_TYPE_GENERIC\x10\x03*\xd4\x01\n" + + "\x15LabelSelectorOperator\x12'\n" + + "#LABEL_SELECTOR_OPERATOR_UNSPECIFIED\x10\x00\x12\x1e\n" + + "\x1aLABEL_SELECTOR_OPERATOR_IN\x10\x01\x12\"\n" + + "\x1eLABEL_SELECTOR_OPERATOR_NOT_IN\x10\x02\x12\"\n" + + "\x1eLABEL_SELECTOR_OPERATOR_EXISTS\x10\x03\x12*\n" + + "&LABEL_SELECTOR_OPERATOR_DOES_NOT_EXIST\x10\x04*\x8c\x02\n" + + "\x14NodeSelectorOperator\x12&\n" + + "\"NODE_SELECTOR_OPERATOR_UNSPECIFIED\x10\x00\x12\x1d\n" + + "\x19NODE_SELECTOR_OPERATOR_IN\x10\x01\x12!\n" + + "\x1dNODE_SELECTOR_OPERATOR_NOT_IN\x10\x02\x12!\n" + + "\x1dNODE_SELECTOR_OPERATOR_EXISTS\x10\x03\x12)\n" + + "%NODE_SELECTOR_OPERATOR_DOES_NOT_EXIST\x10\x04\x12\x1d\n" + + "\x19NODE_SELECTOR_OPERATOR_GT\x10\x05\x12\x1d\n" + + "\x19NODE_SELECTOR_OPERATOR_LT\x10\x06*\xb2\x01\n" + + "\vTaintEffect\x12\x1c\n" + + "\x18TAINT_EFFECT_UNSPECIFIED\x10\x00\x12\x1c\n" + + "\x18TAINT_EFFECT_NO_SCHEDULE\x10\x01\x12#\n" + + "\x1fTAINT_EFFECT_PREFER_NO_SCHEDULE\x10\x02\x12%\n" + + "!TAINT_EFFECT_NO_SCHEDULE_NO_ADMIT\x10\x03\x12\x1b\n" + + "\x17TAINT_EFFECT_NO_EXECUTE\x10\x04*x\n" + + "\x12TolerationOperator\x12#\n" + + "\x1fTOLERATION_OPERATOR_UNSPECIFIED\x10\x00\x12\x1e\n" + + "\x1aTOLERATION_OPERATOR_EXISTS\x10\x01\x12\x1d\n" + + "\x19TOLERATION_OPERATOR_EQUAL\x10\x02*\x9c\x01\n" + + "\x16PodFSGroupChangePolicy\x12*\n" + + "&POD_FS_GROUP_CHANGE_POLICY_UNSPECIFIED\x10\x00\x12/\n" + + "+POD_FS_GROUP_CHANGE_POLICY_ON_ROOT_MISMATCH\x10\x01\x12%\n" + + "!POD_FS_GROUP_CHANGE_POLICY_ALWAYS\x10\x02*[\n" + + "\bProtocol\x12\x18\n" + + "\x14PROTOCOL_UNSPECIFIED\x10\x00\x12\x10\n" + + "\fPROTOCOL_TCP\x10\x01\x12\x10\n" + + "\fPROTOCOL_UDP\x10\x02\x12\x11\n" + + "\rPROTOCOL_SCTP\x10\x03*U\n" + + "\x06Format\x12\x16\n" + + "\x12FORMAT_UNSPECIFIED\x10\x00\x12\x14\n" + + "\x10DECIMAL_EXPONENT\x10\x01\x12\r\n" + + "\tBINARY_SI\x10\x02\x12\x0e\n" + + "\n" + + "DECIMAL_SI\x10\x03*\xb4\x01\n" + + "\x1bResourceResizeRestartPolicy\x12.\n" + + "*RESOURCE_RESIZE_RESTART_POLICY_UNSPECIFIED\x10\x00\x12/\n" + + "+RESOURCE_RESIZE_RESTART_POLICY_NOT_REQUIRED\x10\x01\x124\n" + + "0RESOURCE_RESIZE_RESTART_POLICY_RESTART_CONTAINER\x10\x02*g\n" + + "\x16ContainerRestartPolicy\x12(\n" + + "$CONTAINER_RESTART_POLICY_UNSPECIFIED\x10\x00\x12#\n" + + "\x1fCONTAINER_RESTART_POLICY_ALWAYS\x10\x01*\x9e\x01\n" + + "\x18TerminationMessagePolicy\x12*\n" + + "&TERMINATION_MESSAGE_POLICY_UNSPECIFIED\x10\x00\x12#\n" + + "\x1fTERMINATION_MESSAGE_POLICY_FILE\x10\x01\x121\n" + + "-TERMINATION_MESSAGE_FALLBACK_TO_LOGS_ON_ERROR\x10\x02*x\n" + + "\n" + + "PullPolicy\x12\x1b\n" + + "\x17PULL_POLICY_UNSPECIFIED\x10\x00\x12\x16\n" + + "\x12PULL_POLICY_ALWAYS\x10\x01\x12\x15\n" + + "\x11PULL_POLICY_NEVER\x10\x02\x12\x1e\n" + + "\x1aPULL_POLICY_IF_NOT_PRESENT\x10\x03*k\n" + + "\rProcMountType\x12\x1f\n" + + "\x1bPROC_MOUNT_TYPE_UNSPECIFIED\x10\x00\x12\x1b\n" + + "\x17PROC_MOUNT_TYPE_DEFAULT\x10\x01\x12\x1c\n" + + "\x18PROC_MOUNT_TYPE_UNMASKED\x10\x02*\xad\x01\n" + + "\x12SeccompProfileType\x12$\n" + + " SECCOMP_PROFILE_TYPE_UNSPECIFIED\x10\x00\x12\"\n" + + "\x1eSECCOMP_PROFILE_TYPE_LOCALHOST\x10\x01\x12(\n" + + "$SECCOMP_PROFILE_TYPE_RUNTIME_DEFAULT\x10\x02\x12#\n" + + "\x1fSECCOMP_PROFILE_TYPE_UNCONFINED\x10\x03*O\n" + + "\tURIScheme\x12\x19\n" + + "\x15URISCHEME_UNSPECIFIED\x10\x00\x12\x12\n" + + "\x0eURISCHEME_HTTP\x10\x01\x12\x13\n" + + "\x0fURISCHEME_HTTPS\x10\x022l\n" + + "\x12SparkSubmitService\x12V\n" + + "\x11RunAltSparkSubmit\x12\x1f.spark.RunAltSparkSubmitRequest\x1a .spark.RunAltSparkSubmitResponseB\x1aZ\x18nativesubmit/proto/sparkb\x06proto3" + +var ( + file_proto_spark_submit_proto_rawDescOnce sync.Once + file_proto_spark_submit_proto_rawDescData []byte +) + +func file_proto_spark_submit_proto_rawDescGZIP() []byte { + file_proto_spark_submit_proto_rawDescOnce.Do(func() { + file_proto_spark_submit_proto_rawDescData = protoimpl.X.CompressGZIP(unsafe.Slice(unsafe.StringData(file_proto_spark_submit_proto_rawDesc), len(file_proto_spark_submit_proto_rawDesc))) + }) + return file_proto_spark_submit_proto_rawDescData +} + +var file_proto_spark_submit_proto_enumTypes = make([]protoimpl.EnumInfo, 23) +var file_proto_spark_submit_proto_msgTypes = make([]protoimpl.MessageInfo, 113) +var file_proto_spark_submit_proto_goTypes = []any{ + (ManagedFieldsOperationType)(0), // 0: spark.ManagedFieldsOperationType + (SparkApplicationType)(0), // 1: spark.SparkApplicationType + (DeployMode)(0), // 2: spark.DeployMode + (ServiceType)(0), // 3: spark.ServiceType + (DNSPolicy)(0), // 4: spark.DNSPolicy + (PodConditionType)(0), // 5: spark.PodConditionType + (UnsatisfiableConstraintAction)(0), // 6: spark.UnsatisfiableConstraintAction + (NodeInclusionPolicy)(0), // 7: spark.NodeInclusionPolicy + (SecretType)(0), // 8: spark.SecretType + (LabelSelectorOperator)(0), // 9: spark.LabelSelectorOperator + (NodeSelectorOperator)(0), // 10: spark.NodeSelectorOperator + (TaintEffect)(0), // 11: spark.TaintEffect + (TolerationOperator)(0), // 12: spark.TolerationOperator + (PodFSGroupChangePolicy)(0), // 13: spark.PodFSGroupChangePolicy + (Protocol)(0), // 14: spark.Protocol + (Format)(0), // 15: spark.Format + (ResourceResizeRestartPolicy)(0), // 16: spark.ResourceResizeRestartPolicy + (ContainerRestartPolicy)(0), // 17: spark.ContainerRestartPolicy + (TerminationMessagePolicy)(0), // 18: spark.TerminationMessagePolicy + (PullPolicy)(0), // 19: spark.PullPolicy + (ProcMountType)(0), // 20: spark.ProcMountType + (SeccompProfileType)(0), // 21: spark.SeccompProfileType + (URIScheme)(0), // 22: spark.URIScheme + (*SparkApplicationSpec)(nil), // 23: spark.SparkApplicationSpec + (*ObjectMeta)(nil), // 24: spark.ObjectMeta + (*FieldsV1)(nil), // 25: spark.FieldsV1 + (*ManagedFieldsEntry)(nil), // 26: spark.ManagedFieldsEntry + (*OwnerReference)(nil), // 27: spark.OwnerReference + (*DriverIngressConfiguration)(nil), // 28: spark.DriverIngressConfiguration + (*SparkUIConfiguration)(nil), // 29: spark.SparkUIConfiguration + (*IngressTLS)(nil), // 30: spark.IngressTLS + (*BatchSchedulerConfiguration)(nil), // 31: spark.BatchSchedulerConfiguration + (*MonitoringSpec)(nil), // 32: spark.MonitoringSpec + (*PrometheusSpec)(nil), // 33: spark.PrometheusSpec + (*RestartPolicy)(nil), // 34: spark.RestartPolicy + (*DriverSpec)(nil), // 35: spark.DriverSpec + (*SparkPodSpec)(nil), // 36: spark.SparkPodSpec + (*PodTemplateSpec)(nil), // 37: spark.PodTemplateSpec + (*PodSpec)(nil), // 38: spark.PodSpec + (*EphemeralContainer)(nil), // 39: spark.EphemeralContainer + (*EphemeralContainerCommon)(nil), // 40: spark.EphemeralContainerCommon + (*PodReadinessGate)(nil), // 41: spark.PodReadinessGate + (*TopologySpreadConstraint)(nil), // 42: spark.TopologySpreadConstraint + (*PodSchedulingGate)(nil), // 43: spark.PodSchedulingGate + (*PodOS)(nil), // 44: spark.PodOS + (*PodResourceClaim)(nil), // 45: spark.PodResourceClaim + (*ClaimSource)(nil), // 46: spark.ClaimSource + (*GPUSpec)(nil), // 47: spark.GPUSpec + (*NamePath)(nil), // 48: spark.NamePath + (*SecretInfo)(nil), // 49: spark.SecretInfo + (*Affinity)(nil), // 50: spark.Affinity + (*PodAntiAffinity)(nil), // 51: spark.PodAntiAffinity + (*PodAffinity)(nil), // 52: spark.PodAffinity + (*WeightedPodAffinityTerm)(nil), // 53: spark.WeightedPodAffinityTerm + (*PodAffinityTerm)(nil), // 54: spark.PodAffinityTerm + (*LabelSelector)(nil), // 55: spark.LabelSelector + (*LabelSelectorRequirement)(nil), // 56: spark.LabelSelectorRequirement + (*NodeAffinity)(nil), // 57: spark.NodeAffinity + (*PreferredSchedulingTerm)(nil), // 58: spark.PreferredSchedulingTerm + (*NodeSelector)(nil), // 59: spark.NodeSelector + (*NodeSelectorTerm)(nil), // 60: spark.NodeSelectorTerm + (*NodeSelectorRequirement)(nil), // 61: spark.NodeSelectorRequirement + (*Toleration)(nil), // 62: spark.Toleration + (*PodSecurityContext)(nil), // 63: spark.PodSecurityContext + (*Sysctl)(nil), // 64: spark.Sysctl + (*Container)(nil), // 65: spark.Container + (*ContainerPort)(nil), // 66: spark.ContainerPort + (*ConfigMapEnvSource)(nil), // 67: spark.ConfigMapEnvSource + (*EnvFromSource)(nil), // 68: spark.EnvFromSource + (*SecretEnvSource)(nil), // 69: spark.SecretEnvSource + (*EnvVar)(nil), // 70: spark.EnvVar + (*EnvVarSource)(nil), // 71: spark.EnvVarSource + (*SecretKeySelector)(nil), // 72: spark.SecretKeySelector + (*ConfigMapKeySelector)(nil), // 73: spark.ConfigMapKeySelector + (*LocalObjectReference)(nil), // 74: spark.LocalObjectReference + (*ResourceFieldSelector)(nil), // 75: spark.ResourceFieldSelector + (*ObjectFieldSelector)(nil), // 76: spark.ObjectFieldSelector + (*ResourceRequirements)(nil), // 77: spark.ResourceRequirements + (*ResourceClaim)(nil), // 78: spark.ResourceClaim + (*ResourceListEntry)(nil), // 79: spark.ResourceListEntry + (*Quantity)(nil), // 80: spark.Quantity + (*InfDecAmount)(nil), // 81: spark.InfDecAmount + (*Int64Amount)(nil), // 82: spark.Int64Amount + (*Scale)(nil), // 83: spark.Scale + (*ContainerResizePolicy)(nil), // 84: spark.ContainerResizePolicy + (*VolumeDevice)(nil), // 85: spark.VolumeDevice + (*ProbeHandler)(nil), // 86: spark.ProbeHandler + (*Probe)(nil), // 87: spark.Probe + (*SecurityContext)(nil), // 88: spark.SecurityContext + (*Capabilities)(nil), // 89: spark.Capabilities + (*SELinuxOptions)(nil), // 90: spark.SELinuxOptions + (*WindowsSecurityContextOptions)(nil), // 91: spark.WindowsSecurityContextOptions + (*SeccompProfile)(nil), // 92: spark.SeccompProfile + (*PodDNSConfig)(nil), // 93: spark.PodDNSConfig + (*PodDNSConfigOption)(nil), // 94: spark.PodDNSConfigOption + (*HostAlias)(nil), // 95: spark.HostAlias + (*Lifecycle)(nil), // 96: spark.Lifecycle + (*LifecycleHandler)(nil), // 97: spark.LifecycleHandler + (*SleepAction)(nil), // 98: spark.SleepAction + (*TCPSocketAction)(nil), // 99: spark.TCPSocketAction + (*ExecAction)(nil), // 100: spark.ExecAction + (*HTTPGetAction)(nil), // 101: spark.HTTPGetAction + (*HTTPHeader)(nil), // 102: spark.HTTPHeader + (*IntOrString)(nil), // 103: spark.IntOrString + (*Ports)(nil), // 104: spark.Ports + (*ExecutorSpec)(nil), // 105: spark.ExecutorSpec + (*Volume)(nil), // 106: spark.Volume + (*VolumeMount)(nil), // 107: spark.VolumeMount + (*SparkApplicationStatus)(nil), // 108: spark.SparkApplicationStatus + (*SparkApplication)(nil), // 109: spark.SparkApplication + (*RunAltSparkSubmitRequest)(nil), // 110: spark.RunAltSparkSubmitRequest + (*RunAltSparkSubmitResponse)(nil), // 111: spark.RunAltSparkSubmitResponse + (*Dependencies)(nil), // 112: spark.Dependencies + (*DynamicAllocation)(nil), // 113: spark.DynamicAllocation + nil, // 114: spark.SparkApplicationSpec.SparkConfEntry + nil, // 115: spark.SparkApplicationSpec.HadoopConfEntry + nil, // 116: spark.ObjectMeta.LabelsEntry + nil, // 117: spark.ObjectMeta.AnnotationsEntry + nil, // 118: spark.DriverIngressConfiguration.ServiceAnnotationsEntry + nil, // 119: spark.DriverIngressConfiguration.ServiceLabelsEntry + nil, // 120: spark.DriverIngressConfiguration.IngressAnnotationsEntry + nil, // 121: spark.SparkUIConfiguration.ServiceAnnotationsEntry + nil, // 122: spark.SparkUIConfiguration.ServiceLabelsEntry + nil, // 123: spark.SparkUIConfiguration.IngressAnnotationsEntry + nil, // 124: spark.BatchSchedulerConfiguration.ResourcesEntry + nil, // 125: spark.DriverSpec.ServiceAnnotationsEntry + nil, // 126: spark.DriverSpec.ServiceLabelsEntry + nil, // 127: spark.SparkPodSpec.EnvVarsEntry + nil, // 128: spark.SparkPodSpec.LabelsEntry + nil, // 129: spark.SparkPodSpec.AnnotationsEntry + nil, // 130: spark.SparkPodSpec.NodeSelectorEntry + nil, // 131: spark.PodSpec.NodeSelectorEntry + nil, // 132: spark.PodSpec.OverheadEntry + nil, // 133: spark.LabelSelector.MatchLabelsEntry + nil, // 134: spark.ResourceRequirements.LimitsEntry + nil, // 135: spark.ResourceRequirements.RequestsEntry + (*wrapperspb.StringValue)(nil), // 136: google.protobuf.StringValue + (*wrapperspb.Int32Value)(nil), // 137: google.protobuf.Int32Value + (*wrapperspb.Int64Value)(nil), // 138: google.protobuf.Int64Value + (*timestamppb.Timestamp)(nil), // 139: google.protobuf.Timestamp + (*wrapperspb.BoolValue)(nil), // 140: google.protobuf.BoolValue +} +var file_proto_spark_submit_proto_depIdxs = []int32{ + 1, // 0: spark.SparkApplicationSpec.type:type_name -> spark.SparkApplicationType + 2, // 1: spark.SparkApplicationSpec.mode:type_name -> spark.DeployMode + 136, // 2: spark.SparkApplicationSpec.image:type_name -> google.protobuf.StringValue + 136, // 3: spark.SparkApplicationSpec.image_pull_policy:type_name -> google.protobuf.StringValue + 114, // 4: spark.SparkApplicationSpec.spark_conf:type_name -> spark.SparkApplicationSpec.SparkConfEntry + 115, // 5: spark.SparkApplicationSpec.hadoop_conf:type_name -> spark.SparkApplicationSpec.HadoopConfEntry + 136, // 6: spark.SparkApplicationSpec.spark_config_map:type_name -> google.protobuf.StringValue + 136, // 7: spark.SparkApplicationSpec.hadoop_config_map:type_name -> google.protobuf.StringValue + 136, // 8: spark.SparkApplicationSpec.main_class:type_name -> google.protobuf.StringValue + 136, // 9: spark.SparkApplicationSpec.main_application_file:type_name -> google.protobuf.StringValue + 136, // 10: spark.SparkApplicationSpec.proxy_user:type_name -> google.protobuf.StringValue + 137, // 11: spark.SparkApplicationSpec.failure_retries:type_name -> google.protobuf.Int32Value + 138, // 12: spark.SparkApplicationSpec.retry_interval:type_name -> google.protobuf.Int64Value + 136, // 13: spark.SparkApplicationSpec.memory_overhead_factor:type_name -> google.protobuf.StringValue + 32, // 14: spark.SparkApplicationSpec.monitoring:type_name -> spark.MonitoringSpec + 136, // 15: spark.SparkApplicationSpec.batch_scheduler:type_name -> google.protobuf.StringValue + 138, // 16: spark.SparkApplicationSpec.time_to_live_seconds:type_name -> google.protobuf.Int64Value + 31, // 17: spark.SparkApplicationSpec.batch_scheduler_configuration:type_name -> spark.BatchSchedulerConfiguration + 35, // 18: spark.SparkApplicationSpec.driver:type_name -> spark.DriverSpec + 105, // 19: spark.SparkApplicationSpec.executor:type_name -> spark.ExecutorSpec + 106, // 20: spark.SparkApplicationSpec.volumes:type_name -> spark.Volume + 112, // 21: spark.SparkApplicationSpec.deps:type_name -> spark.Dependencies + 113, // 22: spark.SparkApplicationSpec.dynamic_allocation:type_name -> spark.DynamicAllocation + 34, // 23: spark.SparkApplicationSpec.restart_policy:type_name -> spark.RestartPolicy + 29, // 24: spark.SparkApplicationSpec.spark_ui_configuration:type_name -> spark.SparkUIConfiguration + 28, // 25: spark.SparkApplicationSpec.driver_ingress_configuration:type_name -> spark.DriverIngressConfiguration + 139, // 26: spark.ObjectMeta.creation_timestamp:type_name -> google.protobuf.Timestamp + 139, // 27: spark.ObjectMeta.deletion_timestamp:type_name -> google.protobuf.Timestamp + 138, // 28: spark.ObjectMeta.deletion_grace_period_seconds:type_name -> google.protobuf.Int64Value + 116, // 29: spark.ObjectMeta.labels:type_name -> spark.ObjectMeta.LabelsEntry + 117, // 30: spark.ObjectMeta.annotations:type_name -> spark.ObjectMeta.AnnotationsEntry + 27, // 31: spark.ObjectMeta.owner_references:type_name -> spark.OwnerReference + 26, // 32: spark.ObjectMeta.managed_fields:type_name -> spark.ManagedFieldsEntry + 0, // 33: spark.ManagedFieldsEntry.operation:type_name -> spark.ManagedFieldsOperationType + 139, // 34: spark.ManagedFieldsEntry.my_time:type_name -> google.protobuf.Timestamp + 25, // 35: spark.ManagedFieldsEntry.fields_v1:type_name -> spark.FieldsV1 + 140, // 36: spark.OwnerReference.controller:type_name -> google.protobuf.BoolValue + 140, // 37: spark.OwnerReference.block_owner_deletion:type_name -> google.protobuf.BoolValue + 137, // 38: spark.DriverIngressConfiguration.service_port:type_name -> google.protobuf.Int32Value + 136, // 39: spark.DriverIngressConfiguration.service_port_name:type_name -> google.protobuf.StringValue + 3, // 40: spark.DriverIngressConfiguration.service_type:type_name -> spark.ServiceType + 118, // 41: spark.DriverIngressConfiguration.service_annotations:type_name -> spark.DriverIngressConfiguration.ServiceAnnotationsEntry + 119, // 42: spark.DriverIngressConfiguration.service_labels:type_name -> spark.DriverIngressConfiguration.ServiceLabelsEntry + 120, // 43: spark.DriverIngressConfiguration.ingress_annotations:type_name -> spark.DriverIngressConfiguration.IngressAnnotationsEntry + 30, // 44: spark.DriverIngressConfiguration.ingress_tls:type_name -> spark.IngressTLS + 137, // 45: spark.SparkUIConfiguration.service_port:type_name -> google.protobuf.Int32Value + 136, // 46: spark.SparkUIConfiguration.service_port_name:type_name -> google.protobuf.StringValue + 3, // 47: spark.SparkUIConfiguration.service_type:type_name -> spark.ServiceType + 121, // 48: spark.SparkUIConfiguration.service_annotations:type_name -> spark.SparkUIConfiguration.ServiceAnnotationsEntry + 122, // 49: spark.SparkUIConfiguration.service_labels:type_name -> spark.SparkUIConfiguration.ServiceLabelsEntry + 123, // 50: spark.SparkUIConfiguration.ingress_annotations:type_name -> spark.SparkUIConfiguration.IngressAnnotationsEntry + 30, // 51: spark.SparkUIConfiguration.ingress_tls:type_name -> spark.IngressTLS + 136, // 52: spark.BatchSchedulerConfiguration.queue:type_name -> google.protobuf.StringValue + 136, // 53: spark.BatchSchedulerConfiguration.priority_class_name:type_name -> google.protobuf.StringValue + 124, // 54: spark.BatchSchedulerConfiguration.resources:type_name -> spark.BatchSchedulerConfiguration.ResourcesEntry + 136, // 55: spark.MonitoringSpec.metrics_properties:type_name -> google.protobuf.StringValue + 136, // 56: spark.MonitoringSpec.metrics_properties_file:type_name -> google.protobuf.StringValue + 33, // 57: spark.MonitoringSpec.prometheus:type_name -> spark.PrometheusSpec + 137, // 58: spark.PrometheusSpec.port:type_name -> google.protobuf.Int32Value + 136, // 59: spark.PrometheusSpec.port_name:type_name -> google.protobuf.StringValue + 136, // 60: spark.PrometheusSpec.config_file:type_name -> google.protobuf.StringValue + 136, // 61: spark.PrometheusSpec.configuration:type_name -> google.protobuf.StringValue + 36, // 62: spark.DriverSpec.spark_pod_spec:type_name -> spark.SparkPodSpec + 136, // 63: spark.DriverSpec.pod_name:type_name -> google.protobuf.StringValue + 136, // 64: spark.DriverSpec.core_request:type_name -> google.protobuf.StringValue + 136, // 65: spark.DriverSpec.java_options:type_name -> google.protobuf.StringValue + 96, // 66: spark.DriverSpec.life_cycle:type_name -> spark.Lifecycle + 136, // 67: spark.DriverSpec.kubernetes_master:type_name -> google.protobuf.StringValue + 125, // 68: spark.DriverSpec.service_annotations:type_name -> spark.DriverSpec.ServiceAnnotationsEntry + 126, // 69: spark.DriverSpec.service_labels:type_name -> spark.DriverSpec.ServiceLabelsEntry + 104, // 70: spark.DriverSpec.ports:type_name -> spark.Ports + 136, // 71: spark.DriverSpec.priority_class_name:type_name -> google.protobuf.StringValue + 37, // 72: spark.SparkPodSpec.template:type_name -> spark.PodTemplateSpec + 137, // 73: spark.SparkPodSpec.cores:type_name -> google.protobuf.Int32Value + 47, // 74: spark.SparkPodSpec.gpu:type_name -> spark.GPUSpec + 48, // 75: spark.SparkPodSpec.configmaps:type_name -> spark.NamePath + 49, // 76: spark.SparkPodSpec.secrets:type_name -> spark.SecretInfo + 70, // 77: spark.SparkPodSpec.env:type_name -> spark.EnvVar + 127, // 78: spark.SparkPodSpec.env_vars:type_name -> spark.SparkPodSpec.EnvVarsEntry + 68, // 79: spark.SparkPodSpec.env_from:type_name -> spark.EnvFromSource + 128, // 80: spark.SparkPodSpec.labels:type_name -> spark.SparkPodSpec.LabelsEntry + 129, // 81: spark.SparkPodSpec.annotations:type_name -> spark.SparkPodSpec.AnnotationsEntry + 107, // 82: spark.SparkPodSpec.volume_mounts:type_name -> spark.VolumeMount + 50, // 83: spark.SparkPodSpec.affinity:type_name -> spark.Affinity + 62, // 84: spark.SparkPodSpec.tolerations:type_name -> spark.Toleration + 63, // 85: spark.SparkPodSpec.pod_security_context:type_name -> spark.PodSecurityContext + 88, // 86: spark.SparkPodSpec.security_context:type_name -> spark.SecurityContext + 136, // 87: spark.SparkPodSpec.scheduler_name:type_name -> google.protobuf.StringValue + 65, // 88: spark.SparkPodSpec.sidecars:type_name -> spark.Container + 65, // 89: spark.SparkPodSpec.init_containers:type_name -> spark.Container + 140, // 90: spark.SparkPodSpec.host_network:type_name -> google.protobuf.BoolValue + 130, // 91: spark.SparkPodSpec.node_selector:type_name -> spark.SparkPodSpec.NodeSelectorEntry + 93, // 92: spark.SparkPodSpec.dns_config:type_name -> spark.PodDNSConfig + 136, // 93: spark.SparkPodSpec.service_account:type_name -> google.protobuf.StringValue + 95, // 94: spark.SparkPodSpec.host_aliases:type_name -> spark.HostAlias + 140, // 95: spark.SparkPodSpec.share_process_namespace:type_name -> google.protobuf.BoolValue + 24, // 96: spark.PodTemplateSpec.object_meta:type_name -> spark.ObjectMeta + 38, // 97: spark.PodTemplateSpec.pod_spec:type_name -> spark.PodSpec + 106, // 98: spark.PodSpec.volumes:type_name -> spark.Volume + 65, // 99: spark.PodSpec.containers:type_name -> spark.Container + 39, // 100: spark.PodSpec.ephemeral_containers:type_name -> spark.EphemeralContainer + 34, // 101: spark.PodSpec.restart_policy:type_name -> spark.RestartPolicy + 138, // 102: spark.PodSpec.termination_grace_period_seconds:type_name -> google.protobuf.Int64Value + 138, // 103: spark.PodSpec.active_deadline_seconds:type_name -> google.protobuf.Int64Value + 4, // 104: spark.PodSpec.dns_policy:type_name -> spark.DNSPolicy + 131, // 105: spark.PodSpec.node_selector:type_name -> spark.PodSpec.NodeSelectorEntry + 140, // 106: spark.PodSpec.auto_mount_service_account_token:type_name -> google.protobuf.BoolValue + 140, // 107: spark.PodSpec.share_process_name:type_name -> google.protobuf.BoolValue + 63, // 108: spark.PodSpec.security_context:type_name -> spark.PodSecurityContext + 74, // 109: spark.PodSpec.image_pull_secrets:type_name -> spark.LocalObjectReference + 50, // 110: spark.PodSpec.affinity:type_name -> spark.Affinity + 62, // 111: spark.PodSpec.tolerations:type_name -> spark.Toleration + 95, // 112: spark.PodSpec.host_aliases:type_name -> spark.HostAlias + 137, // 113: spark.PodSpec.priority:type_name -> google.protobuf.Int32Value + 93, // 114: spark.PodSpec.dns_config:type_name -> spark.PodDNSConfig + 41, // 115: spark.PodSpec.readiness_gates:type_name -> spark.PodReadinessGate + 136, // 116: spark.PodSpec.runtime_class_name:type_name -> google.protobuf.StringValue + 140, // 117: spark.PodSpec.enable_service_links:type_name -> google.protobuf.BoolValue + 132, // 118: spark.PodSpec.overhead:type_name -> spark.PodSpec.OverheadEntry + 42, // 119: spark.PodSpec.topology_spread_constraints:type_name -> spark.TopologySpreadConstraint + 140, // 120: spark.PodSpec.set_host_name_as_fqdn:type_name -> google.protobuf.BoolValue + 44, // 121: spark.PodSpec.os:type_name -> spark.PodOS + 140, // 122: spark.PodSpec.host_users:type_name -> google.protobuf.BoolValue + 43, // 123: spark.PodSpec.scheduling_gates:type_name -> spark.PodSchedulingGate + 45, // 124: spark.PodSpec.resource_claims:type_name -> spark.PodResourceClaim + 40, // 125: spark.EphemeralContainer.ephemeral_container_common:type_name -> spark.EphemeralContainerCommon + 66, // 126: spark.EphemeralContainerCommon.ports:type_name -> spark.ContainerPort + 68, // 127: spark.EphemeralContainerCommon.env_from:type_name -> spark.EnvFromSource + 70, // 128: spark.EphemeralContainerCommon.env:type_name -> spark.EnvVar + 77, // 129: spark.EphemeralContainerCommon.resources:type_name -> spark.ResourceRequirements + 84, // 130: spark.EphemeralContainerCommon.resize_policy:type_name -> spark.ContainerResizePolicy + 17, // 131: spark.EphemeralContainerCommon.restart_policy:type_name -> spark.ContainerRestartPolicy + 107, // 132: spark.EphemeralContainerCommon.volume_mounts:type_name -> spark.VolumeMount + 85, // 133: spark.EphemeralContainerCommon.volume_devices:type_name -> spark.VolumeDevice + 87, // 134: spark.EphemeralContainerCommon.readiness_probe:type_name -> spark.Probe + 96, // 135: spark.EphemeralContainerCommon.life_cycle:type_name -> spark.Lifecycle + 18, // 136: spark.EphemeralContainerCommon.termination_message_policy:type_name -> spark.TerminationMessagePolicy + 19, // 137: spark.EphemeralContainerCommon.image_pull_policy:type_name -> spark.PullPolicy + 88, // 138: spark.EphemeralContainerCommon.security_context:type_name -> spark.SecurityContext + 5, // 139: spark.PodReadinessGate.condition_type:type_name -> spark.PodConditionType + 6, // 140: spark.TopologySpreadConstraint.when_unsatisfiable:type_name -> spark.UnsatisfiableConstraintAction + 55, // 141: spark.TopologySpreadConstraint.label_selector:type_name -> spark.LabelSelector + 137, // 142: spark.TopologySpreadConstraint.min_domains:type_name -> google.protobuf.Int32Value + 7, // 143: spark.TopologySpreadConstraint.node_affinity_policy:type_name -> spark.NodeInclusionPolicy + 7, // 144: spark.TopologySpreadConstraint.node_taints_policy:type_name -> spark.NodeInclusionPolicy + 46, // 145: spark.PodResourceClaim.source:type_name -> spark.ClaimSource + 136, // 146: spark.ClaimSource.resource_claim_name:type_name -> google.protobuf.StringValue + 136, // 147: spark.ClaimSource.resource_claim_template_name:type_name -> google.protobuf.StringValue + 8, // 148: spark.SecretInfo.type:type_name -> spark.SecretType + 57, // 149: spark.Affinity.node_affinity:type_name -> spark.NodeAffinity + 52, // 150: spark.Affinity.pod_affinity:type_name -> spark.PodAffinity + 51, // 151: spark.Affinity.pod_anti_affinity:type_name -> spark.PodAntiAffinity + 55, // 152: spark.PodAntiAffinity.label_selector:type_name -> spark.LabelSelector + 55, // 153: spark.PodAntiAffinity.namespace_selector:type_name -> spark.LabelSelector + 54, // 154: spark.PodAffinity.required_during_scheduling_ignored_during_execution:type_name -> spark.PodAffinityTerm + 53, // 155: spark.PodAffinity.preferred_during_scheduling_ignored_during_execution:type_name -> spark.WeightedPodAffinityTerm + 54, // 156: spark.WeightedPodAffinityTerm.pod_affinity_term:type_name -> spark.PodAffinityTerm + 55, // 157: spark.PodAffinityTerm.label_selector:type_name -> spark.LabelSelector + 55, // 158: spark.PodAffinityTerm.namespace_selector:type_name -> spark.LabelSelector + 133, // 159: spark.LabelSelector.match_labels:type_name -> spark.LabelSelector.MatchLabelsEntry + 56, // 160: spark.LabelSelector.match_expressions:type_name -> spark.LabelSelectorRequirement + 9, // 161: spark.LabelSelectorRequirement.operator:type_name -> spark.LabelSelectorOperator + 59, // 162: spark.NodeAffinity.required_during_scheduling_ignored_during_execution:type_name -> spark.NodeSelector + 58, // 163: spark.NodeAffinity.preferred_during_scheduling_ignored_during_execution:type_name -> spark.PreferredSchedulingTerm + 60, // 164: spark.PreferredSchedulingTerm.preference:type_name -> spark.NodeSelectorTerm + 60, // 165: spark.NodeSelector.node_selector_terms:type_name -> spark.NodeSelectorTerm + 61, // 166: spark.NodeSelectorTerm.match_expressions:type_name -> spark.NodeSelectorRequirement + 61, // 167: spark.NodeSelectorTerm.match_fields:type_name -> spark.NodeSelectorRequirement + 10, // 168: spark.NodeSelectorRequirement.operator:type_name -> spark.NodeSelectorOperator + 12, // 169: spark.Toleration.operator:type_name -> spark.TolerationOperator + 11, // 170: spark.Toleration.effect:type_name -> spark.TaintEffect + 138, // 171: spark.Toleration.toleration_seconds:type_name -> google.protobuf.Int64Value + 90, // 172: spark.PodSecurityContext.se_linux_options:type_name -> spark.SELinuxOptions + 91, // 173: spark.PodSecurityContext.windows_security_context_options:type_name -> spark.WindowsSecurityContextOptions + 138, // 174: spark.PodSecurityContext.run_as_user:type_name -> google.protobuf.Int64Value + 138, // 175: spark.PodSecurityContext.run_as_group:type_name -> google.protobuf.Int64Value + 140, // 176: spark.PodSecurityContext.run_as_nonroot:type_name -> google.protobuf.BoolValue + 138, // 177: spark.PodSecurityContext.fs_group:type_name -> google.protobuf.Int64Value + 64, // 178: spark.PodSecurityContext.sys_ctl:type_name -> spark.Sysctl + 13, // 179: spark.PodSecurityContext.fs_group_change_policy:type_name -> spark.PodFSGroupChangePolicy + 92, // 180: spark.PodSecurityContext.sec_comp_profile:type_name -> spark.SeccompProfile + 66, // 181: spark.Container.ports:type_name -> spark.ContainerPort + 68, // 182: spark.Container.env_from:type_name -> spark.EnvFromSource + 70, // 183: spark.Container.env:type_name -> spark.EnvVar + 77, // 184: spark.Container.resources:type_name -> spark.ResourceRequirements + 84, // 185: spark.Container.resize_policy:type_name -> spark.ContainerResizePolicy + 17, // 186: spark.Container.restart_policy:type_name -> spark.ContainerRestartPolicy + 107, // 187: spark.Container.volume_mounts:type_name -> spark.VolumeMount + 85, // 188: spark.Container.volume_devices:type_name -> spark.VolumeDevice + 87, // 189: spark.Container.liveness_probe:type_name -> spark.Probe + 87, // 190: spark.Container.readiness_probe:type_name -> spark.Probe + 87, // 191: spark.Container.startup_probe:type_name -> spark.Probe + 96, // 192: spark.Container.life_cycle:type_name -> spark.Lifecycle + 18, // 193: spark.Container.termination_message_policy:type_name -> spark.TerminationMessagePolicy + 19, // 194: spark.Container.image_pull_policy:type_name -> spark.PullPolicy + 88, // 195: spark.Container.security_context:type_name -> spark.SecurityContext + 14, // 196: spark.ContainerPort.protocol:type_name -> spark.Protocol + 74, // 197: spark.ConfigMapEnvSource.local_object_reference:type_name -> spark.LocalObjectReference + 140, // 198: spark.ConfigMapEnvSource.optional:type_name -> google.protobuf.BoolValue + 67, // 199: spark.EnvFromSource.config_map_ref:type_name -> spark.ConfigMapEnvSource + 69, // 200: spark.EnvFromSource.secret_ref:type_name -> spark.SecretEnvSource + 74, // 201: spark.SecretEnvSource.local_object_reference:type_name -> spark.LocalObjectReference + 140, // 202: spark.SecretEnvSource.optional:type_name -> google.protobuf.BoolValue + 71, // 203: spark.EnvVar.value_from:type_name -> spark.EnvVarSource + 76, // 204: spark.EnvVarSource.field_ref:type_name -> spark.ObjectFieldSelector + 75, // 205: spark.EnvVarSource.resource_field_ref:type_name -> spark.ResourceFieldSelector + 73, // 206: spark.EnvVarSource.config_map_key_ref:type_name -> spark.ConfigMapKeySelector + 72, // 207: spark.EnvVarSource.secret_key_ref:type_name -> spark.SecretKeySelector + 74, // 208: spark.SecretKeySelector.local_object_reference:type_name -> spark.LocalObjectReference + 140, // 209: spark.SecretKeySelector.optional:type_name -> google.protobuf.BoolValue + 74, // 210: spark.ConfigMapKeySelector.local_object_reference:type_name -> spark.LocalObjectReference + 140, // 211: spark.ConfigMapKeySelector.optional:type_name -> google.protobuf.BoolValue + 80, // 212: spark.ResourceFieldSelector.divisor:type_name -> spark.Quantity + 134, // 213: spark.ResourceRequirements.limits:type_name -> spark.ResourceRequirements.LimitsEntry + 135, // 214: spark.ResourceRequirements.requests:type_name -> spark.ResourceRequirements.RequestsEntry + 78, // 215: spark.ResourceRequirements.claims:type_name -> spark.ResourceClaim + 80, // 216: spark.ResourceListEntry.quantity:type_name -> spark.Quantity + 82, // 217: spark.Quantity.i:type_name -> spark.Int64Amount + 81, // 218: spark.Quantity.d:type_name -> spark.InfDecAmount + 15, // 219: spark.Quantity.format:type_name -> spark.Format + 83, // 220: spark.Int64Amount.scale:type_name -> spark.Scale + 16, // 221: spark.ContainerResizePolicy.restart_policy:type_name -> spark.ResourceResizeRestartPolicy + 100, // 222: spark.ProbeHandler.exec:type_name -> spark.ExecAction + 101, // 223: spark.ProbeHandler.http_get:type_name -> spark.HTTPGetAction + 99, // 224: spark.ProbeHandler.tcp_socket:type_name -> spark.TCPSocketAction + 86, // 225: spark.Probe.probe_handler:type_name -> spark.ProbeHandler + 138, // 226: spark.Probe.termination_grace_period_seconds:type_name -> google.protobuf.Int64Value + 89, // 227: spark.SecurityContext.capabilities:type_name -> spark.Capabilities + 140, // 228: spark.SecurityContext.privileged:type_name -> google.protobuf.BoolValue + 90, // 229: spark.SecurityContext.se_linux_options:type_name -> spark.SELinuxOptions + 91, // 230: spark.SecurityContext.windows_security_context_options:type_name -> spark.WindowsSecurityContextOptions + 138, // 231: spark.SecurityContext.run_as_user:type_name -> google.protobuf.Int64Value + 138, // 232: spark.SecurityContext.run_as_group:type_name -> google.protobuf.Int64Value + 140, // 233: spark.SecurityContext.run_as_non_root:type_name -> google.protobuf.BoolValue + 140, // 234: spark.SecurityContext.read_only_file_system:type_name -> google.protobuf.BoolValue + 140, // 235: spark.SecurityContext.allow_privilege_escalation:type_name -> google.protobuf.BoolValue + 20, // 236: spark.SecurityContext.proc_mount:type_name -> spark.ProcMountType + 92, // 237: spark.SecurityContext.sec_comp_profile:type_name -> spark.SeccompProfile + 136, // 238: spark.WindowsSecurityContextOptions.gmsa_credential_spec_name:type_name -> google.protobuf.StringValue + 136, // 239: spark.WindowsSecurityContextOptions.gmsa_credential_spec:type_name -> google.protobuf.StringValue + 136, // 240: spark.WindowsSecurityContextOptions.run_as_user_name:type_name -> google.protobuf.StringValue + 140, // 241: spark.WindowsSecurityContextOptions.host_process:type_name -> google.protobuf.BoolValue + 21, // 242: spark.SeccompProfile.type:type_name -> spark.SeccompProfileType + 136, // 243: spark.SeccompProfile.local_host_profile:type_name -> google.protobuf.StringValue + 94, // 244: spark.PodDNSConfig.options:type_name -> spark.PodDNSConfigOption + 97, // 245: spark.Lifecycle.post_start:type_name -> spark.LifecycleHandler + 97, // 246: spark.Lifecycle.pre_stop:type_name -> spark.LifecycleHandler + 100, // 247: spark.LifecycleHandler.exec:type_name -> spark.ExecAction + 101, // 248: spark.LifecycleHandler.http_get:type_name -> spark.HTTPGetAction + 99, // 249: spark.LifecycleHandler.tcp_socket:type_name -> spark.TCPSocketAction + 98, // 250: spark.LifecycleHandler.sleep:type_name -> spark.SleepAction + 103, // 251: spark.TCPSocketAction.port:type_name -> spark.IntOrString + 103, // 252: spark.HTTPGetAction.port:type_name -> spark.IntOrString + 22, // 253: spark.HTTPGetAction.scheme:type_name -> spark.URIScheme + 102, // 254: spark.HTTPGetAction.http_headers:type_name -> spark.HTTPHeader + 36, // 255: spark.ExecutorSpec.spark_pod_spec:type_name -> spark.SparkPodSpec + 137, // 256: spark.ExecutorSpec.instances:type_name -> google.protobuf.Int32Value + 136, // 257: spark.ExecutorSpec.core_request:type_name -> google.protobuf.StringValue + 136, // 258: spark.ExecutorSpec.java_options:type_name -> google.protobuf.StringValue + 96, // 259: spark.ExecutorSpec.life_cycle:type_name -> spark.Lifecycle + 140, // 260: spark.ExecutorSpec.delete_on_termination:type_name -> google.protobuf.BoolValue + 104, // 261: spark.ExecutorSpec.ports:type_name -> spark.Ports + 136, // 262: spark.ExecutorSpec.priority_class_name:type_name -> google.protobuf.StringValue + 24, // 263: spark.SparkApplication.metadata:type_name -> spark.ObjectMeta + 23, // 264: spark.SparkApplication.spec:type_name -> spark.SparkApplicationSpec + 108, // 265: spark.SparkApplication.status:type_name -> spark.SparkApplicationStatus + 109, // 266: spark.RunAltSparkSubmitRequest.spark_application:type_name -> spark.SparkApplication + 80, // 267: spark.BatchSchedulerConfiguration.ResourcesEntry.value:type_name -> spark.Quantity + 80, // 268: spark.PodSpec.OverheadEntry.value:type_name -> spark.Quantity + 80, // 269: spark.ResourceRequirements.LimitsEntry.value:type_name -> spark.Quantity + 80, // 270: spark.ResourceRequirements.RequestsEntry.value:type_name -> spark.Quantity + 110, // 271: spark.SparkSubmitService.RunAltSparkSubmit:input_type -> spark.RunAltSparkSubmitRequest + 111, // 272: spark.SparkSubmitService.RunAltSparkSubmit:output_type -> spark.RunAltSparkSubmitResponse + 272, // [272:273] is the sub-list for method output_type + 271, // [271:272] is the sub-list for method input_type + 271, // [271:271] is the sub-list for extension type_name + 271, // [271:271] is the sub-list for extension extendee + 0, // [0:271] is the sub-list for field type_name +} + +func init() { file_proto_spark_submit_proto_init() } +func file_proto_spark_submit_proto_init() { + if File_proto_spark_submit_proto != nil { + return + } + file_proto_spark_submit_proto_msgTypes[80].OneofWrappers = []any{ + (*IntOrString_IntVal)(nil), + (*IntOrString_StrVal)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: unsafe.Slice(unsafe.StringData(file_proto_spark_submit_proto_rawDesc), len(file_proto_spark_submit_proto_rawDesc)), + NumEnums: 23, + NumMessages: 113, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_proto_spark_submit_proto_goTypes, + DependencyIndexes: file_proto_spark_submit_proto_depIdxs, + EnumInfos: file_proto_spark_submit_proto_enumTypes, + MessageInfos: file_proto_spark_submit_proto_msgTypes, + }.Build() + File_proto_spark_submit_proto = out.File + file_proto_spark_submit_proto_goTypes = nil + file_proto_spark_submit_proto_depIdxs = nil +} diff --git a/proto/spark_submit.proto b/proto/spark_submit.proto new file mode 100644 index 000000000..73f824fe6 --- /dev/null +++ b/proto/spark_submit.proto @@ -0,0 +1,898 @@ +syntax = "proto3"; + +package spark; + +import "google/protobuf/wrappers.proto"; +import "google/protobuf/timestamp.proto"; + +option go_package = "nativesubmit/proto/spark"; + + +// SparkApplicationSpec +message SparkApplicationSpec { + SparkApplicationType type = 1; + DeployMode mode = 2; + google.protobuf.StringValue image = 3; + google.protobuf.StringValue image_pull_policy = 4; + repeated string image_pull_secrets = 5; + map spark_conf = 6; + map hadoop_conf = 7; + google.protobuf.StringValue spark_config_map = 8; + google.protobuf.StringValue hadoop_config_map = 9; + repeated string arguments = 10; + google.protobuf.StringValue main_class = 11; + google.protobuf.StringValue main_application_file = 12; + google.protobuf.StringValue proxy_user = 13; + google.protobuf.Int32Value failure_retries = 15; + google.protobuf.Int64Value retry_interval = 16; + google.protobuf.StringValue memory_overhead_factor = 17; + MonitoringSpec monitoring = 18; + google.protobuf.StringValue batch_scheduler = 19; + google.protobuf.Int64Value time_to_live_seconds = 20; + BatchSchedulerConfiguration batch_scheduler_configuration = 21; + DriverSpec driver = 22; + ExecutorSpec executor = 23; + repeated Volume volumes = 24; + Dependencies deps = 25; + DynamicAllocation dynamic_allocation = 26; + string python_version = 27; + string spark_version = 28; + RestartPolicy restart_policy = 29; + SparkUIConfiguration spark_ui_configuration = 30; + repeated DriverIngressConfiguration driver_ingress_configuration = 31; + + // Add more fields as needed +} + +// Metadata for Kubernetes objects +message ObjectMeta { + string name = 1; + string generate_name = 2; + string namespace = 3; + string self_link = 4; + string uid = 5; + string resource_version = 6; + int64 generation = 7; + google.protobuf.Timestamp creation_timestamp = 8; + google.protobuf.Timestamp deletion_timestamp = 9; + google.protobuf.Int64Value deletion_grace_period_seconds = 10; + map labels = 11; + map annotations = 12; + repeated OwnerReference owner_references = 13; + repeated string finalizers = 14; + repeated ManagedFieldsEntry managed_fields = 15; +} + +enum ManagedFieldsOperationType { + MANAGED_FIELDS_OPERATION_TYPE_UNSPECIFIED = 0; + MANAGED_FIELDS_OPERATION_TYPE_APPLY = 1; + MANAGED_FIELDS_OPERATION_TYPE_UPDATE = 2; +} + +message FieldsV1 { + string raw = 1; +} + +message ManagedFieldsEntry { + string manager = 1; + ManagedFieldsOperationType operation = 2; + string api_version = 3; + google.protobuf.Timestamp my_time = 4; + string fields_type = 5; + FieldsV1 fields_v1 = 6; + string sub_resource = 7; +} + +message OwnerReference { + string api_version = 1; + string kind = 2; + string name = 3; + string uid = 4; + google.protobuf.BoolValue controller = 5; + google.protobuf.BoolValue block_owner_deletion = 6; +} + +enum SparkApplicationType { + SPARK_APPLICATION_TYPE_UNSPECIFIED = 0; + SPARK_APPLICATION_TYPE_JAVA = 1; + SPARK_APPLICATION_TYPE_SCALA = 2; + SPARK_APPLICATION_TYPE_PYTHON = 3; + SPARK_APPLICATION_TYPE_R = 4; +} +enum DeployMode { + DEPLOY_MODE_UNSPECIFIED = 0; + DEPLOY_MODE_CLUSTER = 1; + DEPLOY_MODE_CLIENT = 2; + DEPLOY_MODE_IN_CLUSTER_CLIENT = 3; +} +message DriverIngressConfiguration { + google.protobuf.Int32Value service_port = 1; + google.protobuf.StringValue service_port_name = 2; + ServiceType service_type = 3; + map service_annotations = 4; + map service_labels = 5; + string ingress_url_format = 6; + map ingress_annotations = 7; + repeated IngressTLS ingress_tls = 8; + +} +message SparkUIConfiguration { + google.protobuf.Int32Value service_port = 1; + google.protobuf.StringValue service_port_name = 2; + ServiceType service_type = 3; + map service_annotations = 4; + map service_labels = 5; + map ingress_annotations = 6; + repeated IngressTLS ingress_tls = 7; +} +message IngressTLS { + repeated string hosts = 1; + string secret_name = 2; +} +enum ServiceType { + SERVICE_TYPE_UNSPECIFIED = 0; + SERVICE_TYPE_CLUSTER_IP = 1; + SERVICE_TYPE_NODE_PORT = 2; + SERVICE_TYPE_LOAD_BALANCER = 3; + SERVICE_TYPE_EXTERNAL_NAME = 4; +} + +message BatchSchedulerConfiguration { + google.protobuf.StringValue queue = 1; + google.protobuf.StringValue priority_class_name = 2; + map resources = 3; +} +message MonitoringSpec { + bool expose_driver_metrics = 1; + bool expose_executor_metrics = 2; + google.protobuf.StringValue metrics_properties = 3; + google.protobuf.StringValue metrics_properties_file = 4; + PrometheusSpec prometheus = 5; + +} +message PrometheusSpec { + string jmx_exporter_jar = 1; + google.protobuf.Int32Value port = 2; + google.protobuf.StringValue port_name = 3; + google.protobuf.StringValue config_file = 4; + google.protobuf.StringValue configuration = 5; +} +// RestartPolicy +message RestartPolicy{ + string type = 1; +} + +// DriverSpec and ExecutorSpec +message DriverSpec { + SparkPodSpec spark_pod_spec = 1; + google.protobuf.StringValue pod_name = 2; + google.protobuf.StringValue core_request = 3; + google.protobuf.StringValue java_options = 4; + Lifecycle life_cycle = 5; + google.protobuf.StringValue kubernetes_master = 6; + map service_annotations = 7; + map service_labels = 8; + repeated Ports ports = 9; + google.protobuf.StringValue priority_class_name = 10; + + // Add more fields as needed +} + +message SparkPodSpec { + PodTemplateSpec template = 1; + google.protobuf.Int32Value cores = 2; + string core_limit = 3; + string memory = 4; + string memory_overhead = 5; + GPUSpec gpu = 6; + string image = 7; + repeated NamePath configmaps = 8; + repeated SecretInfo secrets = 9; + repeated EnvVar env = 10; + map env_vars = 11; + repeated EnvFromSource env_from = 12; + map labels = 13; + map annotations = 14; + repeated VolumeMount volume_mounts = 15; + Affinity affinity = 16; + repeated Toleration tolerations = 17; + PodSecurityContext pod_security_context = 18; + SecurityContext security_context = 19; + google.protobuf.StringValue scheduler_name = 20; + repeated Container sidecars = 21; + repeated Container init_containers = 22; + google.protobuf.BoolValue host_network = 23; + map node_selector = 24; + PodDNSConfig dns_config = 25; + int64 termination_grace_period_seconds = 26; + google.protobuf.StringValue service_account = 27; + repeated HostAlias host_aliases = 28; + google.protobuf.BoolValue share_process_namespace = 29; + +} +message PodTemplateSpec { + ObjectMeta object_meta = 1; + PodSpec pod_spec = 2; +} + +enum DNSPolicy { + DNS_POLICY_UNSPECIFIED = 0; + DNS_POLICY_CLUSTER_FIRST_WITH_HOST_NET = 1; + DNS_POLICY_CLUSTER_FIRST = 2; + DNS_POLICY_DEFAULT = 3; + DNS_POLICY_NONE = 4; +} +message PodSpec { + repeated Volume volumes = 1; + repeated Container containers = 2; + repeated EphemeralContainer ephemeral_containers = 3; + RestartPolicy restart_policy = 4; + google.protobuf.Int64Value termination_grace_period_seconds = 5; + google.protobuf.Int64Value active_deadline_seconds = 6; + DNSPolicy dns_policy = 7; + map node_selector = 8; + string service_account_name = 9; + string deprecated_service_account = 10; + google.protobuf.BoolValue auto_mount_service_account_token = 11; + string node_name = 12; + bool host_network = 13; + bool host_pid = 14; + bool host_ipc = 15; + google.protobuf.BoolValue share_process_name = 16; + PodSecurityContext security_context = 17; + repeated LocalObjectReference image_pull_secrets = 18; + string host_name = 19; + string sub_domain = 20; + Affinity affinity = 21; + string scheduler_name = 22; + repeated Toleration tolerations = 23; + repeated HostAlias host_aliases = 24; + string priority_class_name = 25; + google.protobuf.Int32Value priority = 26; + PodDNSConfig dns_config = 27; + repeated PodReadinessGate readiness_gates = 28; + google.protobuf.StringValue runtime_class_name = 29; + google.protobuf.BoolValue enable_service_links = 30; + map overhead = 31; + repeated TopologySpreadConstraint topology_spread_constraints = 32; + google.protobuf.BoolValue set_host_name_as_fqdn = 33; + PodOS os = 34; + google.protobuf.BoolValue host_users = 35; + repeated PodSchedulingGate scheduling_gates = 36; + repeated PodResourceClaim resource_claims = 37; + +} +message EphemeralContainer { + EphemeralContainerCommon ephemeral_container_common = 1; + string target_container_name = 2; +} +message EphemeralContainerCommon { + string name = 1; + string image = 2; + repeated string command = 3; + repeated string args = 4; + string working_dir = 5; + repeated ContainerPort ports = 6; + repeated EnvFromSource env_from = 7; + repeated EnvVar env = 8; + ResourceRequirements resources = 9; + repeated ContainerResizePolicy resize_policy = 10; + ContainerRestartPolicy restart_policy = 11; + repeated VolumeMount volume_mounts = 12; + repeated VolumeDevice volume_devices = 13; + Probe readiness_probe = 14; + Lifecycle life_cycle = 15; + string termination_message_path = 16; + TerminationMessagePolicy termination_message_policy = 17; + PullPolicy image_pull_policy = 18; + SecurityContext security_context = 19; + bool stdin = 20; + bool stdin_once = 21; + bool tty = 22; +} +message PodReadinessGate { + PodConditionType condition_type = 1; +} +enum PodConditionType { + POD_CONDITION_TYPE_UNSPECIFIED = 0; + POD_CONDITION_TYPE_CONTAINERS_READY = 1; + POD_CONDITION_TYPE_INITIALIZED = 2; + POD_CONDITION_TYPE_READY = 3; + POD_CONDITION_TYPE_POD_SCHEDULED = 4; + POD_CONDITION_TYPE_DISRUPTION_TARGET = 5; + POD_CONDITION_TYPE_POD_READY_TO_START_CONTAINERS = 6; +} + +message TopologySpreadConstraint { + int32 max_skew = 1; + string topology_key = 2; + UnsatisfiableConstraintAction when_unsatisfiable = 3; + LabelSelector label_selector = 4; + google.protobuf.Int32Value min_domains = 5; + NodeInclusionPolicy node_affinity_policy = 6; + NodeInclusionPolicy node_taints_policy = 7; + repeated string match_label_keys = 8; +} + +enum UnsatisfiableConstraintAction{ + UNSATISFIABLE_CONSTRAINT_ACTION_UNSPECIFIED = 0; + UNSATISFIABLE_CONSTRAINT_ACTION_DO_NOT_SCHEDULE = 1; + UNSATISFIABLE_CONSTRAINT_ACTION_SCHEDULE_ANYWAY = 2; +} + +enum NodeInclusionPolicy { + NODE_INCLUSION_POLICY_UNSPECIFIED = 0; + NODE_INCLUSION_POLCIY_IGNORE = 1; + NODE_INCLUSION_POLCIY_HONOR = 2; +} + +message PodSchedulingGate { + string name = 1; +} + +message PodOS { + string name = 1; +} + + +message PodResourceClaim { + string name = 1; + ClaimSource source = 2; +} + +message ClaimSource { + google.protobuf.StringValue resource_claim_name = 1; + google.protobuf.StringValue resource_claim_template_name = 2; +} + + +message GPUSpec { + string name = 1; + int64 quantity = 2; +} +message NamePath { + string name = 1; + string path = 2; +} +message SecretInfo { + string name = 1; + string path = 2; + SecretType type = 3; +} +enum SecretType { + SECRET_TYPE_UNSPECIFIED = 0; + SECRET_TYPE_GCP_SERVICE_ACCOUNT = 1; + SECRET_TYPE_HADOOP_DELEGATION_TOKEN = 2; + SECRET_TYPE_GENERIC = 3; +} +message Affinity { + NodeAffinity node_affinity = 1; + PodAffinity pod_affinity = 2; + PodAntiAffinity pod_anti_affinity = 3; +} +message PodAntiAffinity { + LabelSelector label_selector = 1; + repeated string namespaces = 2; + string topology_key = 3; + LabelSelector namespace_selector = 4; + repeated string match_label_keys = 5; + repeated string mismatch_label_keys = 6; +} +message PodAffinity { + repeated PodAffinityTerm required_during_scheduling_ignored_during_execution = 1; + repeated WeightedPodAffinityTerm preferred_during_scheduling_ignored_during_execution = 2; +} +message WeightedPodAffinityTerm { + int32 weight = 1; + PodAffinityTerm pod_affinity_term = 2; +} +message PodAffinityTerm { + LabelSelector label_selector = 1; + repeated string namespaces = 2; + string topology_key = 3; + LabelSelector namespace_selector = 4; + repeated string match_label_keys = 5; + repeated string mismatch_label_keys = 6; +} + +message LabelSelector { + map match_labels = 1; + LabelSelectorRequirement match_expressions = 2; +} + +message LabelSelectorRequirement { + string key = 1; + LabelSelectorOperator operator = 2; + repeated string values = 3; +} + +enum LabelSelectorOperator { + LABEL_SELECTOR_OPERATOR_UNSPECIFIED = 0; + LABEL_SELECTOR_OPERATOR_IN = 1; + LABEL_SELECTOR_OPERATOR_NOT_IN = 2; + LABEL_SELECTOR_OPERATOR_EXISTS = 3; + LABEL_SELECTOR_OPERATOR_DOES_NOT_EXIST = 4; + +} + +message NodeAffinity { + NodeSelector required_during_scheduling_ignored_during_execution = 1; + repeated PreferredSchedulingTerm preferred_during_scheduling_ignored_during_execution = 2; +} + +message PreferredSchedulingTerm { + int32 weight = 1; + NodeSelectorTerm preference = 2; +} + +message NodeSelector { + repeated NodeSelectorTerm node_selector_terms = 1; +} +message NodeSelectorTerm { + repeated NodeSelectorRequirement match_expressions = 1; + repeated NodeSelectorRequirement match_fields = 2; +} + +message NodeSelectorRequirement { + string key = 1; + NodeSelectorOperator operator = 2; + repeated string values = 3; +} + +enum NodeSelectorOperator { + NODE_SELECTOR_OPERATOR_UNSPECIFIED = 0; + NODE_SELECTOR_OPERATOR_IN = 1; + NODE_SELECTOR_OPERATOR_NOT_IN = 2; + NODE_SELECTOR_OPERATOR_EXISTS = 3; + NODE_SELECTOR_OPERATOR_DOES_NOT_EXIST = 4; + NODE_SELECTOR_OPERATOR_GT = 5; + NODE_SELECTOR_OPERATOR_LT = 6; +} + +message Toleration { + string key = 1; + TolerationOperator operator = 2; + string value = 3; + TaintEffect effect = 4; + google.protobuf.Int64Value toleration_seconds = 5; +} +enum TaintEffect { + TAINT_EFFECT_UNSPECIFIED = 0; + TAINT_EFFECT_NO_SCHEDULE = 1; + TAINT_EFFECT_PREFER_NO_SCHEDULE = 2; + TAINT_EFFECT_NO_SCHEDULE_NO_ADMIT = 3; + TAINT_EFFECT_NO_EXECUTE = 4; +} +enum TolerationOperator { + TOLERATION_OPERATOR_UNSPECIFIED = 0; + TOLERATION_OPERATOR_EXISTS = 1; + TOLERATION_OPERATOR_EQUAL = 2; +} + + +message PodSecurityContext { + SELinuxOptions se_linux_options = 1; + WindowsSecurityContextOptions windows_security_context_options = 2; + google.protobuf.Int64Value run_as_user = 3; + google.protobuf.Int64Value run_as_group = 4; + google.protobuf.BoolValue run_as_nonroot = 5; + repeated int64 supplemental_groups = 6; + google.protobuf.Int64Value fs_group = 7; + repeated Sysctl sys_ctl = 8; + PodFSGroupChangePolicy fs_group_change_policy = 9; + SeccompProfile sec_comp_profile = 10; +} + + +enum PodFSGroupChangePolicy { + POD_FS_GROUP_CHANGE_POLICY_UNSPECIFIED = 0; + POD_FS_GROUP_CHANGE_POLICY_ON_ROOT_MISMATCH = 1; + POD_FS_GROUP_CHANGE_POLICY_ALWAYS = 2; +} + +message Sysctl { + string name = 1; + string value = 2; +} + +message Container { + string name = 1; + string image = 2; + repeated string command = 3; + repeated string args = 4; + string working_dir = 5; + repeated ContainerPort ports = 6; + repeated EnvFromSource env_from = 7; + repeated EnvVar env = 8; + ResourceRequirements resources = 9; + repeated ContainerResizePolicy resize_policy = 10; + ContainerRestartPolicy restart_policy = 11; + repeated VolumeMount volume_mounts = 12; + repeated VolumeDevice volume_devices = 13; + Probe liveness_probe = 14; + Probe readiness_probe = 15; + Probe startup_probe = 16; + Lifecycle life_cycle = 17; + string termination_message_path = 18; + TerminationMessagePolicy termination_message_policy = 19; + PullPolicy image_pull_policy = 20; + SecurityContext security_context = 21; + bool stdin = 22; + bool stdin_once = 23; + bool tty = 24; +} + +message ContainerPort { + string name = 1; + int32 host_port = 2; + int32 container_port = 3; + Protocol protocol = 4; + string host_ip = 5; +} + +enum Protocol { + PROTOCOL_UNSPECIFIED = 0; + PROTOCOL_TCP = 1; + PROTOCOL_UDP = 2; + PROTOCOL_SCTP = 3; + +} + +message ConfigMapEnvSource { + LocalObjectReference local_object_reference = 1; + google.protobuf.BoolValue optional = 2; +} + +message EnvFromSource { + string prefix = 1; + ConfigMapEnvSource config_map_ref = 2; + SecretEnvSource secret_ref = 3; +} + +message SecretEnvSource { + LocalObjectReference local_object_reference = 1; + google.protobuf.BoolValue optional = 3; +} + +message EnvVar { + string name = 1; + string value = 2; + EnvVarSource value_from = 3; +} + +message EnvVarSource { + ObjectFieldSelector field_ref = 1; + ResourceFieldSelector resource_field_ref = 2; + ConfigMapKeySelector config_map_key_ref = 3; + SecretKeySelector secret_key_ref = 4; +} + +message SecretKeySelector { + LocalObjectReference local_object_reference = 1; + string key = 2; + google.protobuf.BoolValue optional = 3; +} + +message ConfigMapKeySelector { + LocalObjectReference local_object_reference = 1; + string key = 2; + google.protobuf.BoolValue optional = 3; +} +message LocalObjectReference { + string name = 1; +} + + +message ResourceFieldSelector { + string container_name = 1; + string resource = 2; + Quantity divisor = 3; +} + +message ObjectFieldSelector { + string api_version = 1; + string field_path = 2; +} + +message ResourceRequirements { + map limits = 1; + map requests = 2; + repeated ResourceClaim claims = 3; +} + +message ResourceClaim { + string name = 1; +} + + + +message ResourceListEntry { + string resource_name = 1; + Quantity quantity = 2; +} + +message Quantity { + Int64Amount i = 1; + InfDecAmount d = 2; + string s = 3; + Format format = 4; +} + +message InfDecAmount { + string dec = 1; // Representing *inf.Dec as a string +} + +message Int64Amount { + int64 value = 1; + Scale scale = 2; +} + +message Scale { + int32 value = 1; +} + +enum Format { + FORMAT_UNSPECIFIED = 0; + DECIMAL_EXPONENT = 1; // e.g., 12e6 + BINARY_SI = 2; // e.g., 12Mi (12 * 2^20) + DECIMAL_SI = 3; // e.g., 12M (12 * 10^6) +} + +message ContainerResizePolicy { + string resource_name = 1; + ResourceResizeRestartPolicy restart_policy = 2; +} +enum ResourceResizeRestartPolicy { + RESOURCE_RESIZE_RESTART_POLICY_UNSPECIFIED = 0; + RESOURCE_RESIZE_RESTART_POLICY_NOT_REQUIRED = 1; + RESOURCE_RESIZE_RESTART_POLICY_RESTART_CONTAINER = 2; +} + + +enum ContainerRestartPolicy { + CONTAINER_RESTART_POLICY_UNSPECIFIED = 0; + CONTAINER_RESTART_POLICY_ALWAYS = 1; +} +message VolumeDevice { + string name = 1; + string device_path = 2; + +} +message ProbeHandler { + ExecAction exec = 1; + HTTPGetAction http_get = 2; + TCPSocketAction tcp_socket = 3; +} + +message Probe { + ProbeHandler probe_handler = 1; + int32 initial_delay_seconds = 2; + int32 timeout_seconds = 3; + int32 period_seconds = 4; + int32 success_threshold = 5; + int32 failure_threshold = 6; + google.protobuf.Int64Value termination_grace_period_seconds = 7; + +} +enum TerminationMessagePolicy { + TERMINATION_MESSAGE_POLICY_UNSPECIFIED = 0; + TERMINATION_MESSAGE_POLICY_FILE = 1; + TERMINATION_MESSAGE_FALLBACK_TO_LOGS_ON_ERROR = 2; +} +enum PullPolicy { + PULL_POLICY_UNSPECIFIED = 0; + PULL_POLICY_ALWAYS = 1; + PULL_POLICY_NEVER = 2; + PULL_POLICY_IF_NOT_PRESENT = 3; +} + +message SecurityContext { + Capabilities capabilities = 1; + google.protobuf.BoolValue privileged = 2; + SELinuxOptions se_linux_options = 3; + WindowsSecurityContextOptions windows_security_context_options = 4; + google.protobuf.Int64Value run_as_user = 5; + google.protobuf.Int64Value run_as_group = 6; + google.protobuf.BoolValue run_as_non_root = 7; + google.protobuf.BoolValue read_only_file_system = 8; + google.protobuf.BoolValue allow_privilege_escalation = 9; + ProcMountType proc_mount = 10; + SeccompProfile sec_comp_profile = 11; + +} +message Capabilities { + repeated string add = 1; + repeated string drop = 2; +} +message SELinuxOptions { + string user = 1; + string role = 2; + string type = 3; + string level = 4; +} +message WindowsSecurityContextOptions { + google.protobuf.StringValue gmsa_credential_spec_name = 1; + google.protobuf.StringValue gmsa_credential_spec = 2; + google.protobuf.StringValue run_as_user_name = 3; + google.protobuf.BoolValue host_process = 4; + +} +enum ProcMountType { + PROC_MOUNT_TYPE_UNSPECIFIED = 0; //default value + PROC_MOUNT_TYPE_DEFAULT = 1; + PROC_MOUNT_TYPE_UNMASKED = 2; +} + +message SeccompProfile { + SeccompProfileType type = 1; + google.protobuf.StringValue local_host_profile = 2; +} + +// Enum for SeccompProfileType +enum SeccompProfileType { + SECCOMP_PROFILE_TYPE_UNSPECIFIED = 0; + SECCOMP_PROFILE_TYPE_LOCALHOST = 1; + SECCOMP_PROFILE_TYPE_RUNTIME_DEFAULT = 2; + SECCOMP_PROFILE_TYPE_UNCONFINED = 3; +} + + +message PodDNSConfig { + repeated string name_servers = 1; + repeated string searches = 2; + repeated PodDNSConfigOption options = 3; +} + +message PodDNSConfigOption { + string name = 1; + string value = 2; +} + + +message HostAlias { + string ip = 1; + repeated string host_names = 2; +} + +message Lifecycle { + LifecycleHandler post_start = 1; + LifecycleHandler pre_stop = 2; + + // Add more fields as needed +} + +message LifecycleHandler { + ExecAction exec = 1; + HTTPGetAction http_get = 2; + TCPSocketAction tcp_socket = 3; + SleepAction sleep = 4; + // Add more fields as needed +} + +message SleepAction { + int64 seconds = 1; +} +message TCPSocketAction { + IntOrString port = 1; + string host = 2; +} + +message ExecAction { + repeated string command = 1; +} + +// Message for HTTPGetAction +message HTTPGetAction { + string path = 1; + IntOrString port = 2; + string host = 3; + URIScheme scheme = 4; + repeated HTTPHeader http_headers = 5; +} + +// Enum for URI Scheme +enum URIScheme { + URISCHEME_UNSPECIFIED = 0; + URISCHEME_HTTP = 1; // http:// + URISCHEME_HTTPS = 2; // https:// +} + +// Message for HTTPHeader +message HTTPHeader { + string name = 1; + string value = 2; +} + +// Message for IntOrString (since intstr.IntOrString can be either int or string) +message IntOrString { + oneof type { + int32 int_val = 1; + string str_val = 2; + } +} + + + + +message Ports { + string name = 1; + string protocol = 2; + string container_port = 3; +} + +message ExecutorSpec { + SparkPodSpec spark_pod_spec = 1; + google.protobuf.Int32Value instances = 2; + google.protobuf.StringValue core_request = 3; + google.protobuf.StringValue java_options = 4; + Lifecycle life_cycle = 5; + google.protobuf.BoolValue delete_on_termination = 6; + repeated Ports ports = 7; + google.protobuf.StringValue priority_class_name = 8; + // Add more fields as needed +} + +// Volume and VolumeMount +message Volume { + string name = 1; + string type = 2; // e.g., "hostPath", "emptyDir", etc. + string path = 3; + // Add more fields as needed for different volume types +} + +message VolumeMount { + string name = 1; + string mount_path = 2; + bool read_only = 3; +} + +// SparkApplicationStatus (simplified) +message SparkApplicationStatus { + string application_state = 1; + string spark_application_id = 2; + string submission_id = 3; + // Add more fields as needed +} + +// The SparkApplication CRD +message SparkApplication { + ObjectMeta metadata = 1; + SparkApplicationSpec spec = 2; + SparkApplicationStatus status = 3; +} + +// The request message containing the SparkApplication and submission ID. +message RunAltSparkSubmitRequest { + SparkApplication spark_application = 1; + string submission_id = 2; +} + +// The response message indicating success or failure. +message RunAltSparkSubmitResponse { + bool success = 1; + string error_message = 2; +} + +// Dependencies specifies all possible types of dependencies of a Spark application. +message Dependencies { + repeated string jars = 1; + repeated string files = 2; + repeated string py_files = 3; + repeated string packages = 4; + repeated string exclude_packages = 5; + repeated string repositories = 6; + repeated string archives = 7; +} + +message DynamicAllocation { + bool enabled = 1; + int32 initial_executors = 2; + int32 min_executors = 3; + int32 max_executors = 4; + int64 shuffle_tracking_timeout = 5; +} + +// The Spark submit service definition. +service SparkSubmitService { + rpc RunAltSparkSubmit(RunAltSparkSubmitRequest) returns (RunAltSparkSubmitResponse); +} \ No newline at end of file diff --git a/proto/spark_submit_grpc.pb.go b/proto/spark_submit_grpc.pb.go new file mode 100644 index 000000000..cef3d1a69 --- /dev/null +++ b/proto/spark_submit_grpc.pb.go @@ -0,0 +1,125 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.5.1 +// - protoc v5.29.3 +// source: proto/spark_submit.proto + +package spark + +import ( + context "context" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.64.0 or later. +const _ = grpc.SupportPackageIsVersion9 + +const ( + SparkSubmitService_RunAltSparkSubmit_FullMethodName = "/spark.SparkSubmitService/RunAltSparkSubmit" +) + +// SparkSubmitServiceClient is the client API for SparkSubmitService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +// +// The Spark submit service definition. +type SparkSubmitServiceClient interface { + RunAltSparkSubmit(ctx context.Context, in *RunAltSparkSubmitRequest, opts ...grpc.CallOption) (*RunAltSparkSubmitResponse, error) +} + +type sparkSubmitServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewSparkSubmitServiceClient(cc grpc.ClientConnInterface) SparkSubmitServiceClient { + return &sparkSubmitServiceClient{cc} +} + +func (c *sparkSubmitServiceClient) RunAltSparkSubmit(ctx context.Context, in *RunAltSparkSubmitRequest, opts ...grpc.CallOption) (*RunAltSparkSubmitResponse, error) { + cOpts := append([]grpc.CallOption{grpc.StaticMethod()}, opts...) + out := new(RunAltSparkSubmitResponse) + err := c.cc.Invoke(ctx, SparkSubmitService_RunAltSparkSubmit_FullMethodName, in, out, cOpts...) + if err != nil { + return nil, err + } + return out, nil +} + +// SparkSubmitServiceServer is the server API for SparkSubmitService service. +// All implementations must embed UnimplementedSparkSubmitServiceServer +// for forward compatibility. +// +// The Spark submit service definition. +type SparkSubmitServiceServer interface { + RunAltSparkSubmit(context.Context, *RunAltSparkSubmitRequest) (*RunAltSparkSubmitResponse, error) + mustEmbedUnimplementedSparkSubmitServiceServer() +} + +// UnimplementedSparkSubmitServiceServer must be embedded to have +// forward compatible implementations. +// +// NOTE: this should be embedded by value instead of pointer to avoid a nil +// pointer dereference when methods are called. +type UnimplementedSparkSubmitServiceServer struct{} + +func (UnimplementedSparkSubmitServiceServer) RunAltSparkSubmit(context.Context, *RunAltSparkSubmitRequest) (*RunAltSparkSubmitResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method RunAltSparkSubmit not implemented") +} +func (UnimplementedSparkSubmitServiceServer) mustEmbedUnimplementedSparkSubmitServiceServer() {} +func (UnimplementedSparkSubmitServiceServer) testEmbeddedByValue() {} + +// UnsafeSparkSubmitServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to SparkSubmitServiceServer will +// result in compilation errors. +type UnsafeSparkSubmitServiceServer interface { + mustEmbedUnimplementedSparkSubmitServiceServer() +} + +func RegisterSparkSubmitServiceServer(s grpc.ServiceRegistrar, srv SparkSubmitServiceServer) { + // If the following call pancis, it indicates UnimplementedSparkSubmitServiceServer was + // embedded by pointer and is nil. This will cause panics if an + // unimplemented method is ever invoked, so we test this at initialization + // time to prevent it from happening at runtime later due to I/O. + if t, ok := srv.(interface{ testEmbeddedByValue() }); ok { + t.testEmbeddedByValue() + } + s.RegisterService(&SparkSubmitService_ServiceDesc, srv) +} + +func _SparkSubmitService_RunAltSparkSubmit_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RunAltSparkSubmitRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(SparkSubmitServiceServer).RunAltSparkSubmit(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: SparkSubmitService_RunAltSparkSubmit_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(SparkSubmitServiceServer).RunAltSparkSubmit(ctx, req.(*RunAltSparkSubmitRequest)) + } + return interceptor(ctx, in, info, handler) +} + +// SparkSubmitService_ServiceDesc is the grpc.ServiceDesc for SparkSubmitService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var SparkSubmitService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "spark.SparkSubmitService", + HandlerType: (*SparkSubmitServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "RunAltSparkSubmit", + Handler: _SparkSubmitService_RunAltSparkSubmit_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "proto/spark_submit.proto", +}