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/10/02 07:49:21 UTC

[GitHub] spark pull request #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

GitHub user ifilonenko opened a pull request:

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

    [SPARK-23257][K8S][TESTS] Kerberos Support Integration Tests

    ## What changes were proposed in this pull request?
    
    This fix includes just the integration tests for Kerberos Support
    
    ## How was this patch tested?
    
    This patch includes a single-noded pseudo-distributed Kerberized Hadoop cluster for the purpose of testing Kerberos interaction. The Keytabs are shared with Persistent Volumes and communication happens all within the same Kubernetes cluster. 


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

    $ git pull https://github.com/ifilonenko/spark SPARK-25152-e2e-tests

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

    https://github.com/apache/spark/pull/22608.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 #22608
    
----
commit 31fc536103538543cd7e114cf737b2712cfec15c
Author: Ilan Filonenko <if...@...>
Date:   2018-09-27T00:41:21Z

    initial commit

commit 9bfa86a947b4ff764762fe27b356480a6e957baa
Author: Ilan Filonenko <if...@...>
Date:   2018-09-29T01:55:31Z

    initial work on secure-hdfs integration testing

commit 77ea92a0c1303f7b4c7dd4a6131e49e691b19b84
Author: Ilan Filonenko <if...@...>
Date:   2018-09-29T02:06:42Z

    small fix

commit 761254c3d4bdd1b35e707077acf0a70defc88ea9
Author: Ilan Filonenko <if...@...>
Date:   2018-09-29T03:02:11Z

    fixed issue of docker building

commit 6e3966fbc98809a962bd9cbd589266d9b8b95834
Author: Ilan Filonenko <if...@...>
Date:   2018-09-29T03:04:16Z

    fixes and organizations

commit 776617dc5328a7a88afde854240d750efd52959f
Author: Ilan Filonenko <if...@...>
Date:   2018-09-29T17:42:42Z

    traits and polymorphosim

commit 7f1ccb6451d53f04d46263f7bb7e81211bfb809f
Author: Ilan Filonenko <if...@...>
Date:   2018-09-30T07:39:19Z

    polymorphism fixes and generuc class types

commit 3ab4358787e5cfb0de289f963122b7f22108fc36
Author: Ilan Filonenko <if...@...>
Date:   2018-10-01T05:28:03Z

    working test cases (just need clusterrolebindings)

commit cfe799033139251df44e584ab06b699cb437ed11
Author: Ilan Filonenko <if...@...>
Date:   2018-10-02T07:40:28Z

    small changes with addition of old tests

commit 54316ba4fbc5ec7b46184d01f6404bd26d3c0f5d
Author: Ilan Filonenko <if...@...>
Date:   2018-10-02T07:48:26Z

    bring back sparkr

----


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229394011
  
    --- Diff: resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml ---
    @@ -0,0 +1,49 @@
    +#
    +# 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.
    +#
    +apiVersion: apps/v1
    +kind: Deployment
    --- End diff --
    
    Yeah, but I am aware that it fits better as a `Job`. (The `hdfs -cp` that is being run is the same file, so if it is re-run it doesn't do any harm.) But if it is a preference to use `Job` I don't see any problems with that as it makes sense


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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/4422/
    Test PASSed.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229523966
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosPVWatcherCache.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.deploy.k8s.integrationtest.kerberos
    +
    +import io.fabric8.kubernetes.api.model.{PersistentVolume, PersistentVolumeClaim}
    +import io.fabric8.kubernetes.client.{KubernetesClientException, Watch, Watcher}
    +import io.fabric8.kubernetes.client.Watcher.Action
    +import org.scalatest.Matchers
    +import org.scalatest.concurrent.Eventually
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite.{INTERVAL, TIMEOUT}
    +import org.apache.spark.internal.Logging
    +
    +/**
    + * This class is responsible for ensuring that the persistent volume claims are bounded
    + * to the correct persistent volume and that they are both created before launching the
    --- End diff --
    
    With `StatefulSet`s, you probably don't need this.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    Although this is a large patch, its impact on existing code is small, and it is nearly all testing code. Unless the tests themselves are unstable, I'd consider this plausible to include with the 2.4 release.


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    > calling an external docker-image like: ifilonenko/hadoop-base:latest for now
    
    for now it's probably ok, but is there a solution before the next release?


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    Just noticed this, but could you open a separate bug for adding these tests, instead of re-using the one where the main code was added? It's a large enough thing that it should be a separate thing.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #98109 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98109/testReport)** for PR 22608 at commit [`b0696da`](https://github.com/apache/spark/commit/b0696dac82762d4f3c1a4c296d3d0f580be04989).
     * 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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    I think the suggestion is to go to master & 2.4


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229372458
  
    --- Diff: resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml ---
    @@ -0,0 +1,33 @@
    +#
    +# 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.
    +#
    +apiVersion: v1
    +kind: Service
    --- End diff --
    
    Why do you need a service for the data populator?


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    @vanzin as per our conversation offline, that is exactly what my current refactor is doing. I will push an update really soon. Thanks for the recommendation!


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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/4050/
    Test PASSed.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    > You seem to be running different pods for KDC, NN and DN. Is there an advantage to that?
    > 
    > Seems to me you could do the same thing with a single pod and simplify things here.
    > 
    > The it README also mentions "3 CPUs and 4G of memory". Is that still enough with these new things that are run?
    
    Think we want different images for each, but that's fine - just run a pod with those three containers in it.


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #97418 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97418/testReport)** for PR 22608 at commit [`cccf027`](https://github.com/apache/spark/commit/cccf0275cc58b464aba544742d7300ba4939f5a6).
     * 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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229372327
  
    --- Diff: resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml ---
    @@ -0,0 +1,49 @@
    +#
    +# 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.
    +#
    +apiVersion: apps/v1
    +kind: Deployment
    --- End diff --
    
    Why Deployment? Should it be a `Job` instead as it runs to completion?


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r230109287
  
    --- Diff: resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-set.yml ---
    @@ -0,0 +1,49 @@
    +#
    +# 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.
    +#
    +apiVersion: apps/v1
    +kind: StatefulSet
    +metadata:
    +  name: kerberos
    +spec:
    +  replicas: 1
    +  selector:
    +    matchLabels:
    +      name: hdfs-kerberos
    +      kerberosService: kerberos
    +      job: kerberostest
    +  template:
    +    metadata:
    +      annotations:
    +        pod.beta.kubernetes.io/hostname: kerberos
    +      labels:
    +        name: hdfs-kerberos
    +        kerberosService: kerberos
    +        job: kerberostest
    +    spec:
    +      containers:
    +      - command: ["sh"]
    +        args: ["/start-kdc.sh"]
    +        name: kerberos
    +        imagePullPolicy: IfNotPresent
    +        volumeMounts:
    +        - mountPath: /var/keytabs
    +          name: kerb-keytab
    +      restartPolicy: Always
    +      volumes:
    +      - name: kerb-keytab
    +        persistentVolumeClaim:
    --- End diff --
    
    +1


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #97421 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97421/testReport)** for PR 22608 at commit [`cccf027`](https://github.com/apache/spark/commit/cccf0275cc58b464aba544742d7300ba4939f5a6).
     * 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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    re: hadoop-2.7.3.tgz is that something Shane needs to install on the testing infra, to build the images you want?


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97478/
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r230255203
  
    --- Diff: resource-managers/kubernetes/docker/src/test/hadoop/conf/yarn-site.xml ---
    @@ -0,0 +1,26 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
    +<!--
    +  Licensed under the Apache License, Version 2.0 (the "License");
    +  you may not use this file except in compliance with the License.
    +  You may obtain a copy of the License at
    +
    +    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. See accompanying LICENSE file.
    +-->
    +
    +<!-- Put site-specific property overrides in this file. -->
    +
    +<configuration>
    +  <!-- must be set for HDFS libraries to obtain delegation tokens -->
    --- End diff --
    
    You could put this in hdfs-site.xml and avoid having to deal with this extra file.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    @mccheah @liyinan926 @erikerlandson for review
    
    Things to note: 
    - [ ] `clusterrolebindings` might be needed to ensure driver can setup necessary resources. 
    - [ ] Any way to include the hadoop-2.7.3.tgz so that the `hadoop-base:latest` image can be built on the fly as opposed to pulling from `ifilonenko/hadoop-base:latest`
    



---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    > for now it's probably ok, but is there a solution before the next release?
    
    This integration-test suite works seemlessly and is quite robust when rebased on-top of the Kerberos PR. So if we leave this PR as is, it should be good for merge. Pulling from `ifilonenko/hadoop-base:latest` makes it soooo much easier :)


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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/4551/
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225611762
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    +  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/R/Dockerfile"}
     
    +  # Spark Base
       docker build $NOCACHEARG "${BUILD_ARGS[@]}" \
         -t $(image_ref spark) \
         -f "$BASEDOCKERFILE" .
     
    +  # PySpark
       docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \
         -t $(image_ref spark-py) \
         -f "$PYDOCKERFILE" .
     
    +  # The following are optional docker builds for Kerberos Testing
    +  docker pull ifilonenko/hadoop-base:latest
    --- End diff --
    
    What about using `docker build` here instead?


---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229371556
  
    --- Diff: resource-managers/kubernetes/docker/src/test/scripts/populate-data.sh ---
    @@ -0,0 +1,42 @@
    +#!/usr/bin/env bash
    +#
    +# 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.
    +#
    +export JAVA_HOME=/usr/lib/jvm/jre-1.8.0-openjdk
    +export PATH=/hadoop/bin:$PATH
    +export HADOOP_CONF_DIR=/hadoop/etc/hadoop
    +export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true -Dsun.security.krb5.debug=true ${HADOOP_OPTS}"
    +export KRB5CCNAME=KRBCONF
    +mkdir -p /hadoop/etc/data
    +cp ${TMP_KRB_LOC} /etc/krb5.conf
    +cp ${TMP_CORE_LOC} /hadoop/etc/hadoop/core-site.xml
    +cp ${TMP_HDFS_LOC} /hadoop/etc/hadoop/hdfs-site.xml
    +
    +until kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.cluster.local; do sleep 2; done
    +
    +until (echo > /dev/tcp/nn.${NAMESPACE}.svc.cluster.local/9000) >/dev/null 2>&1; do sleep 2; done
    +
    +hdfs dfsadmin -safemode wait
    +
    +
    +hdfs dfs -mkdir -p /user/ifilonenko/
    --- End diff --
    
    Please no personal identifier in the path.


---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225714964
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    --- End diff --
    
    > Yes, but I don't think it's meant for also building images for tests.
    
    I don't see why not. A simple flag would toggle it's build.,would that suffice? Having it be all in this script is cleaner given that the `image_ref` is defined in `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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    It depends on how we're getting the Hadoop images. If we're building everything from scratch, we could run everything in one container - though having a container run more than one process simultaneously isn't common. It's more common to have a single container have a single responsibility / process. But you can group multiple containers that have related responsibilities into a single pod, hence we'll use 3 containers in one pod here.
    
    If we're pulling Hadoop images from elsewhere - which it sounds like we aren't doing in the Apache ecosystem in general though - then we'd need to build our own separate image for the KDC anyways.
    
    Multiple containers in the same pod all share the same resource footprint and limit boundaries.


---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229386227
  
    --- Diff: resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml ---
    @@ -0,0 +1,49 @@
    +#
    +# 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.
    +#
    +apiVersion: apps/v1
    +kind: Deployment
    --- End diff --
    
    This is purely because the job sometimes fails due to networking issues or whatnot and I wanted to counter the flakiness of the test suite by hardening it behind a deployment


---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r230228691
  
    --- Diff: resource-managers/kubernetes/docker/src/test/scripts/run-kerberos-test.sh ---
    @@ -0,0 +1,40 @@
    +#!/usr/bin/env bash
    +#
    +# 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.
    +#
    +sed -i -e 's/#//' -e 's/default_ccache_name/# default_ccache_name/' /etc/krb5.conf
    +export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true -Dsun.security.krb5.debug=true"
    +export HADOOP_JAAS_DEBUG=true
    +export HADOOP_ROOT_LOGGER=DEBUG,console
    +cp ${TMP_KRB_LOC} /etc/krb5.conf
    +cp ${TMP_CORE_LOC} /opt/spark/hconf/core-site.xml
    +cp ${TMP_HDFS_LOC} /opt/spark/hconf/hdfs-site.xml
    +mkdir -p /etc/krb5.conf.d
    +/opt/spark/bin/spark-submit \
    +      --deploy-mode cluster \
    +      --class ${CLASS_NAME} \
    +      --master k8s://${MASTER_URL} \
    +      --conf spark.kubernetes.namespace=${NAMESPACE} \
    +      --conf spark.executor.instances=1 \
    +      --conf spark.app.name=spark-hdfs \
    +      --conf spark.driver.extraClassPath=/opt/spark/hconf/core-site.xml:/opt/spark/hconf/hdfs-site.xml:/opt/spark/hconf/yarn-site.xml:/etc/krb5.conf \
    --- End diff --
    
    Adding files to the classpath does not do anything.
    
    ```
    $ scala -cp /etc/krb5.conf
    scala> getClass().getResource("/krb5.conf")
    res0: java.net.URL = null
    
    $ scala -cp /etc
    scala> getClass().getResource("/krb5.conf")
    res0: java.net.URL = file:/etc/krb5.conf
    ```
    
    So this seems not needed. Also because I'd expect spark-submit or the k8s backend code to add the hadoop conf to the driver's classpath somehow.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    > Kubernetes integration test status success
    
    This new build passes and because I resolved the docker image, by building the `hadoop-base` image with each iteration, this PR is now ready for review and hopefully merge soooon :) 
    
    @vanzin @liyinan926 @felixcheung @mccheah for review


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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/4008/
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225619385
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    +  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/R/Dockerfile"}
     
    +  # Spark Base
       docker build $NOCACHEARG "${BUILD_ARGS[@]}" \
         -t $(image_ref spark) \
         -f "$BASEDOCKERFILE" .
     
    +  # PySpark
       docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \
         -t $(image_ref spark-py) \
         -f "$PYDOCKERFILE" .
     
    +  # The following are optional docker builds for Kerberos Testing
    +  docker pull ifilonenko/hadoop-base:latest
    --- End diff --
    
    In fact, `FROM` has the same problem here. What I mean was including the whole docker file from `ifilonenko/hadoop-base`. Although this is a test script, I was surprised that Apache Spark depends on a personal repo officially.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #96845 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96845/testReport)** for PR 22608 at commit [`54316ba`](https://github.com/apache/spark/commit/54316ba4fbc5ec7b46184d01f6404bd26d3c0f5d).


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #96854 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96854/testReport)** for PR 22608 at commit [`56e2c6e`](https://github.com/apache/spark/commit/56e2c6e20b427c883e330d79f45ef6f3841cd518).


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229373633
  
    --- Diff: resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml ---
    @@ -0,0 +1,49 @@
    +#
    +# 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.
    +#
    +apiVersion: apps/v1
    +kind: Deployment
    --- End diff --
    
    Ditto, use a StatefulSet of size 1, which is able to automatically provision a PV for the PVC.


---

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


[GitHub] spark pull request #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225625939
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    +  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/R/Dockerfile"}
     
    +  # Spark Base
       docker build $NOCACHEARG "${BUILD_ARGS[@]}" \
         -t $(image_ref spark) \
         -f "$BASEDOCKERFILE" .
     
    +  # PySpark
       docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \
         -t $(image_ref spark-py) \
         -f "$PYDOCKERFILE" .
     
    +  # The following are optional docker builds for Kerberos Testing
    +  docker pull ifilonenko/hadoop-base:latest
    --- End diff --
    
    I had also discussed concerns re: pulling a "personal" image w/ @ifilonenko - I'd prefer some other solution here. Docker build seems plausible, but it would be best if it can be done once per test run for efficiency reasons


---

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


[GitHub] spark pull request #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225631577
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    --- End diff --
    
    It is a docker image builder, so it seems to be an appropriate place. Especially, since it’s the first stage in our integration tests. But am open to opinions. 


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #97416 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97416/testReport)** for PR 22608 at commit [`436f652`](https://github.com/apache/spark/commit/436f652159b257e3f362118cf0f2d73bd77cd328).


---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229523838
  
    --- Diff: resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-set.yml ---
    @@ -0,0 +1,49 @@
    +#
    +# 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.
    +#
    +apiVersion: apps/v1
    +kind: StatefulSet
    +metadata:
    +  name: kerberos
    +spec:
    +  replicas: 1
    +  selector:
    +    matchLabels:
    +      name: hdfs-kerberos
    +      kerberosService: kerberos
    +      job: kerberostest
    +  template:
    +    metadata:
    +      annotations:
    +        pod.beta.kubernetes.io/hostname: kerberos
    +      labels:
    +        name: hdfs-kerberos
    +        kerberosService: kerberos
    +        job: kerberostest
    +    spec:
    +      containers:
    +      - command: ["sh"]
    +        args: ["/start-kdc.sh"]
    +        name: kerberos
    +        imagePullPolicy: IfNotPresent
    +        volumeMounts:
    +        - mountPath: /var/keytabs
    +          name: kerb-keytab
    +      restartPolicy: Always
    +      volumes:
    +      - name: kerb-keytab
    +        persistentVolumeClaim:
    --- End diff --
    
    With a `StatefulSet`, you don't need to explicitly manage PVCs. You can use `.spec.persistentVolumeClaimTemplate`. The StatefulSet controller automatically creates the PV (or binds to the existing one it created before).


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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/4011/
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225718039
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    --- End diff --
    
    At the minimum, it might confuse people because people who check this script will possibly see there's now a `spark-keberos`  image without even noticing that the Dockerfile is meant for **tests**.  Given that we have the `resource-manager/kubernetes/integration-tests` directory, why not put test-specific Dockerfile and scripts under that directory?


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #97949 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97949/testReport)** for PR 22608 at commit [`51959b2`](https://github.com/apache/spark/commit/51959b22cfdb4606260aa516c41e0d3f6eba56ce).
     * 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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r230227288
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -53,7 +53,7 @@ function build {
         # contain a lot of duplicated jars with the main Spark directory. In a proper distribution,
         # the examples directory is cleaned up before generating the distribution tarball, so this
         # issue does not occur.
    -    IMG_PATH=resource-managers/kubernetes/docker/src/main/dockerfiles
    +    IMG_PATH=resource-managers/kubernetes/docker/src
    --- End diff --
    
    I have the same question. It doesn't seem like you're actually using this script for the new test stuff, nor changing any of the existing calls to it, so do you need any of the changes being made here?


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #98053 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98053/testReport)** for PR 22608 at commit [`66fe408`](https://github.com/apache/spark/commit/66fe40830d9f732b7bd2bb5e1ebf81713d201751).


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #96845 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96845/testReport)** for PR 22608 at commit [`54316ba`](https://github.com/apache/spark/commit/54316ba4fbc5ec7b46184d01f6404bd26d3c0f5d).
     * This patch **fails RAT 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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    @mccheah and @erikerlandson for review as the 
    `Secure HDFS test with HDFS keytab (Cluster Mode)`
    passes and should be merged to enable Secure HDFS interaction.


---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229370867
  
    --- Diff: dev/make-distribution.sh ---
    @@ -191,7 +191,8 @@ fi
     # Only create and copy the dockerfiles directory if the kubernetes artifacts were built.
     if [ -d "$SPARK_HOME"/resource-managers/kubernetes/core/target/ ]; then
       mkdir -p "$DISTDIR/kubernetes/"
    -  cp -a "$SPARK_HOME"/resource-managers/kubernetes/docker/src/main/dockerfiles "$DISTDIR/kubernetes/"
    +  cp -a "$SPARK_HOME"/resource-managers/kubernetes/docker/src "$DISTDIR/kubernetes/"
    --- End diff --
    
    Ditto. Why is this change still needed?


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #97416 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97416/testReport)** for PR 22608 at commit [`436f652`](https://github.com/apache/spark/commit/436f652159b257e3f362118cf0f2d73bd77cd328).
     * This patch **fails Scala style 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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r230109316
  
    --- Diff: resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosPVWatcherCache.scala ---
    @@ -0,0 +1,119 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.deploy.k8s.integrationtest.kerberos
    +
    +import io.fabric8.kubernetes.api.model.{PersistentVolume, PersistentVolumeClaim}
    +import io.fabric8.kubernetes.client.{KubernetesClientException, Watch, Watcher}
    +import io.fabric8.kubernetes.client.Watcher.Action
    +import org.scalatest.Matchers
    +import org.scalatest.concurrent.Eventually
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite.{INTERVAL, TIMEOUT}
    +import org.apache.spark.internal.Logging
    +
    +/**
    + * This class is responsible for ensuring that the persistent volume claims are bounded
    + * to the correct persistent volume and that they are both created before launching the
    --- End diff --
    
    +1


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    @ifilonenko  can we work with the existing service-account-name config parameters for obtaining the resource permissions?


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    > Just noticed this, but could you open a separate bug for adding these tests, instead of re-using the one where the main code was added? It's a large enough thing that it should be a separate thing.
    
    I had https://issues.apache.org/jira/browse/SPARK-25750 and linked this PR to that JIRA issue.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229373938
  
    --- Diff: resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml ---
    @@ -0,0 +1,54 @@
    +#
    +# 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.
    +#
    +apiVersion: apps/v1
    +kind: Deployment
    --- End diff --
    
    Ditto. Use a StatefulSet of size 1.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #97949 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97949/testReport)** for PR 22608 at commit [`51959b2`](https://github.com/apache/spark/commit/51959b22cfdb4606260aa516c41e0d3f6eba56ce).


---

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


[GitHub] spark pull request #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225630500
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    +  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/R/Dockerfile"}
     
    +  # Spark Base
       docker build $NOCACHEARG "${BUILD_ARGS[@]}" \
         -t $(image_ref spark) \
         -f "$BASEDOCKERFILE" .
     
    +  # PySpark
       docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \
         -t $(image_ref spark-py) \
         -f "$PYDOCKERFILE" .
     
    +  # The following are optional docker builds for Kerberos Testing
    +  docker pull ifilonenko/hadoop-base:latest
    --- End diff --
    
    The strategy of using this “personal” image is purely because the HDFS nodes: (kdc, nn, dn) themselves require hadoop-2.7.3.tgz. Unless that can be packaged in the distribution, in which case it would be trivial to build, pulling is easier. I am open for opinions on building strategies. The dockerfile in question can be found here: https://github.com/ifilonenko/hadoop-kerberos-helm/blob/master/Dockerfile 


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    gentle ping for further reviews :) as I would love to have this merged in the Jenkins :) as well as recommendations on how to include the `hadoop-XXXX.tgz` in the distribution, for building the `hadoop-base` docker image, as that is crucial for the hadoop cluster.


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97418/
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225611252
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    +  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/R/Dockerfile"}
     
    +  # Spark Base
       docker build $NOCACHEARG "${BUILD_ARGS[@]}" \
         -t $(image_ref spark) \
         -f "$BASEDOCKERFILE" .
     
    +  # PySpark
       docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \
         -t $(image_ref spark-py) \
         -f "$PYDOCKERFILE" .
     
    +  # The following are optional docker builds for Kerberos Testing
    +  docker pull ifilonenko/hadoop-base:latest
    --- End diff --
    
    Ur, this looks a little bit too personal. is this okay for Apache Spark?


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    @erikerlandson the clusterrolebinding is something the user testing should set up. As such, we may disregard that bullet-point from the conversation. However, I am wondering what are thoughts of calling an external docker-image like: `ifilonenko/hadoop-base:latest` for now? This would just require for the hadoop-base image to be built in the docker-image-builder and for the distribution to contain the `hadoop-2.7.3.tgz` file for the image to build. 
    
    > Although this is a large patch, its impact on existing code is small, and it is nearly all testing code. Unless the tests themselves are unstable, I'd consider this plausible to include with the 2.4 release.
    
    Very true, this feature is very isolated and was designed to be extremely stable (via the WatcherCaches), but should only be merged with https://github.com/apache/spark/pull/21669. Would like a review on the design so that we may merge this in ASAP when the above PR is merged as they are completely isolated.



---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    I haven't reviewed this (and I do have concerns about using an external image in the long term), just wanted to perhaps give some ideas for the future.
    
    Livy does integration tests using some helper code that runs MiniDFSCluster and MiniYARNCluster:
    https://github.com/apache/incubator-livy/blob/master/integration-test/src/main/scala/org/apache/livy/test/framework/MiniCluster.scala
    
    (It doesn't use MiniKdc yet.)
    
    It should not be hard to create a docker image with all the needed Hadoop dependencies using maven-dependency-plugin:copy-dependencies + a shell script, and have some simple code like Livy's to bring up the servers.


---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229391389
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -53,7 +53,7 @@ function build {
         # contain a lot of duplicated jars with the main Spark directory. In a proper distribution,
         # the examples directory is cleaned up before generating the distribution tarball, so this
         # issue does not occur.
    -    IMG_PATH=resource-managers/kubernetes/docker/src/main/dockerfiles
    +    IMG_PATH=resource-managers/kubernetes/docker/src
    --- End diff --
    
    The dockerfiles and files for building the kerberos/ hadoop docker images are in `src/test`. It still seemed like a logical place to keep them with the `/test` tag, no? 


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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/4510/
    Test PASSed.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #97478 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97478/testReport)** for PR 22608 at commit [`4c9b886`](https://github.com/apache/spark/commit/4c9b886c1f23bbdd3d8e1ec7df25f03e45892d88).


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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/4640/
    Test PASSed.


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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/3618/
    Test FAILed.


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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/3627/
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229373205
  
    --- Diff: resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml ---
    @@ -0,0 +1,49 @@
    +#
    +# 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.
    +#
    +apiVersion: apps/v1
    +kind: Deployment
    --- End diff --
    
    Use a StatefulSet of size 1 as Deployment has no guarantee of exactly once semantics.


---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229389845
  
    --- Diff: resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml ---
    @@ -0,0 +1,49 @@
    +#
    +# 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.
    +#
    +apiVersion: apps/v1
    +kind: Deployment
    --- End diff --
    
    How does a Deployment helps with that, restarting of failed pod? 


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #97477 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97477/testReport)** for PR 22608 at commit [`5d270f1`](https://github.com/apache/spark/commit/5d270f17dccbb2eac6d3c2ab8c12987e3d992086).
     * 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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225615864
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    +  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/R/Dockerfile"}
     
    +  # Spark Base
       docker build $NOCACHEARG "${BUILD_ARGS[@]}" \
         -t $(image_ref spark) \
         -f "$BASEDOCKERFILE" .
     
    +  # PySpark
       docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \
         -t $(image_ref spark-py) \
         -f "$PYDOCKERFILE" .
     
    +  # The following are optional docker builds for Kerberos Testing
    +  docker pull ifilonenko/hadoop-base:latest
    --- End diff --
    
    I comment on this earlier... https://github.com/apache/spark/pull/22608#issuecomment-427617701


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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/4508/
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225617789
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    +  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/R/Dockerfile"}
     
    +  # Spark Base
       docker build $NOCACHEARG "${BUILD_ARGS[@]}" \
         -t $(image_ref spark) \
         -f "$BASEDOCKERFILE" .
     
    +  # PySpark
       docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \
         -t $(image_ref spark-py) \
         -f "$PYDOCKERFILE" .
     
    +  # The following are optional docker builds for Kerberos Testing
    +  docker pull ifilonenko/hadoop-base:latest
    --- End diff --
    
    Thanks. Got it, @felixcheung .


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r230227737
  
    --- Diff: dev/make-distribution.sh ---
    @@ -191,7 +191,8 @@ fi
     # Only create and copy the dockerfiles directory if the kubernetes artifacts were built.
     if [ -d "$SPARK_HOME"/resource-managers/kubernetes/core/target/ ]; then
       mkdir -p "$DISTDIR/kubernetes/"
    -  cp -a "$SPARK_HOME"/resource-managers/kubernetes/docker/src/main/dockerfiles "$DISTDIR/kubernetes/"
    +  cp -a "$SPARK_HOME"/resource-managers/kubernetes/docker/src "$DISTDIR/kubernetes/"
    +  cp -a "$SPARK_HOME"/resource-managers/kubernetes/integration-tests/scripts "$DISTDIR/kubernetes/"
    --- End diff --
    
    This is following the existing pattern in the line below; but is there a purpose in packaging these test artifacts with a binary Spark distribution?
    
    Seems to me like they should be left in the source package and that's it.


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    > Think we want different images for each
    
    You don't need to, right? You can have a single image with all the stuff needed. That would also make setting up the test faster (less images to build).
    
    > just run a pod with those three containers 
    
    That's mostly me still getting used to names here; to me pod == one container running with some stuff.
    
    But in any case, my main concern in this case is resource utilization - it we can keep things slimmer by running less containers, I think that's better. Individually, the NN, DN and the KDC don't need a lot of resources for this particular test to run.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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/4006/
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225634934
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    --- End diff --
    
    Yes, but I don't think it's meant for also building images for tests. Can you create a separate script under `resource-managers/kubernetes/docker/src/test/scripts`?


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #98053 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98053/testReport)** for PR 22608 at commit [`66fe408`](https://github.com/apache/spark/commit/66fe40830d9f732b7bd2bb5e1ebf81713d201751).
     * 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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229370525
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -53,7 +53,7 @@ function build {
         # contain a lot of duplicated jars with the main Spark directory. In a proper distribution,
         # the examples directory is cleaned up before generating the distribution tarball, so this
         # issue does not occur.
    -    IMG_PATH=resource-managers/kubernetes/docker/src/main/dockerfiles
    +    IMG_PATH=resource-managers/kubernetes/docker/src
    --- End diff --
    
    Do you still need changes to this file given you have moved the test stuffs out?


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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/4049/
    Test FAILed.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #97478 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97478/testReport)** for PR 22608 at commit [`4c9b886`](https://github.com/apache/spark/commit/4c9b886c1f23bbdd3d8e1ec7df25f03e45892d88).
     * 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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225614226
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    +  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/R/Dockerfile"}
     
    +  # Spark Base
       docker build $NOCACHEARG "${BUILD_ARGS[@]}" \
         -t $(image_ref spark) \
         -f "$BASEDOCKERFILE" .
     
    +  # PySpark
       docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \
         -t $(image_ref spark-py) \
         -f "$PYDOCKERFILE" .
     
    +  # The following are optional docker builds for Kerberos Testing
    +  docker pull ifilonenko/hadoop-base:latest
    --- End diff --
    
    Isn't the explicit pull unnecessary?  Trying to build the new Kerberos image which is `FROM` this will implicitly pull it anyway


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #98055 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98055/testReport)** for PR 22608 at commit [`0639099`](https://github.com/apache/spark/commit/0639099f35ad249fc3b1149f92cf2b3453aae47d).
     * This patch **fails Spark unit 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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    > I think the suggestion is to go to master & 2.4
    
    The integration tests depend on the Kerberos support for Spark on k8s, which was merged into the master but not branch-2.4. So this should only go into the master.


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96845/
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225620898
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    +  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/R/Dockerfile"}
     
    +  # Spark Base
       docker build $NOCACHEARG "${BUILD_ARGS[@]}" \
         -t $(image_ref spark) \
         -f "$BASEDOCKERFILE" .
     
    +  # PySpark
       docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \
         -t $(image_ref spark-py) \
         -f "$PYDOCKERFILE" .
     
    +  # The following are optional docker builds for Kerberos Testing
    +  docker pull ifilonenko/hadoop-base:latest
    --- End diff --
    
    @ifilonenko why do you need to pull that image here?


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #96854 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96854/testReport)** for PR 22608 at commit [`56e2c6e`](https://github.com/apache/spark/commit/56e2c6e20b427c883e330d79f45ef6f3841cd518).
     * This patch **fails Scala style 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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    **[Test build #98281 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98281/testReport)** for PR 22608 at commit [`0de8c87`](https://github.com/apache/spark/commit/0de8c87f971d9dccb681678faffe92231a1f0c38).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait UnevaluableAggregate extends DeclarativeAggregate `
      * `case class Average(child: Expression) extends DeclarativeAggregate with ImplicitCastInputTypes `
      * `case class Count(children: Seq[Expression]) extends DeclarativeAggregate `
      * `abstract class UnevaluableBooleanAggBase(arg: Expression)`
      * `case class EveryAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) `
      * `case class AnyAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) `
      * `case class SomeAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) `
      * `case class UnresolvedCatalystToExternalMap(`


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97421/
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r225618723
  
    --- Diff: bin/docker-image-tool.sh ---
    @@ -71,18 +71,29 @@ function build {
         --build-arg
         base_img=$(image_ref spark)
       )
    -  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"}
    -  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"}
    -  local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"}
    +  local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/Dockerfile"}
    +  local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/main/dockerfiles/spark/bindings/python/Dockerfile"}
    +  local KDOCKERFILE=${KDOCKERFILE:-"$IMG_PATH/test/dockerfiles/spark/kerberos/Dockerfile"}
    --- End diff --
    
    While, it's quite odd to add a Dockerfile here just for testing purpose.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark pull request #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r229373845
  
    --- Diff: resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml ---
    @@ -0,0 +1,43 @@
    +#
    +# 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.
    +#
    +apiVersion: apps/v1
    +kind: Deployment
    --- End diff --
    
    Ditto. A Job makes more sense for run to completion test.


---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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


[GitHub] spark issue #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integr...

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

    https://github.com/apache/spark/pull/22608#discussion_r230228286
  
    --- Diff: resource-managers/kubernetes/docker/src/test/scripts/populate-data.sh ---
    @@ -0,0 +1,39 @@
    +#!/usr/bin/env bash
    +#
    +# 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.
    +#
    +export JAVA_HOME=/usr/lib/jvm/jre-1.8.0-openjdk
    +export PATH=/hadoop/bin:$PATH
    +export HADOOP_CONF_DIR=/hadoop/etc/hadoop
    +export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true -Dsun.security.krb5.debug=true ${HADOOP_OPTS}"
    +export KRB5CCNAME=KRBCONF
    +mkdir -p /hadoop/etc/data
    +cp ${TMP_KRB_LOC} /etc/krb5.conf
    +cp ${TMP_CORE_LOC} /hadoop/etc/hadoop/core-site.xml
    +cp ${TMP_HDFS_LOC} /hadoop/etc/hadoop/hdfs-site.xml
    +
    +until kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.cluster.local; do sleep 2; done
    +
    +until (echo > /dev/tcp/nn.${NAMESPACE}.svc.cluster.local/9000) >/dev/null 2>&1; do sleep 2; done
    +
    +hdfs dfsadmin -safemode wait
    +
    +
    +hdfs dfs -mkdir -p /user/userone/
    +hdfs dfs -copyFromLocal /people.txt /user/userone
    +
    +hdfs dfs -chmod -R 755 /user/userone
    +hdfs dfs -chown -R ifilonenko /user/userone
    --- End diff --
    
    `ifilonenko`?


---

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


[GitHub] spark issue #22608: [SPARK-23257][K8S][TESTS] Kerberos Support Integration T...

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

    https://github.com/apache/spark/pull/22608
  
    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 #22608: [SPARK-25750][K8S][TESTS] Kerberos Support Integration T...

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

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



---

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