flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From se...@apache.org
Subject [3/6] flink git commit: [FLINK-5581] [doc] Improve user accessibility for Kerberos-related documentation
Date Tue, 31 Jan 2017 13:03:27 GMT
[FLINK-5581] [doc] Improve user accessibility for Kerberos-related documentation

This closes #3181


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

Branch: refs/heads/master
Commit: a459b0b3bde62daabb56939d7ba6f2c8a69a2226
Parents: 8b2620f
Author: Tzu-Li (Gordon) Tai <tzulitai@apache.org>
Authored: Fri Jan 20 16:05:45 2017 +0100
Committer: Stephan Ewen <sewen@apache.org>
Committed: Tue Jan 31 14:02:44 2017 +0100

----------------------------------------------------------------------
 docs/dev/connectors/kafka.md                  | 21 ++++++
 docs/ops/security-kerberos.md                 | 52 ++++---------
 docs/setup/config.md                          | 85 ++++++++++++----------
 docs/setup/jobmanager_high_availability.md    | 14 ++++
 flink-dist/src/main/resources/flink-conf.yaml | 19 +++--
 5 files changed, 107 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a459b0b3/docs/dev/connectors/kafka.md
----------------------------------------------------------------------
diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md
index cc51071..6a58b7a 100644
--- a/docs/dev/connectors/kafka.md
+++ b/docs/dev/connectors/kafka.md
@@ -353,3 +353,24 @@ The offsets committed to ZK or the broker can also be used to track the
read pro
 the committed offset and the most recent offset in each partition is called the *consumer
lag*. If the Flink topology is consuming
 the data slower from the topic than new data is added, the lag will increase and the consumer
will fall behind.
 For large production deployments we recommend monitoring that metric to avoid increasing
latency.
+
+### Enabling Kerberos Authentication (for versions 0.9+ and above only)
+
+Flink provides first-class support through the Kafka connector to authenticate to a Kafka
installation
+configured for Kerberos. Simply configure Flink in `flink-conf.yaml` to enable Kerberos authentication
for Kafka like so:
+
+1. Configure Kerberos credentials by setting the following -
+ - `security.kerberos.login.use-ticket-cache`: By default, this is `true` and Flink will
attempt to use Kerberos credentials in ticket caches managed by `kinit`. 
+ Note that when using the Kafka connector in Flink jobs deployed on YARN, Kerberos authorization
using ticket caches will not work. This is also the case when deploying using Mesos, as authorization
using ticket cache is not supported for Mesos deployments. 
+ - `security.kerberos.login.keytab` and `security.kerberos.login.principal`: To use Kerberos
keytabs instead, set values for both of these properties.
+ 
+2. Append `KafkaClient` to `security.kerberos.login.contexts`: This tells Flink to provide
the configured Kerberos credentials to the Kafka login context to be used for Kafka authentication.
+
+Once Kerberos-based Flink security is enabled, you can authenticate to Kafka with either
the Flink Kafka Consumer or Producer by simply including the following two settings in the
provided properties configuration that is passed to the internal Kafka client:
+
+- Set `security.protocol` to `SASL_PLAINTEXT` (default `NONE`): The protocol used to communicate
to Kafka brokers.
+When using standalone Flink deployment, you can also use `SASL_SSL`; please see how to configure
the Kafka client for SSL [here](https://kafka.apache.org/documentation/#security_configclients).

+- Set `sasl.kerberos.service.name` to `kafka` (default `kafka`): The value for this should
match the `sasl.kerberos.service.name` used for Kafka broker configurations. A mismatch in
service name between client and server configuration will cause the authentication to fail.
+
+For more information on Flink configuration for Kerberos security, please see [here]({{ site.baseurl}}/setup/config.html).
+You can also find [here]({{ site.baseurl}}/ops/security-kerberos.html) further details on
how Flink internally setups Kerberos-based security.

http://git-wip-us.apache.org/repos/asf/flink/blob/a459b0b3/docs/ops/security-kerberos.md
----------------------------------------------------------------------
diff --git a/docs/ops/security-kerberos.md b/docs/ops/security-kerberos.md
index 2afe760..3e5cad9 100644
--- a/docs/ops/security-kerberos.md
+++ b/docs/ops/security-kerberos.md
@@ -28,6 +28,7 @@ filesystems, connectors, and state backends.
 
 ## Objective
 The primary goals of the Flink Kerberos security infrastructure are:
+
 1. to enable secure data access for jobs within a cluster via connectors (e.g. Kafka)
 2. to authenticate to ZooKeeper (if configured to use SASL)
 3. to authenticate to Hadoop components (e.g. HDFS, HBase) 
@@ -36,14 +37,14 @@ In a production deployment scenario, streaming jobs are understood to
run for lo
 data sources throughout the life of the job.  Kerberos keytabs do not expire in that timeframe,
unlike a Hadoop delegation token
 or ticket cache entry.
 
-The current implementation supports running Flink clusters (Job Manager/Task Manager/jobs)
with either a configured keytab credential
+The current implementation supports running Flink clusters (JobManager / TaskManager / jobs)
with either a configured keytab credential
 or with Hadoop delegation tokens.   Keep in mind that all jobs share the credential configured
for a given cluster.   To use a different keytab
 for for a certain job, simply launch a separate Flink cluster with a different configuration.
  Numerous Flink clusters may run side-by-side in a YARN
 or Mesos environment.
 
 ## How Flink Security works
 In concept, a Flink program may use first- or third-party connectors (Kafka, HDFS, Cassandra,
Flume, Kinesis etc.) necessitating arbitrary authentication methods (Kerberos, SSL/TLS, username/password,
etc.).  While satisfying the security requirements for all connectors is an ongoing effort,
-Flink provides first-class support for Kerberos authentication only.  The following services
and connectors are tested for Kerberos authentication:
+Flink provides first-class support for Kerberos authentication only.  The following services
and connectors are supported for Kerberos authentication:
 
 - Kafka (0.9+)
 - HDFS
@@ -55,7 +56,7 @@ Hadoop security without necessitating the use of Kerberos for ZooKeeper,
or vice
 Kerbreros credentials, which is then explicitly used by each component.
 
 The internal architecture is based on security modules (implementing `org.apache.flink.runtime.security.modules.SecurityModule`)
which
-are installed at startup.  The next section describes each security module.
+are installed at startup.  The following sections describes each security module.
 
 ### Hadoop Security Module
 This module uses the Hadoop `UserGroupInformation` (UGI) class to establish a process-wide
*login user* context.   The login user is
@@ -75,51 +76,22 @@ dynamic entries provided by this module.
 This module configures certain process-wide ZooKeeper security-related settings, namely the
ZooKeeper service name (default: `zookeeper`)
 and the JAAS login context name (default: `Client`).
 
-## Security Configuration
-
-### Flink Configuration
-The user's Kerberos ticket cache (managed with `kinit`) is used automatically, based on the
following configuration option:
-
-- `security.kerberos.login.use-ticket-cache`: Indicates whether to read from the user's Kerberos
ticket cache (default: `true`).
-
-A Kerberos keytab can be supplied by adding below configuration elements to the Flink configuration
file:
-
-- `security.kerberos.login.keytab`: Absolute path to a Kerberos keytab file that contains
the user credentials.
-
-- `security.kerberos.login.principal`: Kerberos principal name associated with the keytab.
-
-These configuration options establish a cluster-wide credential to be used in a Hadoop and/or
JAAS context.  Whether the credential is used in a Hadoop context is based on the Hadoop configuration
(see next section).   To be used in a JAAS context, the configuration specifies which JAAS
*login contexts* (or *applications*) are enabled with the following configuration option:
-
-- `security.kerberos.login.contexts`: A comma-separated list of login contexts to provide
the Kerberos credentials to (for example, `Client` to use the credentials for ZooKeeper authentication).
-
-ZooKeeper-related configuration overrides:
-
-- `zookeeper.sasl.service-name`: The Kerberos service name that the ZooKeeper cluster is
configured to use (default: `zookeeper`). Facilitates mutual-authentication between the client
(Flink) and server.
-
-- `zookeeper.sasl.login-context-name`: The JAAS login context name that the ZooKeeper client
uses to request the login context (default: `Client`). Should match
-one of the values specified in `security.kerberos.login.contexts`.
-
-### Hadoop Configuration
-
-The Hadoop configuration is located via the `HADOOP_CONF_DIR` environment variable and by
other means (see `org.apache.flink.api.java.hadoop.mapred.utils.HadoopUtils`).   The Kerberos
credential (configured above) is used automatically if Hadoop security is enabled.
-
-Note that Kerberos credentials found in the ticket cache aren't transferrable to other hosts.
  In this scenario, the Flink CLI acquires Hadoop
-delegation tokens (for HDFS and for HBase).
-
 ## Deployment Modes
 Here is some information specific to each deployment mode.
 
 ### Standalone Mode
 
 Steps to run a secure Flink cluster in standalone/cluster mode:
-1. Add security-related configuration options to the Flink configuration file (on all cluster
nodes).
+
+1. Add security-related configuration options to the Flink configuration file (on all cluster
nodes) (see [here]({{site.baseurl}}/setup/config.html#kerberos-based-security)).
 2. Ensure that the keytab file exists at the path indicated by `security.kerberos.login.keytab`
on all cluster nodes.
 3. Deploy Flink cluster as normal.
 
 ### YARN/Mesos Mode
 
 Steps to run a secure Flink cluster in YARN/Mesos mode:
-1. Add security-related configuration options to the Flink configuration file on the client.
+
+1. Add security-related configuration options to the Flink configuration file on the client
(see [here]({{site.baseurl}}/setup/config.html#kerberos-based-security)).
 2. Ensure that the keytab file exists at the path as indicated by `security.kerberos.login.keytab`
on the client node.
 3. Deploy Flink cluster as normal.
 
@@ -130,15 +102,17 @@ For more information, see <a href="https://github.com/apache/hadoop/blob/trunk/h
 #### Using `kinit` (YARN only)
 
 In YARN mode, it is possible to deploy a secure Flink cluster without a keytab, using only
the ticket cache (as managed by `kinit`).
-This avoids the complexity of generating a keytab and avoids entrusting the cluster manager
with it.  The main drawback is
-that the cluster is necessarily short-lived since the generated delegation tokens will expire
(typically within a week).
+This avoids the complexity of generating a keytab and avoids entrusting the cluster manager
with it.  In this scenario, the Flink CLI acquires Hadoop delegation tokens (for HDFS and
for HBase).
+The main drawback is that the cluster is necessarily short-lived since the generated delegation
tokens will expire (typically within a week).
 
 Steps to run a secure Flink cluster using `kinit`:
-1. Add security-related configuration options to the Flink configuration file on the client.
+
+1. Add security-related configuration options to the Flink configuration file on the client
(see [here]({{site.baseurl}}/setup/config.html#kerberos-based-security)).
 2. Login using the `kinit` command.
 3. Deploy Flink cluster as normal.
 
 ## Further Details
+
 ### Ticket Renewal
 Each component that uses Kerberos is independently responsible for renewing the Kerberos
ticket-granting-ticket (TGT).
 Hadoop, ZooKeeper, and Kafka all renew the TGT automatically when provided a keytab.  In
the delegation token scenario,

http://git-wip-us.apache.org/repos/asf/flink/blob/a459b0b3/docs/setup/config.md
----------------------------------------------------------------------
diff --git a/docs/setup/config.md b/docs/setup/config.md
index 1a72e27..269633c 100644
--- a/docs/setup/config.md
+++ b/docs/setup/config.md
@@ -100,25 +100,15 @@ These options are useful for debugging a Flink application for memory
and garbag
 
 Flink supports Kerberos authentication for the following services:
 
-+ Hadoop Components (such as HDFS, YARN, or HBase)
-+ Kafka Connectors (version 0.9+)
++ Hadoop Components, such as HDFS, YARN, or HBase *(version 2.6.1 and above; all other versions
have critical bugs which might fail the Flink job unexpectedly)*.
++ Kafka Connectors *(version 0.9+ and above)*.
 + Zookeeper
 
-**Kerberos is supported only in Hadoop version 2.6.1 and above. All
-  other versions have critical bugs which might fail the Flink job
-  unexpectedly.**
+Configuring Flink for Kerberos security involves three aspects, explained separately in the
following sub-sections.
 
-Configuring Flink for Kerberos security involves three aspects:
+##### 1. Providing the cluster with a Kerberos credential (i.e. a keytab or a ticket via
`kinit`)
 
-1. Providing the cluster with a Kerberos credential (i.e. a keytab or a ticket via `kinit`)
-2. Making the Kerberos credential available to components and connectors as needed
-3. Configuring the component and/or connector to use Kerberos authentication
-
-To provide the cluster with a Kerberos credential, either configure the login keytab using
the below configuration options,
-or login using `kinit` before starting the cluster.
-
-It is preferable to use keytabs for long-running jobs, to avoid ticket expiration issues.
  If you prefer to use the ticket cache,
-talk to your administrator about increasing the Hadoop delegation token lifetime.
+To provide the cluster with a Kerberos credential, Flink supports using a Kerberos keytab
file or ticket caches managed by `kinit`.
 
 - `security.kerberos.login.use-ticket-cache`: Indicates whether to read from your Kerberos
ticket cache (default: `true`).
 
@@ -126,28 +116,35 @@ talk to your administrator about increasing the Hadoop delegation token
lifetime
 
 - `security.kerberos.login.principal`: Kerberos principal name associated with the keytab.
 
-If Hadoop security is enabled (in `core-site.xml`), Flink will automatically use the configured
Kerberos credentials when connecting to HDFS, HBase, and other Hadoop components.
+If both `security.kerberos.login.keytab` and `security.kerberos.login.principal` have values
provided, keytabs will be used for authentication.
+It is preferable to use keytabs for long-running jobs, to avoid ticket expiration issues.
  If you prefer to use the ticket cache,
+talk to your administrator about increasing the Hadoop delegation token lifetime.
+
+Note that authentication using ticket caches is only supported when deploying Flink as a
standalone cluster or on YARN.
+
+##### 2. Making the Kerberos credential available to components and connectors as needed
+
+For Hadoop components, Flink will automatically detect if the configured Kerberos credentials
should be used when connecting to HDFS, HBase, and other Hadoop components depending on whether
Hadoop security is enabled (in `core-site.xml`).
 
-Make the Kerberos credentials available to any connector or component that uses a JAAS configuration
file by configuring JAAS login contexts.
+For any connector or component that uses a JAAS configuration file, make the Kerberos credentials
available to them by configuring JAAS login contexts for each one respectively, using the
following configuration:
 
 - `security.kerberos.login.contexts`: A comma-separated list of login contexts to provide
the Kerberos credentials to (for example, `Client,KafkaClient` to use the credentials for
ZooKeeper authentication and for Kafka authentication).
 
-You may also provide a static JAAS configuration file, whose entries override those produced
by the above configuration option.
+This allows enabling Kerberos authentication for different connectors or components independently.
For example, you can enable Hadoop security without necessitating the use of Kerberos for
ZooKeeper, or vice versa.
 
-Be sure to configure the connector within your Flink program as necessary to use Kerberos
authentication.  For the Kafka connector,
-use the following properties:
+You may also provide a static JAAS configuration file using the mechanisms described in the
[Java SE Documentation](http://docs.oracle.com/javase/7/docs/technotes/guides/security/jgss/tutorials/LoginConfigFile.html),
whose entries will override those produced by the above configuration option.
 
-```
-security.protocol=SASL_PLAINTEXT (or SASL_SSL)
-sasl.kerberos.service.name=kafka
-```
+##### 3. Configuring the component and/or connector to use Kerberos authentication
+
+Finally, be sure to configure the connector within your Flink program or component as necessary
to use Kerberos authentication.
 
-Flink provides some additional options to configure ZooKeeper security:
+Below is a list of currently first-class supported connectors or components by Flink for
Kerberos authentication:
 
-- `zookeeper.sasl.service-name`: The Kerberos service name that the ZooKeeper cluster is
configured to use (default: `zookeeper`).
+- Kafka: see [here]({{site.baseurl}}/dev/connectors/kafka.html#enabling-kerberos-authentication-for-versions-above-09-only)
for details on configuring the Kafka connector to use Kerberos authentication.
 
-- `zookeeper.sasl.login-context-name`: The JAAS login context name that the ZooKeeper client
uses to request the login context (default: `Client`). Should match
-one of the values specified in `security.kerberos.login.contexts`.
+- Zookeeper (for HA): see [here]({{site.baseurl}}/setup/jobmanager_high_availability.html#configuring-for-zookeeper-security)
for details on Zookeeper security configuration to work with the Kerberos-based security configurations
mentioned here.
+
+For more information on how Flink security internally setups Kerberos authentication, please
see [here]({{site.baseurl}}/ops/security-kerberos.html). 
 
 ### Other
 
@@ -401,7 +398,7 @@ The configuration keys in this section are independent of the used resource
mana
 of the JobManager, because the same ActorSystem is used. Its not possible to use this configuration
key to define port ranges.
 
 
-## YARN
+### YARN
 
 - `yarn.heap-cutoff-ratio`: (Default 0.25) Percentage of heap space to remove from containers
started by YARN. When a user requests a certain amount of memory for each TaskManager container
(for example 4 GB), we can not pass this amount as the maximum heap space for the JVM (`-Xmx`
argument) because the JVM is also allocating memory outside the heap. YARN is very strict
with killing containers which are using more memory than requested. Therefore, we remove a
15% of the memory from the requested heap as a safety margin.
 
@@ -435,7 +432,7 @@ use the `env.java.opts` setting, which is the `%jvmopts%` variable in
the String
   For example when running Flink on YARN on an environment with a restrictive firewall, this
option allows specifying a range of allowed ports.
 
 
-## High Availability (HA)
+### High Availability (HA)
 
 - `high-availability`: Defines the high availability mode used for the cluster execution.
Currently, Flink supports the following modes:
   - `none` (default): No high availability. A single JobManager runs and no JobManager state
is checkpointed.
@@ -443,9 +440,9 @@ use the `env.java.opts` setting, which is the `%jvmopts%` variable in
the String
 
 Previously this key was named `recovery.mode` and the default value was `standalone`.
 
-### ZooKeeper-based HA Mode
+#### ZooKeeper-based HA Mode
 
-- `high-availability.zookeeper.quorum`: Defines the ZooKeeper quorum URL which is used to
connect to the ZooKeeper cluster when the 'zookeeper' HA mode is selected. Previously this
key was name `recovery.zookeeper.quorum`.
+- `high-availability.zookeeper.quorum`: Defines the ZooKeeper quorum URL which is used to
connect to the ZooKeeper cluster when the 'zookeeper' HA mode is selected. Previously this
key was named `recovery.zookeeper.quorum`.
 
 - `high-availability.zookeeper.path.root`: (Default `/flink`) Defines the root dir under
which the ZooKeeper HA mode will create namespace directories. Previously this ket was named
`recovery.zookeeper.path.root`.
 
@@ -469,19 +466,29 @@ Previously this key was named `recovery.mode` and the default value
was `standal
 
 - `high-availability.zookeeper.client.acl`: (Default `open`) Defines the ACL (open|creator)
to be configured on ZK node. The configuration value can be set to "creator" if the ZooKeeper
server configuration has the "authProvider" property mapped to use SASLAuthenticationProvider
and the cluster is configured to run in secure mode (Kerberos). The ACL options are based
on https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_BuiltinACLSchemes
 
-### ZooKeeper-Security
+#### ZooKeeper Security
 
-- `zookeeper.sasl.disable`: (Default: `true`) Defines if SASL based authentication needs
to be enabled or disabled. The configuration value can be set to "true" if ZooKeeper cluster
is running in secure mode (Kerberos)
+- `zookeeper.sasl.disable`: (Default: `true`) Defines if SASL based authentication needs
to be enabled or disabled. The configuration value can be set to "true" if ZooKeeper cluster
is running in secure mode (Kerberos).
 
 - `zookeeper.sasl.service-name`: (Default: `zookeeper`) If the ZooKeeper server is configured
with a different service name (default:"zookeeper") then it can be supplied using this configuration.
A mismatch in service name between client and server configuration will cause the authentication
to fail.
 
-## Environment
+### Kerberos-based Security
+
+- `security.kerberos.login.use-ticket-cache`: Indicates whether to read from your Kerberos
ticket cache (default: `true`).
+
+- `security.kerberos.login.keytab`: Absolute path to a Kerberos keytab file that contains
the user credentials.
+
+- `security.kerberos.login.principal`: Kerberos principal name associated with the keytab.
+
+- `security.kerberos.login.contexts`: A comma-separated list of login contexts to provide
the Kerberos credentials to (for example, `Client,KafkaClient` to use the credentials for
ZooKeeper authentication and for Kafka authentication).
+
+### Environment
 
 - `env.log.dir`: (Defaults to the `log` directory under Flink's home) Defines the directory
where the Flink logs are saved. It has to be an absolute path.
 
-## Queryable State
+### Queryable State
 
-### Server
+#### Server
 
 - `query.server.enable`: Enable queryable state (Default: `true`).
 
@@ -491,7 +498,7 @@ Previously this key was named `recovery.mode` and the default value was
`standal
 
 - `query.server.query-threads`: Number of query Threads for queryable state server (Default:
`0`, picks number of slots).
 
-### Client
+#### Client
 
 - `query.client.network-threads`: Number of network (Netty's event loop) Threads for queryable
state client (Default: `0`, picks number of available cores as returned by `Runtime.getRuntime().availableProcessors()`).
 
@@ -499,7 +506,7 @@ Previously this key was named `recovery.mode` and the default value was
`standal
 
 - `query.client.lookup.retry-delay`: Retry delay in milliseconds on KvState lookup failure
due to unavailable JobManager (Default: `1000`).
 
-## Metrics
+### Metrics
 
 - `metrics.reporters`: The list of named reporters, i.e. "foo,bar".
 

http://git-wip-us.apache.org/repos/asf/flink/blob/a459b0b3/docs/setup/jobmanager_high_availability.md
----------------------------------------------------------------------
diff --git a/docs/setup/jobmanager_high_availability.md b/docs/setup/jobmanager_high_availability.md
index 9dcc7cc..aa18a4b 100644
--- a/docs/setup/jobmanager_high_availability.md
+++ b/docs/setup/jobmanager_high_availability.md
@@ -211,6 +211,20 @@ Starting zookeeper daemon on host localhost.</pre>
    <pre>
 $ bin/yarn-session.sh -n 2</pre>
 
+## Configuring for Zookeeper Security
+
+If ZooKeeper is running in secure mode with Kerberos, you can override the following configurations
in `flink-conf.yaml` as necessary:
+
+<pre>
+zookeeper.sasl.service-name: zookeeper     # default is "zookeeper". If the ZooKeeper quorum
is configured
+                                           # with a different service name then it can be
supplied here.
+zookeeper.sasl.login-context-name: Client  # default is "Client". The value needs to match
one of the values
+                                           # configured in "security.kerberos.login.contexts".
+</pre>
+
+For more information on Flink configuration for Kerberos security, please see [here]({{ site.baseurl}}/setup/config.html).
+You can also find [here]({{ site.baseurl}}/ops/security-kerberos.html) further details on
how Flink internally setups Kerberos-based security.
+
 ## Bootstrap ZooKeeper
 
 If you don't have a running ZooKeeper installation, you can use the helper scripts, which
ship with Flink.

http://git-wip-us.apache.org/repos/asf/flink/blob/a459b0b3/flink-dist/src/main/resources/flink-conf.yaml
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml
index 108bd58..0f30595 100644
--- a/flink-dist/src/main/resources/flink-conf.yaml
+++ b/flink-dist/src/main/resources/flink-conf.yaml
@@ -89,7 +89,7 @@ jobmanager.web.port: 8081
 #
 # Supported backends: jobmanager, filesystem, rocksdb, <class-name-of-factory>
 #
-#state.backend: filesystem
+# state.backend: filesystem
 
 
 # Directory for storing checkpoints in a Flink-supported filesystem
@@ -169,11 +169,16 @@ jobmanager.web.port: 8081
 # 3. make the credentials available to various JAAS login contexts
 # 4. configure the connector to use JAAS/SASL
 
-#security.kerberos.login.keytab: /path/to/kerberos/keytab
-#security.kerberos.login.principal: flink-user
-#security.kerberos.login.use-ticket-cache: true
+# The below configure how Kerberos credentials are provided. A keytab will be used instead
of
+# a ticket cache if the keytab path and principal are set.
 
-#security.kerberos.login.contexts: Client,KafkaClient
+# security.kerberos.login.use-ticket-cache: true
+# security.kerberos.login.keytab: /path/to/kerberos/keytab
+# security.kerberos.login.principal: flink-user
+
+# The configuration below defines which JAAS login contexts
+
+# security.kerberos.login.contexts: Client,KafkaClient
 
 #==============================================================================
 # ZK Security Configuration (optional configuration)
@@ -182,5 +187,7 @@ jobmanager.web.port: 8081
 # Below configurations are applicable if ZK ensemble is configured for security
 
 # Override below configuration to provide custom ZK service name if configured
-#
 # zookeeper.sasl.service-name: zookeeper
+
+# The configuration below must match one of the values set in "security.kerberos.login.contexts"
+# zookeeper.sasl.login-context-name: Client


Mime
View raw message