You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by ifilonenko <gi...@git.apache.org> on 2018/06/18 16:18:57 UTC

[GitHub] spark pull request #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for Sp...

GitHub user ifilonenko opened a pull request:

    https://github.com/apache/spark/pull/21584

    [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on K8s

    ## What changes were proposed in this pull request?
    
    Introducing R Bindings for Spark R on K8s
    
    - [ ] Running SparkR Job
    
    ## How was this patch tested?
    
    This patch was tested with 
    
    - [x] Unit Tests
    - [ ] Integration Tests


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ifilonenko/spark spark-r

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/21584.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #21584
    
----
commit 45aaefcd6b148c3e83f03d07575a9bb199835267
Author: Ilan Filonenko <if...@...>
Date:   2018-06-18T05:08:06Z

    initial R bindings

commit 884a4e71f25dd8276cd198ad8c3deb91e61634d3
Author: Ilan Filonenko <if...@...>
Date:   2018-06-18T05:11:03Z

    fix Dockerfile

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93663 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93663/testReport)** for PR 21584 at commit [`1f0cba5`](https://github.com/apache/spark/commit/1f0cba59b650e4458e9472933068928d52a54777).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait BarrierTaskContext extends TaskContext `
      * `class BarrierTaskInfo(val address: String)`
      * `class RDDBarrier[T: ClassTag](rdd: RDD[T]) `
      * `case class WorkerOffer(`
      * `trait AnalysisHelper extends QueryPlan[LogicalPlan] `


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #92034 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92034/testReport)** for PR 21584 at commit [`884a4e7`](https://github.com/apache/spark/commit/884a4e71f25dd8276cd198ad8c3deb91e61634d3).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r204713851
  
    --- Diff: resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile ---
    @@ -0,0 +1,29 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You 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
    +#
    +#    http://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.
    +#
    +
    +ARG base_img
    +FROM $base_img
    +WORKDIR /
    +RUN mkdir ${SPARK_HOME}/R
    +COPY R ${SPARK_HOME}/R
    +
    +RUN apk add --no-cache R R-dev
    +
    --- End diff --
    
    This is only for Python packaging. R does not have `/root/.cache` when created by alpine. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r204229663
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala ---
    @@ -71,8 +75,9 @@ private[spark] class KubernetesDriverBuilder(
             case JavaMainAppResource(_) =>
               provideJavaStep(kubernetesConf)
             case PythonMainAppResource(_) =>
    -          providePythonStep(kubernetesConf)}
    -      .getOrElse(provideJavaStep(kubernetesConf))
    +          providePythonStep(kubernetesConf)
    +        case RMainAppResource(_) =>
    +          provideRStep(kubernetesConf)}.getOrElse(provideJavaStep(kubernetesConf))
    --- End diff --
    
    I'd move `}.getOrElse(provideJavaStep(kubernetesConf))` to the next line for clarity?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #94862 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94862/testReport)** for PR 21584 at commit [`6584029`](https://github.com/apache/spark/commit/658402919c080ae4d878d355a4b3a14a4d4d0aad).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r207605696
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -205,7 +218,7 @@ private[spark] object Config extends Logging {
           .createWithDefault(0.1)
     
       val PYSPARK_MAJOR_PYTHON_VERSION =
    -    ConfigBuilder("spark.kubernetes.pyspark.pythonversion")
    +    ConfigBuilder("spark.kubernetes.pyspark.pythonVersion")
    --- End diff --
    
    hmm good call.... I might do that then


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93294 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93294/testReport)** for PR 21584 at commit [`d809f67`](https://github.com/apache/spark/commit/d809f67357afb955c5f398a9ada6523ed43c7f99).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93294/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93547 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93547/testReport)** for PR 21584 at commit [`d036673`](https://github.com/apache/spark/commit/d0366732a9ebef710cb78e52ee42a869002e9040).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test status failure
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1313/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1138/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1399/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #92635 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92635/testReport)** for PR 21584 at commit [`a2f2825`](https://github.com/apache/spark/commit/a2f282555b8a30ee2a50448cb7045819a6f3d483).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #94862 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94862/testReport)** for PR 21584 at commit [`6584029`](https://github.com/apache/spark/commit/658402919c080ae4d878d355a4b3a14a4d4d0aad).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2256/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1201/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r207382976
  
    --- Diff: resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile ---
    @@ -0,0 +1,29 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You 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
    +#
    +#    http://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.
    +#
    +
    +ARG base_img
    +FROM $base_img
    +WORKDIR /
    +RUN mkdir ${SPARK_HOME}/R
    +COPY R ${SPARK_HOME}/R
    +
    +RUN apk add --no-cache R R-dev
    --- End diff --
    
    Agreed. Minor versions are something we *might* want to support in the future. Let's look to include this in a follow-up PR if there is enough desire in custom version arguments to be passed as build_args to the `bin/docker-image-tool.sh` 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r205928623
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -205,7 +218,7 @@ private[spark] object Config extends Logging {
           .createWithDefault(0.1)
     
       val PYSPARK_MAJOR_PYTHON_VERSION =
    -    ConfigBuilder("spark.kubernetes.pyspark.pythonversion")
    +    ConfigBuilder("spark.kubernetes.pyspark.pythonVersion")
    --- End diff --
    
    idiomatic for configs to have camelCase. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    @felixcheung the PRB isn’t currently configured with capability to build —r distributions. Waiting on @shaneknapp, which is why it is unable to build the image. With an R supported environment, it passes all tests. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r204229514
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -79,11 +80,16 @@ function build {
       docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \
         -t $(image_ref spark-py) \
         -f "$PYDOCKERFILE" .
    +
    +    docker build "${BINDING_BUILD_ARGS[@]}" \
    --- End diff --
    
    nit: this doesn't align with the pattern above


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for Sp...

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r196577500
  
    --- Diff: resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile ---
    @@ -0,0 +1,29 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You 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
    +#
    +#    http://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.
    +#
    +
    +ARG base_img
    +FROM $base_img
    +WORKDIR /
    +RUN mkdir ${SPARK_HOME}/R
    +COPY R ${SPARK_HOME}/R
    +
    +RUN apk add --no-cache R R-dev
    --- End diff --
    
    Thoughts @felixcheung ? 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93274/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1400/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1123/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    LGTM


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    yap


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Let's merge at the end of the day pacific time (~5PM-ish) on Friday, August 17, pending any additional feedback on the mailing list thread discussing the subject of including this in 2.4.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92635/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1123/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93664 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93664/testReport)** for PR 21584 at commit [`1f0cba5`](https://github.com/apache/spark/commit/1f0cba59b650e4458e9472933068928d52a54777).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/2256/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    All tests pass locally. Waiting on PRB to be configured. 
    ```
    KubernetesSuite:
    - Run SparkPi with no resources
    - Run SparkPi with a very long application name.
    - Use SparkLauncher.NO_RESOURCE
    - Run SparkPi with a master URL without a scheme.
    - Run SparkPi with an argument.
    - Run SparkPi with custom labels, annotations, and environment variables.
    - Run extraJVMOptions check on driver
    - Run SparkRemoteFileTest using a remote data file
    - Run PySpark on simple pi.py example
    - Run PySpark with Python2 to test a pyfiles example
    - Run PySpark with Python3 to test a pyfiles example
    - Run SparkR on simple dataframe.R example
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/281/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r207603254
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -205,7 +218,7 @@ private[spark] object Config extends Logging {
           .createWithDefault(0.1)
     
       val PYSPARK_MAJOR_PYTHON_VERSION =
    -    ConfigBuilder("spark.kubernetes.pyspark.pythonversion")
    +    ConfigBuilder("spark.kubernetes.pyspark.pythonVersion")
    --- End diff --
    
    We _might_ want to make that (small) change in a separate PR just so that if this slips and doesn't make the cut for 2.4 we aren't trying to change a config string later.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by shaneknapp <gi...@git.apache.org>.
Github user shaneknapp commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    just wanted to briefly chime in:
    
    TL;DR:  this build will fail until the PRB is running on our ubuntu build nodes.
    
    we are currently blocked from testing this stuff w/the current iteration of the PRB on our jenkins for a couple of reasons:
    
    1) the current pull request builder runs only on our old, ancient and crufty centos 6.9 hosts
    2) these hosts have old, ancient and crufty installations of R (3.1.1) and associated libraries
    3) trying to re-create (2) on our shiny, new ubuntu 16.04 boxen has, so far, been an exercise in futility.  :\
    4) this is why we currently have a limited number of ubuntu boxes, and also why things like testing for spark/k8s integration requires an addition build *on top* of the standard PRB build to make things work.
    
    i'm working now w/some folks at databricks (@weshsiao) to get this under control enough so that we can get things moving to the ubuntu hosts ASAP.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1313/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1138/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93666 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93666/testReport)** for PR 21584 at commit [`0c43a1c`](https://github.com/apache/spark/commit/0c43a1cb66597437dbba29594009133afa4bcada).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93662/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93662 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93662/testReport)** for PR 21584 at commit [`131f11f`](https://github.com/apache/spark/commit/131f11f8deb96fa7fa4f78522b73e5bbf2b9345e).
     * This patch **fails to build**.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test status success
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/695/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r207603046
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -205,7 +218,7 @@ private[spark] object Config extends Logging {
           .createWithDefault(0.1)
     
       val PYSPARK_MAJOR_PYTHON_VERSION =
    -    ConfigBuilder("spark.kubernetes.pyspark.pythonversion")
    +    ConfigBuilder("spark.kubernetes.pyspark.pythonVersion")
    --- End diff --
    
    K


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/21584


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r205132947
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala ---
    @@ -71,8 +75,9 @@ private[spark] class KubernetesDriverBuilder(
             case JavaMainAppResource(_) =>
               provideJavaStep(kubernetesConf)
             case PythonMainAppResource(_) =>
    -          providePythonStep(kubernetesConf)}
    -      .getOrElse(provideJavaStep(kubernetesConf))
    +          providePythonStep(kubernetesConf)
    +        case RMainAppResource(_) =>
    +          provideRStep(kubernetesConf)}.getOrElse(provideJavaStep(kubernetesConf))
    --- End diff --
    
    done


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Ok I am merging this to master now. Thanks for the work on this!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/4175/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93666 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93666/testReport)** for PR 21584 at commit [`0c43a1c`](https://github.com/apache/spark/commit/0c43a1cb66597437dbba29594009133afa4bcada).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93273/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r205853788
  
    --- Diff: resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile ---
    @@ -0,0 +1,29 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You 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
    +#
    +#    http://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.
    +#
    +
    +ARG base_img
    +FROM $base_img
    +WORKDIR /
    +RUN mkdir ${SPARK_HOME}/R
    +COPY R ${SPARK_HOME}/R
    +
    +RUN apk add --no-cache R R-dev
    --- End diff --
    
    I think this could make sense to support, the same could be true for Python as well since minor version mismatch can lead to very bad behaviour. It _might_ make sense to do in a follow up PR since we probably can't install all the versions a user might want from apk?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    This PR has been updated to pass Jenkins by removing the `with RTestsSuite` line in `KubernetesSuite`. As such, this feature may be merged and the `with RTestsSuite` will be re-included in a separate PR for when the Jenkins is updated with the new Ubuntu OS. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93666/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test status failure
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1401/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93663/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test status failure
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1498/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93272/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    I see. you are referring to https://github.com/apache/spark/pull/21584#issuecomment-406091994
    got it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93547/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    IMO it's fine we have one version supported in the image and stick with that.
    the tricky thing is having maintainers to keep updating/testing the newer versions in the images (we have history of not able to keep up)
    
    would it be possible for the integration test to build the image running `docker-image-tool.sh -m -t testing build
    `, and then run the integration test with it?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/695/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    I filed https://issues.apache.org/jira/browse/SPARK-25152 for the integration tests.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93386/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93663 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93663/testReport)** for PR 21584 at commit [`1f0cba5`](https://github.com/apache/spark/commit/1f0cba59b650e4458e9472933068928d52a54777).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93802 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93802/testReport)** for PR 21584 at commit [`bd60d05`](https://github.com/apache/spark/commit/bd60d05aa5df3aa67cfd62084ba00dd26d1bf67c).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93274 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93274/testReport)** for PR 21584 at commit [`24fcd9b`](https://github.com/apache/spark/commit/24fcd9b9f3fce302be2088a1b3577cbddc0bdf73).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1120/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r205851170
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -97,12 +103,13 @@ Commands:
       push        Push a pre-built image to a registry. Requires a repository address to be provided.
     
     Options:
    -  -f file     Dockerfile to build for JVM based Jobs. By default builds the Dockerfile shipped with Spark.
    -  -p file     Dockerfile with Python baked in. By default builds the Dockerfile shipped with Spark.
    -  -r repo     Repository address.
    -  -t tag      Tag to apply to the built image, or to identify the image to be pushed.
    -  -m          Use minikube's Docker daemon.
    -  -n          Build docker image with --no-cache
    +  -f file               Dockerfile to build for JVM based Jobs. By default builds the Dockerfile shipped with Spark.
    +  -p file               Dockerfile with Python baked in. By default builds the Dockerfile shipped with Spark.
    --- End diff --
    
    nit: Maybe for -p & -R clarify that it build's the Python or R docker file shipped with Spark since we are going to be shipping multiple ones and if someone goes and looks at the default one they might get confused about how this builds.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1313/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test status failure
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1123/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/281/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    looks like test failed
    
    ```
    - Run SparkR on simple dataframe.R example *** FAILED ***
      The code passed to eventually never returned normally. Attempted 70 times over 2.0071955115 minutes. Last failure message: Failure executing: GET at: https://192.168.39.28:8443/api/v1/namespaces/3346b714f192421eaae8d115154cec0e/pods/spark-test-app-da59d78dbe144e7e84581429199be99a/log?pretty=false. Message: container "spark-kubernetes-driver" in pod "spark-test-app-da59d78dbe144e7e84581429199be99a" is waiting to start: image can't be pulled. Received status: Status(apiVersion=v1, code=400, details=null, kind=Status, message=container "spark-kubernetes-driver" in pod "spark-test-app-da59d78dbe144e7e84581429199be99a" is waiting to start: image can't be pulled, metadata=ListMeta(resourceVersion=null, selfLink=null, additionalProperties={}), reason=BadRequest, status=Failure, additionalProperties={}).. (KubernetesSuite.scala:221)
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #92635 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92635/testReport)** for PR 21584 at commit [`a2f2825`](https://github.com/apache/spark/commit/a2f282555b8a30ee2a50448cb7045819a6f3d483).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    @felixcheung @mccheah by that logic.. should then both Python and R images be version configurable? This is all handled in the `bin/docker-image-tool.sh` ? Aren't those configurations just way to hard to deal with (i.e. make the build script very cumbersome with all the build-args... why can't they just create their own images with unique versions? i.e. 
    ```
    ARG VERSION
    apk-add R-base:$VERSION R-dev:$VERSION
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92034/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r204229713
  
    --- Diff: resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile ---
    @@ -0,0 +1,29 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You 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
    +#
    +#    http://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.
    +#
    +
    +ARG base_img
    +FROM $base_img
    +WORKDIR /
    +RUN mkdir ${SPARK_HOME}/R
    +COPY R ${SPARK_HOME}/R
    +
    +RUN apk add --no-cache R R-dev
    +
    --- End diff --
    
    add `rm -r /root/.cache` like in python?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1201/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    We have a min supported version of R
    https://github.com/apache/spark/blob/master/R/pkg/DESCRIPTION
    
    But beyond that we haven’t seen much problems when new releases of R are coming up.
    
    It might be helpful to make it configurable if any user has any particular need with R and/or R packages versio



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93547 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93547/testReport)** for PR 21584 at commit [`d036673`](https://github.com/apache/spark/commit/d0366732a9ebef710cb78e52ee42a869002e9040).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r205852075
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala ---
    @@ -205,7 +218,7 @@ private[spark] object Config extends Logging {
           .createWithDefault(0.1)
     
       val PYSPARK_MAJOR_PYTHON_VERSION =
    -    ConfigBuilder("spark.kubernetes.pyspark.pythonversion")
    +    ConfigBuilder("spark.kubernetes.pyspark.pythonVersion")
    --- End diff --
    
    nit / curiousity: why are we changing this?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93802 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93802/testReport)** for PR 21584 at commit [`bd60d05`](https://github.com/apache/spark/commit/bd60d05aa5df3aa67cfd62084ba00dd26d1bf67c).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93273 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93273/testReport)** for PR 21584 at commit [`5bdbc02`](https://github.com/apache/spark/commit/5bdbc026cc5081afe7fb50ce9082b40e3786c5d4).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #92034 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92034/testReport)** for PR 21584 at commit [`884a4e7`](https://github.com/apache/spark/commit/884a4e71f25dd8276cd198ad8c3deb91e61634d3).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1122/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r204229518
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -79,11 +80,16 @@ function build {
       docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \
         -t $(image_ref spark-py) \
         -f "$PYDOCKERFILE" .
    +
    +    docker build "${BINDING_BUILD_ARGS[@]}" \
    --- End diff --
    
    is should `$NOCACHEARG` too?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1498/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1401/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    @felixcheung for initial review 
    There is still a need for the integration testing environment to be setup on the PRB side. But I would like community opinion on the format of the Dockerfile. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93386 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93386/testReport)** for PR 21584 at commit [`1e19374`](https://github.com/apache/spark/commit/1e1937492d16c84d5b3bcba61b55cc718c87a458).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93294 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93294/testReport)** for PR 21584 at commit [`d809f67`](https://github.com/apache/spark/commit/d809f67357afb955c5f398a9ada6523ed43c7f99).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1498/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by shaneknapp <gi...@git.apache.org>.
Github user shaneknapp commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    @felixcheung -- SPARK-24908 is just one of many layers of the onion that i'm having to peel away to get the GHPRB ported to ubuntu.  we're close to getting it to reliably pass, but not quite there yet.  there's another potential bug w/some scala unit tests that i will be creating an issue for today.
    
    once i get the GHPRB working reliably enough for us to deem it safe enough to move to ubuntu, we'll need to start wiping the existing centos workers and reinstalling them w/ubuntu.  this will definitely take time...  and i'm guessing at the minimum a couple of weeks for this to happen.
    
    i'm doing my best to get this done but there's no way that we'll get things finished before 2.4 is cut.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test status failure
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1138/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93272 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93272/testReport)** for PR 21584 at commit [`2d55f06`](https://github.com/apache/spark/commit/2d55f06c0c82a53c994c960965132776dd136a72).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/2256/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94862/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by shaneknapp <gi...@git.apache.org>.
Github user shaneknapp commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    LGTM
    
    On Fri, Aug 17, 2018 at 12:16 AM, Felix Cheung <no...@github.com>
    wrote:
    
    > LGTM
    >
    > —
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub
    > <https://github.com/apache/spark/pull/21584#issuecomment-413779297>, or mute
    > the thread
    > <https://github.com/notifications/unsubscribe-auth/ABiDrNapEtXAPlZ123UtX5viDA9ymGIuks5uRm27gaJpZM4UsEQR>
    > .
    >



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR ...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r205354798
  
    --- Diff: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala ---
    @@ -71,8 +75,9 @@ private[spark] class KubernetesDriverBuilder(
             case JavaMainAppResource(_) =>
               provideJavaStep(kubernetesConf)
             case PythonMainAppResource(_) =>
    -          providePythonStep(kubernetesConf)}
    -      .getOrElse(provideJavaStep(kubernetesConf))
    +          providePythonStep(kubernetesConf)
    +        case RMainAppResource(_) =>
    +          provideRStep(kubernetesConf)}.getOrElse(provideJavaStep(kubernetesConf))
    --- End diff --
    
    btw, this wasn't move?
    ```
            case RMainAppResource(_) =>
              provideRStep(kubernetesConf)}
        .getOrElse(provideJavaStep(kubernetesConf))
    ```
    
    I meant it like this


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93664/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93662 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93662/testReport)** for PR 21584 at commit [`131f11f`](https://github.com/apache/spark/commit/131f11f8deb96fa7fa4f78522b73e5bbf2b9345e).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test status failure
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/1201/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for Sp...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21584#discussion_r196237071
  
    --- Diff: resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile ---
    @@ -0,0 +1,29 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You 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
    +#
    +#    http://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.
    +#
    +
    +ARG base_img
    +FROM $base_img
    +WORKDIR /
    +RUN mkdir ${SPARK_HOME}/R
    +COPY R ${SPARK_HOME}/R
    +
    +RUN apk add --no-cache R R-dev
    --- End diff --
    
    Should we install a specific version of R here? Perhaps allow it to be parameterized as a build argument with some default.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93274 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93274/testReport)** for PR 21584 at commit [`24fcd9b`](https://github.com/apache/spark/commit/24fcd9b9f3fce302be2088a1b3577cbddc0bdf73).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1401/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93802/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    @felixcheung The integration tests already run `docker-image-tool.sh -t testing build` which build the `spark-base`, `spark-py`, and `spark-r` images. At which point integration tests would be run from the `KubernetesSuite`. As such, is this okay as an initial PR to get SparkR support, (in its current state) after integration tests are included in this PR? 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by ifilonenko <gi...@git.apache.org>.
Github user ifilonenko commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Ready for merge upon PRB passing the tests. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    SPARK-24908 went in, it should unblock R?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Kubernetes integration test starting
    URL: https://amplab.cs.berkeley.edu/jenkins/job/testing-k8s-prb-make-spark-distribution-unified/281/



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93386 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93386/testReport)** for PR 21584 at commit [`1e19374`](https://github.com/apache/spark/commit/1e1937492d16c84d5b3bcba61b55cc718c87a458).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    **[Test build #93664 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93664/testReport)** for PR 21584 at commit [`1f0cba5`](https://github.com/apache/spark/commit/1f0cba59b650e4458e9472933068928d52a54777).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait BarrierTaskContext extends TaskContext `
      * `class BarrierTaskInfo(val address: String)`
      * `class RDDBarrier[T: ClassTag](rdd: RDD[T]) `
      * `case class WorkerOffer(`
      * `trait AnalysisHelper extends QueryPlan[LogicalPlan] `


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21584: [SPARK-24433][K8S][WIP] Initial R Bindings for SparkR on...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21584
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/695/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org