Re: spark-submit exit status on k8s

2020-04-05 Thread Yinan Li
Not sure if you are aware of this new feature in Airflow
https://issues.apache.org/jira/browse/AIRFLOW-6542. It's a way to use
Airflow to orchestrate spark applications run using the Spark K8S operator (
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator).


On Sun, Apr 5, 2020 at 8:25 AM Masood Krohy 
wrote:

> Another, simpler solution that I just thought of: just add an operation at
> the end of your Spark program to write an empty file somewhere, with
> filename SUCCESS for example. Add a stage to your AirFlow graph to check
> the existence of this file after running spark-submit. If the file is
> absent, then the Spark app must have failed.
>
> The above should work if you want to avoid dealing with the REST API for
> monitoring.
>
> Masood
>
> __
>
> Masood Krohy, Ph.D.
> Data Science Advisor|Platform Architecthttps://www.analytical.works
>
> On 4/4/20 10:54 AM, Masood Krohy wrote:
>
> I'm not in the Spark dev team, so cannot tell you why that priority was
> chosen for the JIRA issue or if anyone is about to finish the work on that;
> I'll let others jump in if they know.
>
> Just wanted to offer a potential solution so that you can move ahead in
> the meantime.
>
> Masood
>
> __
>
> Masood Krohy, Ph.D.
> Data Science Advisor|Platform Architecthttps://www.analytical.works
>
> On 4/4/20 7:49 AM, Marshall Markham wrote:
>
> Thank you very much Masood for your fast response. Last question, is the
> current status in Jira representative of the status of the ticket within
> the project team? This seems like a big deal for the K8s implementation and
> we were surprised to find it marked as priority low. Is there any
> discussion of picking up this work in the near future?
>
>
>
> Thanks,
>
> Marshall
>
>
>
> *From:* Masood Krohy 
> 
> *Sent:* Friday, April 3, 2020 9:34 PM
> *To:* Marshall Markham 
> ; user 
> 
> *Subject:* Re: spark-submit exit status on k8s
>
>
>
> While you wait for a fix on that JIRA ticket, you may be able to add an
> intermediary step in your AirFlow graph, calling Spark's REST API after
> submitting the job, and dig into the actual status of the application, and
> make a success/fail decision accordingly. You can make repeated calls in a
> loop to the REST API with few seconds delay between each call while the
> execution is in progress until the application fails or succeeds.
>
> https://spark.apache.org/docs/latest/monitoring.html#rest-api
> 
>
> Hope this helps.
>
> Masood
>
> __
>
>
>
> Masood Krohy, Ph.D.
>
> Data Science Advisor|Platform Architect
>
> https://www.analytical.works 
> 
>
> On 4/3/20 8:23 AM, Marshall Markham wrote:
>
> Hi Team,
>
>
>
> My team recently conducted a POC of Kubernetes/Airflow/Spark with great
> success. The major concern we have about this system, after the completion
> of our POC is a behavior of spark-submit. When called with a Kubernetes API
> endpoint as master spark-submit seems to always return exit status 0. This
> is obviously a major issue preventing us from conditioning job graphs on
> the success or failure of our Spark jobs. I found Jira ticket SPARK-27697
> under the Apache issues covering this bug. The ticket is listed as minor
> and does not seem to have any activity recently. I would like to up vote it
> and ask if there is anything I can do to move this forward. This could be
> the one thing standing between my team and our preferred batch workload
> implementation. Thank you.
>
>
>
> *Marshall Markham*
>
> Data Engineer
>
> PrecisionLender, a Q2 Company
>
>
>
> NOTE: This communication and any attachments are for the sole use of the
> intended recipient(s) and may contain confidential and/or privileged
> information. Any unauthorized review, use, disclosure or distribution is
> prohibited. If you are not the intended recipient, please contact the
> sender by replying to this email, and destroy all copies of the original
> message.
>
> NOTE: This communication and any attachments are for the sole use of the
> intended recipient(s) and may contain confidential and/or privileged
> information. Any unauthorized review, use, disclosure or distribution is
> prohibited. If you are not the intended recipient, please contact the
> sender by replying to this email, and destroy all copies of the original
> message.
>
>


Re: Spark Kubernetes Architecture: Deployments vs Pods that create Pods

2019-01-29 Thread Yinan Li
Hi Wilson,

The behavior of a Deployment doesn't fit with the way Spark executor pods
are run and managed. For example, executor pods are created and deleted per
the requests from the driver dynamically and normally they run to
completion. A Deployment assumes uniformity and statelessness of the set of
Pods it manages, which is not necessarily the case for Spark executors. For
example, executor Pods have unique executor IDs. Dynamic resource
allocation doesn't play well with a Deployment as scaling or shrinking the
number of executor Pods requires a rolling update with a Deployment, which
means restarting all the executor Pods. In the Kubernetes mode, the driver
is effectively a custom controller of executor Pods that adds or deletes
Pods per requests from the driver, and watches the status of the Pods.

The way Flink on Kubernetes works, as you said, is basically running the
Flink job/task managers using Deployments. A equivalent is running a
standalone Spark cluster on top of Kubernetes. If you want auto-restart for
Spark streaming jobs, I would suggest you take a look at the K8S Spark
Operator .

On Tue, Jan 29, 2019 at 5:53 AM WILSON Frank <
frank.wil...@uk.thalesgroup.com> wrote:

> Hi,
>
>
>
> I’ve been playing around with Spark Kubernetes deployments over the past
> week and I’m curious to know why Spark deploys as a driver pod that creates
> more worker pods.
>
>
>
> I’ve read that it’s normal to use Kubernetes Deployments to create a
> distributed service, so I am wondering why Spark just creates Pods. I
> suppose the driver program
>
> is ‘the odd one out’ so it doesn’t belong in a Deployment or ReplicaSet,
> but maybe the workers could be Deployment? Is this something to do with
> data locality?
>
>
>
> I have tried Streaming pipelines on Kubernetes yet, are these also Pods
> that create Pods rather than Deployments? It seems more important for a
> streaming pipeline to be ‘durable’[1] as the Kubernetes documentation might
> say.
>
>
>
> I ask this question partly because the Kubernetes deployment of Spark is
> still experimental and I am wondering whether this aspect of the deployment
> might change.
>
>
>
> I had a look at the Flink[2] documentation and it does seem to use
> Deployments however these seem to be a lightweight job/task manager that
> accepts Flink jobs. It sounds actually like running a lightweight version
> YARN inside containers on Kubernetes.
>
>
>
>
>
> Thanks,
>
>
>
>
>
> Frank
>
>
>
> [1]
> https://kubernetes.io/docs/concepts/workloads/pods/pod/#durability-of-pods-or-lack-thereof
>
> [2]
> https://ci.apache.org/projects/flink/flink-docs-stable/ops/deployment/kubernetes.html
>


Re: [K8S] Option to keep the executor pods after job finishes

2018-10-09 Thread Yinan Li
There is currently no such an option. But this has been raised before in
https://issues.apache.org/jira/browse/SPARK-25515.


On Tue, Oct 9, 2018 at 2:17 PM Li Gao  wrote:

> Hi,
>
> Is there an option to keep the executor pods on k8s after the job
> finishes? We want to extract the logs and stats before removing the
> executor pods.
>
> Thanks,
> Li
>


Re: Spark 2.3.1: k8s driver pods stuck in Initializing state

2018-09-26 Thread Yinan Li
The spark-init ConfigMap is used for the init-container that is responsible
for downloading remote dependencies. The k8s submission client run by
spark-submit should create the ConfigMap and add a ConfigMap volume in the
driver pod. Can you provide the command you used to run the job?

On Wed, Sep 26, 2018 at 2:36 PM purna pradeep 
wrote:

> Hello ,
>
>
> We're running spark 2.3.1 on kubernetes v1.11.0 and our driver pods from
> k8s are getting stuck in initializing state like so:
>
> NAME
>   READY STATUS RESTARTS   AGE
>
> my-pod-fd79926b819d3b34b05250e23347d0e7-driver   0/1   Init:0/1   0
> 18h
>
>
> And from *kubectl describe pod*:
>
> *Warning  FailedMount  9m (x128 over 4h) * kubelet, 10.47.96.167  Unable
> to mount volumes for pod
> "my-pod-fd79926b819d3b34b05250e23347d0e7-driver_spark(1f3aba7b-c10f-11e8-bcec-1292fec79aba)":
> timeout expired waiting for volumes to attach or mount for pod
> "spark"/"my-pod-fd79926b819d3b34b05250e23347d0e7-driver". list of unmounted
> volumes=[spark-init-properties]. list of unattached
> volumes=[spark-init-properties download-jars-volume download-files-volume
> spark-token-tfpvp]
>   *Warning  FailedMount  4m (x153 over 4h)  kubelet,* 10.47.96.167
> MountVolume.SetUp failed for volume "spark-init-properties" : configmaps
> "my-pod-fd79926b819d3b34b05250e23347d0e7-init-config" not found
>
> From what I can see in *kubectl get configmap* the init config map for
> the driver pod isn't there.
>
> Am I correct in assuming since the configmap isn't being created the
> driver pod will never start (hence stuck in init)?
>
> Where does the init config map come from?
>
> Why would it not be created?
>
>
> Please suggest
>
> Thanks,
> Purna
>
>


Re: Python kubernetes spark 2.4 branch

2018-09-25 Thread Yinan Li
Can you give more details on how you ran your app, did you build your own
image, and which image are you using?

On Tue, Sep 25, 2018 at 10:23 AM Garlapati, Suryanarayana (Nokia -
IN/Bangalore)  wrote:

> Hi,
>
> I am trying to run spark python testcases on k8s based on tag
> spark-2.4-rc1. When the dependent files are passed through the --py-files
> option, they are not getting resolved by the main python script. Please let
> me know, is this a known issue?
>
>
>
> Regards
>
> Surya
>
>
>


Re: [K8S] Spark initContainer custom bootstrap support for Spark master

2018-08-16 Thread Yinan Li
Yes, the init-container has been removed in the master branch. The
init-container was used in 2.3.x only for downloading remote dependencies,
which is now handled by running spark-submit in the driver. If you need to
run custom bootstrap scripts using an init-container, the best option would
be to use a mutating admission webhook to inject your init-container into
the Spark pods. Another option is to create a custom image that runs the
scripts prior to entering the entrypoint.

Yinan

On Wed, Aug 15, 2018 at 9:12 AM Li Gao  wrote:

> Hi,
>
> We've noticed on the latest Master (not Spark 2.3.1 branch), the support
> for Kubernetes initContainer is no longer there. What would be the path
> forward if we need to do custom bootstrap actions (i.e. run additional
> scripts) prior to driver/executor container entering running mode?
>
> Thanks,
> Li
>
>


Re: Kubernetes security context when submitting job through k8s servers

2018-07-09 Thread Yinan Li
It's still under design review. It's unlikely that it will go into 2.4.

On Mon, Jul 9, 2018 at 3:46 PM trung kien  wrote:

> Thanks Li,
>
> Inread through the ticket, be able to pass pod YAML file would be amazing.
>
> Do you have any target date for production or incubator? I really want to
> try out this feature.
>
> On Mon, Jul 9, 2018 at 4:48 PM Yinan Li  wrote:
>
>> Spark on k8s currently doesn't support specifying a custom
>> SecurityContext of the driver/executor pods. This will be supported by the
>> solution to https://issues.apache.org/jira/browse/SPARK-24434.
>>
>> On Mon, Jul 9, 2018 at 2:06 PM trung kien  wrote:
>>
>>> Dear all,
>>>
>>> Is there any way to includes security context (
>>> https://kubernetes.io/docs/tasks/configure-pod-container/security-context/)
>>> when submitting job through k8s servers?
>>>
>>> I'm trying to first spark jobs on Kubernetes through spark-submit:
>>>
>>> bin/spark-submit --master k8s://https://API_SERVERS --deploy-mode
>>> cluster --name spark-pi --class org.apache.spark.examples.SparkPi --conf
>>> spark.kubernetes.namespace=NAMESPACE --conf spark.executor.instances=3
>>> --conf spark.kubernetes.container.image= --conf
>>> spark.kubernetes.driver.pod.name=spark-pi-driver
>>> local:///opt/spark/examples/jars/spark-examples_2.11-2.3.1.jar
>>>
>>> But the job was rejected because the pod (created by spark-submit)
>>> doesn't have security context to run as my account (Our policy doesn't
>>> allow us to runAsUser root)
>>>
>>> I check the code under KubernetesClientApplication.scala
>>> <https://github.com/apache/spark/blob/master/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala>,
>>> it doesn't seems to support security context by configuration.
>>>
>>> Is there any solution to get arround this issue? is there any patch that
>>> support this?
>>>
>>> --
>>> Thanks
>>> Kien
>>>
>> --
> Thanks
> Kien
>


Re: Kubernetes security context when submitting job through k8s servers

2018-07-09 Thread Yinan Li
Spark on k8s currently doesn't support specifying a custom SecurityContext
of the driver/executor pods. This will be supported by the solution to
https://issues.apache.org/jira/browse/SPARK-24434.

On Mon, Jul 9, 2018 at 2:06 PM trung kien  wrote:

> Dear all,
>
> Is there any way to includes security context (
> https://kubernetes.io/docs/tasks/configure-pod-container/security-context/)
> when submitting job through k8s servers?
>
> I'm trying to first spark jobs on Kubernetes through spark-submit:
>
> bin/spark-submit --master k8s://https://API_SERVERS --deploy-mode cluster
> --name spark-pi --class org.apache.spark.examples.SparkPi --conf
> spark.kubernetes.namespace=NAMESPACE --conf spark.executor.instances=3
> --conf spark.kubernetes.container.image= --conf
> spark.kubernetes.driver.pod.name=spark-pi-driver
> local:///opt/spark/examples/jars/spark-examples_2.11-2.3.1.jar
>
> But the job was rejected because the pod (created by spark-submit) doesn't
> have security context to run as my account (Our policy doesn't allow us to
> runAsUser root)
>
> I check the code under KubernetesClientApplication.scala
> ,
> it doesn't seems to support security context by configuration.
>
> Is there any solution to get arround this issue? is there any patch that
> support this?
>
> --
> Thanks
> Kien
>


Re: Spark 2.3 driver pod stuck in Running state — Kubernetes

2018-06-08 Thread Yinan Li
Yes, it looks like it is because there's not enough resources to run the
executor pods. Have you seen pending executor pods?

On Fri, Jun 8, 2018, 11:49 AM Thodoris Zois  wrote:

> As far as I know from Mesos with Spark, it is a running state and not a
> pending one. What you see is normal, but if I am wrong somebody correct me.
>
>  Spark driver at start operates normally (running state) but when it comes
> to start up executors, then it cannot allocate resources for them and
> hangs..
>
> - Thodoris
>
> On 8 Jun 2018, at 21:24, purna pradeep  wrote:
>
> Hello,
>
> When I run spark-submit on k8s cluster I’m
>
> Seeing driver pod stuck in Running state and when I pulled driver pod logs
> I’m able to see below log
>
> I do understand that this warning might be because of lack of cpu/ Memory
> , but I expect driver pod be in “Pending” state rather than “ Running”
> state though actually it’s not Running
>
> So I had kill the driver pod and resubmit the job
>
> Please suggest here !
>
> 2018-06-08 14:38:01 WARN TaskSchedulerImpl:66 - Initial job has not
> accepted any resources; check your cluster UI to ensure that workers are
> registered and have sufficient resources
>
> 2018-06-08 14:38:16 WARN TaskSchedulerImpl:66 - Initial job has not
> accepted any resources; check your cluster UI to ensure that workers are
> registered and have sufficient resources
>
> 2018-06-08 14:38:31 WARN TaskSchedulerImpl:66 - Initial job has not
> accepted any resources; check your cluster UI to ensure that workers are
> registered and have sufficient resources
>
> 2018-06-08 14:38:46 WARN TaskSchedulerImpl:66 - Initial job has not
> accepted any resources; check your cluster UI to ensure that workers are
> registered and have sufficient resources
>
> 2018-06-08 14:39:01 WARN TaskSchedulerImpl:66 - Initial job has not
> accepted any resources; check your cluster UI to ensure that workers are
> registered and have sufficient resources
>
>


Re: [Spark on Google Kubernetes Engine] Properties File Error

2018-04-30 Thread Yinan Li
Also looks like you are mixing configuration properties from different
versions of Spark on Kubernetes.
"spark.kubernetes.{driver|executor}.docker.image" is only available in the
apache-spark-on-k8s fork, whereas "spark.kubernetes.container.image" is new
in Spark 2.3.0. Please make sure you use the same version of Spark to run
spark-submit and to build your image. You also need to make sure that you
use the right properties for the version of Spark you use. BTW: we suggest
people to use the official Spark 2.3.0 release instead of the fork. The
guide is at http://spark.apache.org/docs/latest/running-on-kubernetes.html.

On Mon, Apr 30, 2018 at 12:09 PM, Yinan Li <liyinan...@gmail.com> wrote:

>  Which version of Spark are you using to run spark-submit, and which
> version of Spark your container image is based off? This looks to be caused
> my mismatched versions of Spark used for spark-submit and for the
> driver/executor at runtime.
>
> On Mon, Apr 30, 2018 at 12:00 PM, Holden Karau <hol...@pigscanfly.ca>
> wrote:
>
>> So, while its not perfect, I have a guide focused on running custom Spark
>> on GKE https://cloud.google.com/blog/big-data/2018/03/testing-f
>> uture-apache-spark-releases-and-changes-on-google-kubernetes
>> -engine-and-cloud-dataproc and if you want to run pre-built Spark on GKE
>> there is a solutions article - https://cloud.google.com/sol
>> utions/spark-on-kubernetes-engine which could be relevant.
>>
>> On Mon, Apr 30, 2018 at 7:51 PM, Eric Wang <ericswang1...@gmail.com>
>> wrote:
>>
>>> Hello all,
>>>
>>> I've been trying to spark-submit a job to the Google Kubernetes Engine
>>> but I keep encountering a "Exception in thread "main"
>>> java.lang.IllegalArgumentException: Server properties file given at
>>> /opt/spark/work-dir/driver does not exist or is not a file."
>>> error. I'm unsure of how to even begin debugging this so any help would
>>> be greatly appreciated. I've attached the logs and the full spark-submit
>>> command I'm running here: https://gist.github.com/
>>> erkkel/c04a0b5ca60ad755cf62e9ad18e5b7ed
>>>
>>> For reference, I've been following this guide: https://apache-spark-on
>>> -k8s.github.io/userdocs/running-on-kubernetes.html
>>>
>>> Thanks,
>>> Eric
>>>
>>>
>>
>>
>> --
>> Twitter: https://twitter.com/holdenkarau
>>
>
>


Re: [Spark on Google Kubernetes Engine] Properties File Error

2018-04-30 Thread Yinan Li
 Which version of Spark are you using to run spark-submit, and which
version of Spark your container image is based off? This looks to be caused
my mismatched versions of Spark used for spark-submit and for the
driver/executor at runtime.

On Mon, Apr 30, 2018 at 12:00 PM, Holden Karau  wrote:

> So, while its not perfect, I have a guide focused on running custom Spark
> on GKE https://cloud.google.com/blog/big-data/2018/03/testing-
> future-apache-spark-releases-and-changes-on-google-
> kubernetes-engine-and-cloud-dataproc and if you want to run pre-built
> Spark on GKE there is a solutions article - https://cloud.google.com/
> solutions/spark-on-kubernetes-engine which could be relevant.
>
> On Mon, Apr 30, 2018 at 7:51 PM, Eric Wang 
> wrote:
>
>> Hello all,
>>
>> I've been trying to spark-submit a job to the Google Kubernetes Engine
>> but I keep encountering a "Exception in thread "main"
>> java.lang.IllegalArgumentException: Server properties file given at
>> /opt/spark/work-dir/driver does not exist or is not a file."
>> error. I'm unsure of how to even begin debugging this so any help would
>> be greatly appreciated. I've attached the logs and the full spark-submit
>> command I'm running here: https://gist.github.com/
>> erkkel/c04a0b5ca60ad755cf62e9ad18e5b7ed
>>
>> For reference, I've been following this guide: https://apache-spark-on
>> -k8s.github.io/userdocs/running-on-kubernetes.html
>>
>> Thanks,
>> Eric
>>
>>
>
>
> --
> Twitter: https://twitter.com/holdenkarau
>


Re: Spark Kubernetes Volumes

2018-04-12 Thread Yinan Li
Hi Marius,

Spark on Kubernetes does not yet support mounting user-specified volumes
natively. But mounting volume is supported in
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator. Please see
https://github.com/GoogleCloudPlatform/spark-on-k8s-operator/blob/master/docs/user-guide.md#mounting-volumes
.

On Thu, Apr 12, 2018 at 7:50 AM, Marius  wrote:

> Hey,
>
> i have a question regarding the Spark on Kubernetes feature. I would like
> to mount a pre-populated Kubernetes volume into the execution pods of
> Spark. One of my tools that i invoke using the Sparks pipe command requires
> these files to be available on a POSIX compatible FS and they are too large
> to justify copying them around using addFile. If this is not possible i
> would like to know if the community be interested in such a feature.
>
> Cheers
>
> Marius
>
> -
> To unsubscribe e-mail: user-unsubscr...@spark.apache.org
>
>


Re: Spark on Kubernetes (minikube) 2.3 fails with class not found exception

2018-04-10 Thread Yinan Li
The example jar path should be local:///opt/spark/examples/*jars*
/spark-examples_2.11-2.3.0.jar.

On Tue, Apr 10, 2018 at 1:34 AM, Dmitry  wrote:

> Hello spent a lot of time to find what I did wrong , but not found.
> I have a minikube WIndows based cluster ( Hyper V as hypervisor ) and try
> to run examples against Spark 2.3. Tried several  docker images builds:
> * several  builds that I build myself
> * andrusha/spark-k8s:2.3.0-hadoop2.7 from docker  hub
> But when I try to submit job driver log returns  class not found exception
> org.apache.spark.examples.SparkPi
>
> spark-submit --master k8s://https://ip:8443  --deploy-mode cluster
> --name spark-pi --class org.apache.spark.examples.SparkPi --conf
> spark.executor.instances=1 --executor-memory 1G --conf spark.kubernete
> s.container.image=andrusha/spark-k8s:2.3.0-hadoop2.7
> local:///opt/spark/examples/spark-examples_2.11-2.3.0.jar
>
> I tried to use https://github.com/apache-spark-on-k8s/spark fork and it
> is works without problems, more complex examples work also.
>


Re: Scala program to spark-submit on k8 cluster

2018-04-04 Thread Yinan Li
Hi Kittu,

What do you mean by "a Scala program"? Do you mean a program that submits a
Spark job to a k8s cluster by running spark-submit programmatically, or
some example Scala application that is to run on the cluster?

On Wed, Apr 4, 2018 at 4:45 AM, Kittu M  wrote:

> Hi,
>
> I’m looking for a Scala program to spark submit a Scala application (spark
> 2.3 job) on k8 cluster .
>
> Any help  would be much appreciated. Thanks
>
>
>


Re: Rest API for Spark2.3 submit on kubernetes(version 1.8.*) cluster

2018-03-20 Thread Yinan Li
One option is the Spark Operator
. It allows
specifying and running Spark applications on Kubernetes using Kubernetes
custom resources objects. It takes SparkApplication CRD objects and
automatically submits the applications to run on a Kubernetes cluster.

Yinan

On Tue, Mar 20, 2018 at 7:47 PM, purna pradeep 
wrote:

> Im using kubernetes cluster on AWS to run spark jobs ,im using spark 2.3
> ,now i want to run spark-submit from AWS lambda function to k8s
> master,would like to know if there is any REST interface to run Spark
> submit on k8s Master


Re: Spark 2.3 submit on Kubernetes error

2018-03-11 Thread Yinan Li
Spark on Kubernetes requires the presence of the kube-dns add-on properly
configured. The executors connect to the driver through a headless
Kubernetes service using the DNS name of the service. Can you check if you
have the add-on installed in your cluster? This issue
https://github.com/apache-spark-on-k8s/spark/issues/558 might help.


On Sun, Mar 11, 2018 at 5:01 PM, purna pradeep 
wrote:

> Getting below errors when I’m trying to run spark-submit on k8 cluster
>
>
> *Error 1*:This looks like a warning it doesn’t interrupt the app running
> inside executor pod but keeps on getting this warning
>
>
> *2018-03-09 11:15:21 WARN  WatchConnectionManager:192 - Exec Failure*
> *java.io.EOFException*
> *   at okio.RealBufferedSource.require(RealBufferedSource.java:60)*
> *   at
> okio.RealBufferedSource.readByte(RealBufferedSource.java:73)*
> *   at okhttp3.internal.ws
> .WebSocketReader.readHeader(WebSocketReader.java:113)*
> *   at okhttp3.internal.ws
> .WebSocketReader.processNextFrame(WebSocketReader.java:97)*
> *   at okhttp3.internal.ws
> .RealWebSocket.loopReader(RealWebSocket.java:262)*
> *   at okhttp3.internal.ws
> .RealWebSocket$2.onResponse(RealWebSocket.java:201)*
> *   at okhttp3.RealCall$AsyncCall.execute(RealCall.java:141)*
> *   at okhttp3.internal.NamedRunnable.run(NamedRunnable.java:32)*
> *   at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)*
> *   at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)*
> *   at java.lang.Thread.run(Thread.java:748)*
>
>
>
> *Error2:* This is intermittent error  which is failing the executor pod
> to run
>
>
> *org.apache.spark.SparkException: External scheduler cannot be
> instantiated*
> * at
> org.apache.spark.SparkContext$.org$apache$spark$SparkContext$$createTaskScheduler(SparkContext.scala:2747)*
> * at org.apache.spark.SparkContext.(SparkContext.scala:492)*
> * at
> org.apache.spark.SparkContext$.getOrCreate(SparkContext.scala:2486)*
> * at
> org.apache.spark.sql.SparkSession$Builder$$anonfun$7.apply(SparkSession.scala:930)*
> * at
> org.apache.spark.sql.SparkSession$Builder$$anonfun$7.apply(SparkSession.scala:921)*
> * at scala.Option.getOrElse(Option.scala:121)*
> * at
> org.apache.spark.sql.SparkSession$Builder.getOrCreate(SparkSession.scala:921)*
> * at
> com.capitalone.quantum.spark.core.QuantumSession$.initialize(QuantumSession.scala:62)*
> * at
> com.capitalone.quantum.spark.core.QuantumSession$.getSparkSession(QuantumSession.scala:80)*
> * at
> com.capitalone.quantum.workflow.WorkflowApp$.getSession(WorkflowApp.scala:116)*
> * at
> com.capitalone.quantum.workflow.WorkflowApp$.main(WorkflowApp.scala:90)*
> * at
> com.capitalone.quantum.workflow.WorkflowApp.main(WorkflowApp.scala)*
> *Caused by: io.fabric8.kubernetes.client.KubernetesClientException:
> Operation: [get]  for kind: [Pod]  with name:
> [myapp-ef79db3d9f4831bf85bda14145fdf113-driver-driver]  in namespace:
> [default]  failed.*
> * at
> io.fabric8.kubernetes.client.KubernetesClientException.launderThrowable(KubernetesClientException.java:62)*
> * at
> io.fabric8.kubernetes.client.KubernetesClientException.launderThrowable(KubernetesClientException.java:71)*
> * at
> io.fabric8.kubernetes.client.dsl.base.BaseOperation.getMandatory(BaseOperation.java:228)*
> * at
> io.fabric8.kubernetes.client.dsl.base.BaseOperation.get(BaseOperation.java:184)*
> * at
> org.apache.spark.scheduler.cluster.k8s.KubernetesClusterSchedulerBackend.(KubernetesClusterSchedulerBackend.scala:70)*
> * at
> org.apache.spark.scheduler.cluster.k8s.KubernetesClusterManager.createSchedulerBackend(KubernetesClusterManager.scala:120)*
> * at
> org.apache.spark.SparkContext$.org$apache$spark$SparkContext$$createTaskScheduler(SparkContext.scala:2741)*
> * ... 11 more*
> *Caused by: java.net.UnknownHostException: kubernetes.default.svc: Try
> again*
> * at java.net.Inet4AddressImpl.lookupAllHostAddr(Native Method)*
> * at java.net.InetAddress$2.lookupAllHostAddr(InetAddress.java:928)*
> * at
> java.net.InetAddress.getAddressesFromNameService(InetAddress.java:1323)*
> * at java.net.InetAddress.getAllByName0(InetAddress.java:1276)*
> * at java.net.InetAddress.getAllByName(InetAddress.java:1192)*
> * at java.net.InetAddress.getAllByName(InetAddress.java:1126)*
> * at okhttp3.Dns$1.lookup(Dns.java:39)*
> * at
> okhttp3.internal.connection.RouteSelector.resetNextInetSocketAddress(RouteSelector.java:171)*
> * at
> okhttp3.internal.connection.RouteSelector.nextProxy(RouteSelector.java:137)*
> * at
> okhttp3.internal.connection.RouteSelector.next(RouteSelector.java:82)*
> * at
> 

Re: handling Remote dependencies for spark-submit in spark 2.3 with kubernetes

2018-03-08 Thread Yinan Li
One thing to note is you may need to have the S3 credentials in the
init-container unless you use a publicly accessible URL. If this is the
case, you can either create a Kubernetes secret and use the Spark config
option for mounting secrets (secrets will be mounted into the
init-container as well as into the main container), or you create a custom
init-container with the credentials baked in.

Yinan

On Thu, Mar 8, 2018 at 12:05 PM, Anirudh Ramanathan <
ramanath...@google.com.invalid> wrote:

> You don't need to create the init-container. It's an implementation detail.
> If you provide a remote uri, and specify 
> spark.kubernetes.container.image=,
> Spark *internally* will add the init container to the pod spec for you.
> *If *for some reason, you want to customize the init container image, you
> can choose to do that using the specific options, but I don't think this is
> necessary in most scenarios. The init container image, driver and executor
> images can be identical by default.
>
>
> On Thu, Mar 8, 2018 at 6:52 AM purna pradeep 
> wrote:
>
>> Im trying to run spark-submit to kubernetes cluster with spark 2.3 docker
>> container image
>>
>> The challenge im facing is application have a mainapplication.jar and
>> other dependency files & jars which are located in Remote location like AWS
>> s3 ,but as per spark 2.3 documentation there is something called kubernetes
>> init-container to download remote dependencies but in this case im not
>> creating any Podspec to include init-containers in kubernetes, as per
>> documentation Spark 2.3 spark/kubernetes internally creates Pods
>> (driver,executor) So not sure how can i use init-container for spark-submit
>> when there are remote dependencies.
>>
>> https://spark.apache.org/docs/latest/running-on-kubernetes.
>> html#using-remote-dependencies
>>
>> Please suggest
>>
>
>
> --
> Anirudh Ramanathan
>


Re: Spark on K8s - using files fetched by init-container?

2018-02-26 Thread Yinan Li
The files specified through --files are localized by the init-container
to /var/spark-data/spark-files by default. So in your case, the file should
be located at /var/spark-data/spark-files/flights.csv locally in the
container.

On Mon, Feb 26, 2018 at 10:51 AM, Jenna Hoole  wrote:

> This is probably stupid user error, but I can't for the life of me figure
> out how to access the files that are staged by the init-container.
>
> I'm trying to run the SparkR example data-manipulation.R which requires
> the path to its datafile. I supply the hdfs location via --files and then
> the full hdfs path.
>
>
> --files hdfs://192.168.0.1:8020/user/jhoole/flights.csv
> local:///opt/spark/examples/src/main/r/data-manipulation.R hdfs://
> 192.168.0.1:8020/user/jhoole/flights.csv
>
> The init-container seems to load my file.
>
> 18/02/26 18:29:09 INFO spark.SparkContext: Added file hdfs://
> 192.168.0.1:8020/user/jhoole/flights.csv at hdfs://192.168.0.1:8020/user/
> jhoole/flights.csv with timestamp 1519669749519
>
> 18/02/26 18:29:09 INFO util.Utils: Fetching hdfs://192.168.0.1:8020/user/
> jhoole/flights.csv to /var/spark/tmp/spark-d943dae6-
> 9b95-4df0-87a3-9f7978d6d4d2/userFiles-4112b7aa-b9e7-47a9-
> bcbc-7f7a01f93e38/fetchFileTemp7872615076522023165.tmp
>
> However, I get an error that my file does not exist.
>
> Error in file(file, "rt") : cannot open the connection
>
> Calls: read.csv -> read.table -> file
>
> In addition: Warning message:
>
> In file(file, "rt") :
>
>   cannot open file 'hdfs://192.168.0.1:8020/user/jhoole/flights.csv': No
> such file or directory
>
> Execution halted
>
> Exception in thread "main" org.apache.spark.SparkUserAppException: User
> application exited with 1
>
> at org.apache.spark.deploy.RRunner$.main(RRunner.scala:104)
>
> at org.apache.spark.deploy.RRunner.main(RRunner.scala)
>
> If I try supplying just flights.csv, I get a different error
>
> --files hdfs://192.168.0.1:8020/user/jhoole/flights.csv
> local:///opt/spark/examples/src/main/r/data-manipulation.R flights.csv
>
> Error: Error in loadDF : analysis error - Path does not exist: hdfs://
> 192.168.0.1:8020/user/root/flights.csv;
>
> Execution halted
>
> Exception in thread "main" org.apache.spark.SparkUserAppException: User
> application exited with 1
>
> at org.apache.spark.deploy.RRunner$.main(RRunner.scala:104)
>
> at org.apache.spark.deploy.RRunner.main(RRunner.scala)
>
> If the path /user/root/flights.csv does exist and I only supply
> "flights.csv" as the file path, it runs to completion successfully.
> However, if I provide the file path as "hdfs://192.168.0.1:8020/user/
> root/flights.csv," I get the same "No such file or directory" error as I
> do initially.
>
> Since I obviously can't put all my hdfs files under /user/root, how do I
> get it to use the file that the init-container is fetching?
>
> Thanks,
> Jenna
>


Re: Spark on K8s with Romana

2018-02-12 Thread Yinan Li
We actually moved away from using the driver pod IP because of
https://github.com/apache-spark-on-k8s/spark/issues/482. The current way
this works is that the driver url is constructed based on the value of
"spark.driver.host" that is set to the DNS name of the headless driver
service in the submission client. So the assumption is that kube-dns exists
and is working. Unfortunately, this is not configurable.


On Mon, Feb 12, 2018 at 1:21 PM, Jenna Hoole  wrote:

> So, we've run into something interesting. In our case, we've got some
> proprietary networking HW which is very feature limited in the TCP/IP
> space, so using Romana, executors can't seem to find the driver using the
> hostname lookup method it's attempting. Is there any way to make it use IP?
>
> Thanks,
> Jenna
>


Announcing Spark on Kubernetes release 0.5.0

2017-11-01 Thread Yinan Li
The Spark on Kubernetes development community is pleased to announce
release 0.5.0
of Apache Spark with Kubernetes as a native scheduler back-end!

This release includes a few bug fixes and the following features:

   - Spark R support
   - Kubernetes 1.8 support
   - Mounts emptyDir volumes for temporary directories on executors in
   static allocation mode

The full release notes are available here:
https://github.com/apache-spark-on-k8s/spark/releases/
tag/v2.2.0-kubernetes-0.5.0

Community resources for Spark on Kubernetes are available at:

   - Slack: https://kubernetes.slack.com
   - User Docs: https://apache-spark-on-k8s.github.io/userdocs/
   - GitHub: https://github.com/apache-spark-on-k8s/spark