Repository: samza
Updated Branches:
  refs/heads/master de52eea07 -> 72daa81d4


SAMZA-928: add documentation for Kerberos support in Samza


Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/72daa81d
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/72daa81d
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/72daa81d

Branch: refs/heads/master
Commit: 72daa81d490216100e1b411e42966250e7306f60
Parents: de52eea
Author: Chen Song <[email protected]>
Authored: Tue Jun 21 15:28:04 2016 -0700
Committer: Yi Pan (Data Infrastructure) <[email protected]>
Committed: Tue Jun 21 15:28:04 2016 -0700

----------------------------------------------------------------------
 docs/learn/documentation/versioned/index.html   |  1 +
 .../versioned/jobs/configuration-table.html     | 38 ++++++++
 .../documentation/versioned/jobs/yarn-jobs.md   |  2 +-
 .../documentation/versioned/yarn/isolation.md   |  2 +-
 .../versioned/yarn/yarn-security.md             | 94 ++++++++++++++++++++
 5 files changed, 135 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/72daa81d/docs/learn/documentation/versioned/index.html
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/index.html 
b/docs/learn/documentation/versioned/index.html
index c2a7b29..1e79bd6 100644
--- a/docs/learn/documentation/versioned/index.html
+++ b/docs/learn/documentation/versioned/index.html
@@ -83,6 +83,7 @@ title: Documentation
   <li><a href="yarn/isolation.html">Isolation</a></li>
   <li><a href="yarn/yarn-host-affinity.html">Host Affinity & Yarn</a></li>
   <li><a href="hdfs/producer.html">Writing to HDFS</a></li>
+  <li><a href="hdfs/yarn-security.html">Yarn Security</a></li>
 <!-- TODO write yarn pages
   <li><a href="">Fault Tolerance</a></li>
   <li><a href="">Security</a></li>

http://git-wip-us.apache.org/repos/asf/samza/blob/72daa81d/docs/learn/documentation/versioned/jobs/configuration-table.html
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/jobs/configuration-table.html 
b/docs/learn/documentation/versioned/jobs/configuration-table.html
index ea8d5e9..77907eb 100644
--- a/docs/learn/documentation/versioned/jobs/configuration-table.html
+++ b/docs/learn/documentation/versioned/jobs/configuration-table.html
@@ -340,6 +340,18 @@
                         This setting controls if the job should fail(true) or 
just warn(false) in case the validation of checkpoint partition number fails. 
<br/> <b>CAUTION</b>: this configuration needs to be used w/ care. It should 
only be used as a work-around after the checkpoint has been auto-created with 
wrong number of partitions by mistake.
                     </td>
                 </tr>
+                <tr>
+                    <td class="property" 
id="job-security-manager-factory">job.security.manager.factory</td>
+                    <td class="default"></td>
+                    <td class="description">
+                        This is the factory class used to create the proper <a 
href="../api/javadocs/org/apache/samza/container/SecurityManager.html">SecurityManager</a>
 to handle security for Samza containers when running in a secure environment, 
such as Yarn with Kerberos eanbled.
+                        Samza ships with one security manager by default:
+                        <dl>
+                            
<dt><code>org.apache.samza.job.yarn.SamzaYarnSecurityManagerFactory</code></dt>
+                            <dd>Supports Samza containers to run properly in a 
Kerberos enabled Yarn cluster. Each Samza container, once started, will create 
a <a 
href="../api/javadocs/org/apache/samza/job/yarn/SamzaContainerSecurityManager.html">SamzaContainerSecurityManager</a>.
 SamzaContainerSecurityManager runs on its separate thread and update user's 
delegation tokens at the interval specified by <a 
href="#yarn-token-renewal-interval-seconds" 
class="property">yarn.token.renewal.interval.seconds</a>. See <a 
href="../yarn/yarn-security.html">Yarn Security</a> for details.</dd>
+                        </dl>
+                    </td>
+                </tr>
 
                 <tr>
                     <th colspan="3" class="section" id="task"><a 
href="../api/overview.html">Task configuration</a></th>
@@ -1514,6 +1526,32 @@
                 </tr>
 
                 <tr>
+                    <td class="property" 
id="yarn-kerberos-principal">yarn.kerberos.principal</td>
+                    <td class="default"></td>
+                    <td class="description">
+                        Principal the Samza job uses to authenticate itself 
into KDC, when running on a Kerberos enabled YARN cluster.
+                    </td>
+                </tr>
+
+                <tr>
+                    <td class="property" 
id="yarn-kerberos-keytab">yarn.kerberos.keytab</td>
+                    <td class="default"></td>
+                    <td class="description">
+                        The full path to the file containing keytab for the 
principal, specified by <a href="#yarn-kerberos-principal" 
class="property">yarn.kerberos.principal</a>.
+                        The keytab file is uploaded to the staging directory 
unique to each application on HDFS and the Application Master then uses the 
keytab and principal to
+                        periodically logs in to recreate the delegation tokens.
+                    </td>
+                </tr>
+
+                <tr>
+                    <td class="property" 
id="yarn-token-renewal-interval-seconds">yarn.token.renewal.interval.seconds</td>
+                    <td class="default"></td>
+                    <td class="description">
+                        The time interval by which the Application Master 
re-authenticates and renew the delegation tokens. This value should be smaller 
than the length of time a delegation token is valid on hadoop namenodes before 
expiration.
+                    </td>
+                </tr>
+
+                <tr>
                     <th colspan="3" class="section" id="metrics"><a 
href="../container/metrics.html">Metrics</a></th>
                 </tr>
 

http://git-wip-us.apache.org/repos/asf/samza/blob/72daa81d/docs/learn/documentation/versioned/jobs/yarn-jobs.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/jobs/yarn-jobs.md 
b/docs/learn/documentation/versioned/jobs/yarn-jobs.md
index 827cc14..d0e5991 100644
--- a/docs/learn/documentation/versioned/jobs/yarn-jobs.md
+++ b/docs/learn/documentation/versioned/jobs/yarn-jobs.md
@@ -19,7 +19,7 @@ title: YARN Jobs
    limitations under the License.
 -->
 
-When you define `job.factory.class=org.apache.samza.job.yarn.YarnJobFactory` 
in your job's configuration, Samza will use YARN to execute your job. The 
YarnJobFactory will use the HADOOP_YARN_HOME environment variable on the 
machine that run-job.sh is executed on to get the appropriate YARN 
configuration, which will define where the YARN resource manager is. The 
YarnJob will work with the resource manager to get your job started on the YARN 
cluster.
+When you define `job.factory.class=org.apache.samza.job.yarn.YarnJobFactory` 
in your job's configuration, Samza will use YARN to execute your job. The 
YarnJobFactory will use the `HADOOP_YARN_HOME` environment variable on the 
machine that run-job.sh is executed on to get the appropriate YARN 
configuration, which will define where the YARN resource manager is. The 
YarnJob will work with the resource manager to get your job started on the YARN 
cluster.
 
 If you want to use YARN to run your Samza job, you'll also need to define the 
location of your Samza job's package. For example, you might say:
 

http://git-wip-us.apache.org/repos/asf/samza/blob/72daa81d/docs/learn/documentation/versioned/yarn/isolation.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/yarn/isolation.md 
b/docs/learn/documentation/versioned/yarn/isolation.md
index 1eb3bf5..a76f6fd 100644
--- a/docs/learn/documentation/versioned/yarn/isolation.md
+++ b/docs/learn/documentation/versioned/yarn/isolation.md
@@ -43,4 +43,4 @@ If YARN is setup to use CGroups, then YARN will guarantee 
that a container will
 
 See [this blog 
post](http://riccomini.name/posts/hadoop/2013-06-14-yarn-with-cgroups/) for 
details on setting up YARN with CGroups.
 
-## [Security &raquo;](../operations/security.html)
+## [YARN Security &raquo;](../yarn/yarn-secuirty.html)

http://git-wip-us.apache.org/repos/asf/samza/blob/72daa81d/docs/learn/documentation/versioned/yarn/yarn-security.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/yarn/yarn-security.md 
b/docs/learn/documentation/versioned/yarn/yarn-security.md
new file mode 100644
index 0000000..848326b
--- /dev/null
+++ b/docs/learn/documentation/versioned/yarn/yarn-security.md
@@ -0,0 +1,94 @@
+---
+layout: page
+title: YARN Security
+--------------------
+<!--
+   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.
+-->
+
+You can run a Samza job on a secure YARN cluster. YARN uses Kerberos as its 
authentication and authorization mechanism. See [this 
article](https://hadoop.apache.org/docs/r2.7.2/hadoop-project-dist/hadoop-common/SecureMode.html)
 for details on Hadoop in secure mode.
+
+### Delegation token management strategy
+
+One of the challenges for long-lived application running on a secure YARN 
cluster is its token renewal strategy. Samza takes the following approach to 
manage token creation and renewal.
+
+1. Client running Samza app needs to kinit into KDC with his credentials and 
add the HDFS delegation tokens to the launcher context before submitting the 
application.
+
+2. Next, client prepares the local resources for the application as follows.
+2.1. First, it creates a staging directory on HDFS. This directory is only 
accessible by the running user and used to store resources required for 
Application Master (AM) and Containers.
+2.2. Client then adds the keytab as a local resource in the container launcher 
context for AM.
+2.3. Finally, it sends the corresponding principal and the path to the keytab 
file in the staging directory to the coordinator stream. Samza currently uses 
the staging directory to store both the keytab and refreshed tokens because the 
access to the directory is secured via Kerberos.
+
+3. Once the resource is allocated for the Application Master, the Node Manager 
will localizes app resources from HDFS using the HDFS delegation tokens in the 
launcher context. Same rule applies to Container localization too. 
+
+4. When Application Master starts, it localizes the keytab file into its 
working directory and reads the principal from the coordinator stream.
+
+5. The Application Master periodically re-authenticate itself with the given 
principal and keytab. In each iteration, it creates new delegation tokens and 
stores them in the given job specific staging directory on HDFS.
+
+6. Each running container will get new delegation tokens from the credentials 
file on HDFS before the current ones expire.
+
+7. Application Master and Containers don't communicate with each other for 
that matter. Each side proceeds independently by reading or writing the tokens 
on HDFS.
+
+By default, any HDFS delegation token has a maximum life of 7 days (configured 
by `dfs.namenode.delegation.token.max-lifetime` in hdfs-site.xml) and the token 
is normally renewed every 24 hours (configured by 
`dfs.namenode.delegation.token.renew-interval` in hdfs-site.xml). What if the 
Application Master dies and needs restarts after 7 days? The original HDFS 
delegation token stored in the launcher context will be invalid no matter what. 
Luckily, Samza can rely on Resource Manager to handle this scenario. See the 
Configuration section below for details.  
+
+### Components
+
+#### SecurityManager
+
+When ApplicationMaster starts, it spawns `SamzaAppMasterSecurityManager`, 
which runs on its separate thread. The `SamzaAppMasterSecurityManager` is 
responsible for periodically logging in through the given Kerberos keytab and 
regenerates the HDFS delegation tokens regularly. After each run, it writes new 
tokens on a pre-defined job specific directory on HDFS. The frequency of this 
process is determined by `yarn.token.renewal.interval.seconds`.
+
+Each container, upon start, runs a `SamzaContainerSecurityManager`. It reads 
from the credentials file on HDFS and refreshes its delegation tokens at the 
same interval.
+
+### Configuration
+
+1. For the Samza job, the following job configurations are required on a YARN 
cluster with security enabled.
+# Job
+job.security.manager.factory=org.apache.samza.job.yarn.SamzaYarnSecurityManagerFactory
+
+# YARN
+{% highlight properties %}
+yarn.kerberos.principal=user/localhost
+yarn.kerberos.keytab=/etc/krb5.keytab.user
+yarn.token.renewal.interval.seconds=86400
+{% endhighlight %}
+
+2. Configure the Hadoop cluster to enable Resource Manager to recreate and 
renew the delegation token on behalf of the application user. This will address 
the following 2 scenarios.
+
+    * When Application Master dies unexpectedly and needs a restart after 7 
days (the default maximum lifespan a delegation token can be renewed).
+
+    * When the Samza job terminates and log aggregation is turned on for the 
job. Node managers need to be able to upload all the local application logs to 
HDFS.
+
+    1. Enable the resource manager as a privileged user in yarn-site.xml.
+    {% highlight xml %}
+        <property>
+            <name>yarn.resourcemanager.proxy-user-privileges.enabled</name>
+            <value>true</value>
+        </property>
+    {% endhighlight %}
+
+    2. Make `yarn` as a proxy user, in core-site.xml
+    {% highlight xml %}
+        <property>
+            <name>hadoop.proxyuser.yarn.hosts</name>
+            <value>*</value>
+        </property>
+        <property>
+            <name>hadoop.proxyuser.yarn.groups</name>
+            <value>*</value>
+        </property>
+    {% endhighlight %}
+
+## [Security &raquo;](../operations/security.html)

Reply via email to