You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Zac Harvey <za...@welltok.com> on 2016/11/17 21:44:06 UTC

Can Kafka/SSL be terminated at a load balancer?

We have two Kafka nodes and for reasons outside of this question, would like to set up a load balancer to terminate SSL with producers (clients). The SSL cert hosted by the load balancer will be signed by trusted/root CA that clients should natively trust.


Is this possible to do, or does Kafka somehow require SSL to be setup directly on the Kafka servers themselves?


Thanks!

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Rajini Sivaram <ra...@googlemail.com>.
Ignore the comment about lookups. Your client is finding mybalancer01 since
it was working earlier and kafka doesn't need to lookup mybalancer01. It
will be good to check the jaas config and then run with debug logging.

On Mon, Nov 21, 2016 at 5:16 PM, Rajini Sivaram <
rajinisivaram@googlemail.com> wrote:

> Not sure if this is the exact jaas.conf that you have, because this has
> mismatched passwords:
>
> KafkaServer {
>    org.apache.kafka.common.security.plain.PlainLoginModule required
>    username="someuser"
>    user_kafka="somePassword"
>    password="kafka-password";
> };
>
>
> To use username "some user", password "somePassword", the config should be:
>
> KafkaServer {
>    org.apache.kafka.common.security.plain.PlainLoginModule required
>    username="someuser"
>    user_*someuser*="somePassword"
>    password="*somePassword*";
> };
>
>
> But I would have expected to see an error in the Kafka logs if the
> inter-broker config was incorrect.
>
> I am assuming all your hostnames can be found from the different machines
> since PLAINTEXT was working earlier. But it would be worth checking that
> mykafka01 can lookup mybalancer01. It would be worth running Kafka and a
> console producer with debug logging turned on. Kafka uses
> *config/log4j.properties* and console producer uses
> *config/tools-log4j.properties.* Since you are testing with PLAINTEXT, it
> should be easy to run a console producer with just standard arguments with
> bootstrap server set to mybalancer01:9093.
>
>
>
> On Mon, Nov 21, 2016 at 4:37 PM, Zac Harvey <za...@welltok.com>
> wrote:
>
>> Thanks again. So this might be very telling of the underlying problem:
>>
>>
>> I did what you suggested:
>>
>>
>> 1) I disabled (actually deleted) the first rule; then
>>
>> 2) I changed the load balancer's second (which is now its only) rule to
>> accept TCP:9093 and to translate that to TCP:9093, making the conneciton
>> PLAINTEXT all the way through to Kafka; then
>>
>> 3) I tried connecting a Scala consumer to the load balancer URL (
>> mybalancer01.example.com) and I'm getting that ClosedChannelException
>>
>>
>> For now there is only one Kafka broker sitting behind the load balancer.
>> It's server.properties look like:
>>
>>
>> listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092
>>
>> advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
>> ,SASL_PLAINTEXT://mykafka01.example.com:9092
>>
>> advertised.host.name=mykafka01.example.com
>>
>> security.inter.broker.protocol=SASL_PLAINTEXT
>>
>> sasl.enabled.mechanisms=PLAIN
>>
>> sasl.mechanism.inter.broker.protocol=PLAIN
>>
>> broker.id=1
>>
>> num.partitions=4
>>
>> zookeeper.connect=zkA:2181,zkB:2181,zkC:2181
>>
>> num.network.threads=3
>>
>> num.io.threads=8
>>
>> socket.send.buffer.bytes=102400
>>
>> socket.receive.buffer.bytes=102400
>>
>> log.dirs=/tmp/kafka-logs
>>
>> num.recovery.threads.per.data.dir=1
>>
>> log.retention.hours=168
>>
>> log.segment.bytes=1073741824
>>
>> log.retention.check.interval.ms=300000
>>
>> zookeeper.connection.timeout.ms=6000
>>
>> offset.metadata.max.bytes=4096
>>
>>
>> Above, 'zkA', 'zkB' and 'zkC' are defined inside `/etc/hosts` and are
>> valid server names.
>>
>>
>> And then inside the kafka-run-class.sh script, instead of the default:
>>
>>
>> if [ -z "$KAFKA_OPTS" ]; then
>>
>>   KAFKA_OPTS=""
>>
>> fi
>>
>>
>> I have:
>>
>>
>> if [ -z "$KAFKA_OPTS" ]; then
>>
>>   KAFKA_OPTS="-Djava.security.auth.login.config=/opt/kafka/con
>> fig/jaas.conf"
>>
>> fi
>>
>>
>> I also added the /opt/kafka/config/jaas.conf file like you suggested, and
>> only changed the names of users and passwords:
>>
>>
>> KafkaServer {
>>
>>   org.apache.kafka.common.security.plain.PlainLoginModule required
>>
>>   username="someuser"
>>
>>   user_kafka="somePassword"
>>
>>   password="kafka-password";
>>
>> };
>>
>>
>> The fact that I can no longer even consume from a topic over PLAINTEXT
>> (which is a regression of where I was before we started trying to add SSL)
>> tells me there is something wrong in either server.properties or jaas.conf.
>> I've checked the Kafka broker logs (server.log) each time I try connecting
>> and this is the only line that gets printed:
>>
>>
>> [2016-11-21 15:18:14,859] INFO [Group Metadata Manager on Broker 2]:
>> Removed 0 expired offsets in 0 milliseconds. (kafka.coordinator.GroupMetada
>> taManager)
>>
>>
>> Not sure if that means anything. Any idea where I might be going wrong?
>> Thanks again!
>>
>> ________________________________
>> From: Rajini Sivaram <ra...@googlemail.com>
>> Sent: Monday, November 21, 2016 11:03:14 AM
>> To: users@kafka.apache.org
>> Subject: Re: Can Kafka/SSL be terminated at a load balancer?
>>
>> Rule #1 and Rule #2 cannot co-exist. You are basically configuring your LB
>> to point to a Kafka broker and you are pointing each Kafka broker to point
>> to a LB. So you need a pair of ports with a security protocol for the
>> connection to work. With two rules, Kafka picks up the wrong LB port for
>> one of the security protocols.
>>
>> If you want to try without SSL first, the simplest way to try it out would
>> be to disable Rule #1 and change Rule #2 to use port 9093 instead of 9095.
>> Then you should be able to connect using PLAINTEXT (the test that is
>> currently not working).
>>
>> I think you have the configuration:
>>
>> advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
>> ,SASL_PLAINTEXT://mykafka01.example.com:9092
>>
>> And you have a client connecting with PLAINTEXT on mybalancer01:*9095*.
>> The
>> first connection would work, but subsequent connections would use the
>> address provided by Kafka from advertised.listeners. The client  will
>> start
>> connecting with PLAINTEXT on mybalancer01:*9093*, which is expecting SSL.
>> If you disable Rule #1 and change Rule #2 to use port 9093, you should be
>> able to test PLAINTEXT without changing Kafka config.
>>
>> On Mon, Nov 21, 2016 at 3:32 PM, Zac Harvey <za...@welltok.com>
>> wrote:
>>
>> > In the last email I should have mentioned: don't pay too much attention
>> to
>> > the code snippet, and after reviewing it, I can see it actually
>> incomplete
>> > (I forgot to include the section where I configure the topics and broker
>> > configs to talk to Kafka!).
>> >
>> >
>> > What I'm really concerned about is that before we added all these SSL
>> > configs, I had plaintext (plaintext:9092 in/out of the load balancer
>> > to/from Kafka) working fine. Now my consumer code can't even connect to
>> the
>> > load balancer/Kafka.
>> >
>> >
>> > So I guess what I was really asking was: does that exception
>> > (ClosedChannelException) indicate bad configs on the Kafka broker?
>> >
>> > ________________________________
>> > From: Zac Harvey <za...@welltok.com>
>> > Sent: Thursday, November 17, 2016 4:44:06 PM
>> > To: users@kafka.apache.org
>> > Subject: Can Kafka/SSL be terminated at a load balancer?
>> >
>> > We have two Kafka nodes and for reasons outside of this question, would
>> > like to set up a load balancer to terminate SSL with producers
>> (clients).
>> > The SSL cert hosted by the load balancer will be signed by trusted/root
>> CA
>> > that clients should natively trust.
>> >
>> >
>> > Is this possible to do, or does Kafka somehow require SSL to be setup
>> > directly on the Kafka servers themselves?
>> >
>> >
>> > Thanks!
>> >
>>
>>
>>
>> --
>> Regards,
>>
>> Rajini
>>
>
>
>
> --
> Regards,
>
> Rajini
>



-- 
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Rajini Sivaram <ra...@googlemail.com>.
Not sure if this is the exact jaas.conf that you have, because this has
mismatched passwords:

KafkaServer {
   org.apache.kafka.common.security.plain.PlainLoginModule required
   username="someuser"
   user_kafka="somePassword"
   password="kafka-password";
};


To use username "some user", password "somePassword", the config should be:

KafkaServer {
   org.apache.kafka.common.security.plain.PlainLoginModule required
   username="someuser"
   user_*someuser*="somePassword"
   password="*somePassword*";
};


But I would have expected to see an error in the Kafka logs if the
inter-broker config was incorrect.

I am assuming all your hostnames can be found from the different machines
since PLAINTEXT was working earlier. But it would be worth checking that
mykafka01 can lookup mybalancer01. It would be worth running Kafka and a
console producer with debug logging turned on. Kafka uses
*config/log4j.properties* and console producer uses
*config/tools-log4j.properties.* Since you are testing with PLAINTEXT, it
should be easy to run a console producer with just standard arguments with
bootstrap server set to mybalancer01:9093.



On Mon, Nov 21, 2016 at 4:37 PM, Zac Harvey <za...@welltok.com> wrote:

> Thanks again. So this might be very telling of the underlying problem:
>
>
> I did what you suggested:
>
>
> 1) I disabled (actually deleted) the first rule; then
>
> 2) I changed the load balancer's second (which is now its only) rule to
> accept TCP:9093 and to translate that to TCP:9093, making the conneciton
> PLAINTEXT all the way through to Kafka; then
>
> 3) I tried connecting a Scala consumer to the load balancer URL (
> mybalancer01.example.com) and I'm getting that ClosedChannelException
>
>
> For now there is only one Kafka broker sitting behind the load balancer.
> It's server.properties look like:
>
>
> listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092
>
> advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093,SASL_
> PLAINTEXT://mykafka01.example.com:9092
>
> advertised.host.name=mykafka01.example.com
>
> security.inter.broker.protocol=SASL_PLAINTEXT
>
> sasl.enabled.mechanisms=PLAIN
>
> sasl.mechanism.inter.broker.protocol=PLAIN
>
> broker.id=1
>
> num.partitions=4
>
> zookeeper.connect=zkA:2181,zkB:2181,zkC:2181
>
> num.network.threads=3
>
> num.io.threads=8
>
> socket.send.buffer.bytes=102400
>
> socket.receive.buffer.bytes=102400
>
> log.dirs=/tmp/kafka-logs
>
> num.recovery.threads.per.data.dir=1
>
> log.retention.hours=168
>
> log.segment.bytes=1073741824
>
> log.retention.check.interval.ms=300000
>
> zookeeper.connection.timeout.ms=6000
>
> offset.metadata.max.bytes=4096
>
>
> Above, 'zkA', 'zkB' and 'zkC' are defined inside `/etc/hosts` and are
> valid server names.
>
>
> And then inside the kafka-run-class.sh script, instead of the default:
>
>
> if [ -z "$KAFKA_OPTS" ]; then
>
>   KAFKA_OPTS=""
>
> fi
>
>
> I have:
>
>
> if [ -z "$KAFKA_OPTS" ]; then
>
>   KAFKA_OPTS="-Djava.security.auth.login.config=/opt/kafka/
> config/jaas.conf"
>
> fi
>
>
> I also added the /opt/kafka/config/jaas.conf file like you suggested, and
> only changed the names of users and passwords:
>
>
> KafkaServer {
>
>   org.apache.kafka.common.security.plain.PlainLoginModule required
>
>   username="someuser"
>
>   user_kafka="somePassword"
>
>   password="kafka-password";
>
> };
>
>
> The fact that I can no longer even consume from a topic over PLAINTEXT
> (which is a regression of where I was before we started trying to add SSL)
> tells me there is something wrong in either server.properties or jaas.conf.
> I've checked the Kafka broker logs (server.log) each time I try connecting
> and this is the only line that gets printed:
>
>
> [2016-11-21 15:18:14,859] INFO [Group Metadata Manager on Broker 2]:
> Removed 0 expired offsets in 0 milliseconds. (kafka.coordinator.
> GroupMetadataManager)
>
>
> Not sure if that means anything. Any idea where I might be going wrong?
> Thanks again!
>
> ________________________________
> From: Rajini Sivaram <ra...@googlemail.com>
> Sent: Monday, November 21, 2016 11:03:14 AM
> To: users@kafka.apache.org
> Subject: Re: Can Kafka/SSL be terminated at a load balancer?
>
> Rule #1 and Rule #2 cannot co-exist. You are basically configuring your LB
> to point to a Kafka broker and you are pointing each Kafka broker to point
> to a LB. So you need a pair of ports with a security protocol for the
> connection to work. With two rules, Kafka picks up the wrong LB port for
> one of the security protocols.
>
> If you want to try without SSL first, the simplest way to try it out would
> be to disable Rule #1 and change Rule #2 to use port 9093 instead of 9095.
> Then you should be able to connect using PLAINTEXT (the test that is
> currently not working).
>
> I think you have the configuration:
>
> advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
> ,SASL_PLAINTEXT://mykafka01.example.com:9092
>
> And you have a client connecting with PLAINTEXT on mybalancer01:*9095*. The
> first connection would work, but subsequent connections would use the
> address provided by Kafka from advertised.listeners. The client  will start
> connecting with PLAINTEXT on mybalancer01:*9093*, which is expecting SSL.
> If you disable Rule #1 and change Rule #2 to use port 9093, you should be
> able to test PLAINTEXT without changing Kafka config.
>
> On Mon, Nov 21, 2016 at 3:32 PM, Zac Harvey <za...@welltok.com>
> wrote:
>
> > In the last email I should have mentioned: don't pay too much attention
> to
> > the code snippet, and after reviewing it, I can see it actually
> incomplete
> > (I forgot to include the section where I configure the topics and broker
> > configs to talk to Kafka!).
> >
> >
> > What I'm really concerned about is that before we added all these SSL
> > configs, I had plaintext (plaintext:9092 in/out of the load balancer
> > to/from Kafka) working fine. Now my consumer code can't even connect to
> the
> > load balancer/Kafka.
> >
> >
> > So I guess what I was really asking was: does that exception
> > (ClosedChannelException) indicate bad configs on the Kafka broker?
> >
> > ________________________________
> > From: Zac Harvey <za...@welltok.com>
> > Sent: Thursday, November 17, 2016 4:44:06 PM
> > To: users@kafka.apache.org
> > Subject: Can Kafka/SSL be terminated at a load balancer?
> >
> > We have two Kafka nodes and for reasons outside of this question, would
> > like to set up a load balancer to terminate SSL with producers (clients).
> > The SSL cert hosted by the load balancer will be signed by trusted/root
> CA
> > that clients should natively trust.
> >
> >
> > Is this possible to do, or does Kafka somehow require SSL to be setup
> > directly on the Kafka servers themselves?
> >
> >
> > Thanks!
> >
>
>
>
> --
> Regards,
>
> Rajini
>



-- 
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Zac Harvey <za...@welltok.com>.
Thanks again. So this might be very telling of the underlying problem:


I did what you suggested:


1) I disabled (actually deleted) the first rule; then

2) I changed the load balancer's second (which is now its only) rule to accept TCP:9093 and to translate that to TCP:9093, making the conneciton PLAINTEXT all the way through to Kafka; then

3) I tried connecting a Scala consumer to the load balancer URL (mybalancer01.example.com) and I'm getting that ClosedChannelException


For now there is only one Kafka broker sitting behind the load balancer. It's server.properties look like:


listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092

advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093,SASL_PLAINTEXT://mykafka01.example.com:9092

advertised.host.name=mykafka01.example.com

security.inter.broker.protocol=SASL_PLAINTEXT

sasl.enabled.mechanisms=PLAIN

sasl.mechanism.inter.broker.protocol=PLAIN

broker.id=1

num.partitions=4

zookeeper.connect=zkA:2181,zkB:2181,zkC:2181

num.network.threads=3

num.io.threads=8

socket.send.buffer.bytes=102400

socket.receive.buffer.bytes=102400

log.dirs=/tmp/kafka-logs

num.recovery.threads.per.data.dir=1

log.retention.hours=168

log.segment.bytes=1073741824

log.retention.check.interval.ms=300000

zookeeper.connection.timeout.ms=6000

offset.metadata.max.bytes=4096


Above, 'zkA', 'zkB' and 'zkC' are defined inside `/etc/hosts` and are valid server names.


And then inside the kafka-run-class.sh script, instead of the default:


if [ -z "$KAFKA_OPTS" ]; then

  KAFKA_OPTS=""

fi


I have:


if [ -z "$KAFKA_OPTS" ]; then

  KAFKA_OPTS="-Djava.security.auth.login.config=/opt/kafka/config/jaas.conf"

fi


I also added the /opt/kafka/config/jaas.conf file like you suggested, and only changed the names of users and passwords:


KafkaServer {

  org.apache.kafka.common.security.plain.PlainLoginModule required

  username="someuser"

  user_kafka="somePassword"

  password="kafka-password";

};


The fact that I can no longer even consume from a topic over PLAINTEXT (which is a regression of where I was before we started trying to add SSL) tells me there is something wrong in either server.properties or jaas.conf. I've checked the Kafka broker logs (server.log) each time I try connecting and this is the only line that gets printed:


[2016-11-21 15:18:14,859] INFO [Group Metadata Manager on Broker 2]: Removed 0 expired offsets in 0 milliseconds. (kafka.coordinator.GroupMetadataManager)


Not sure if that means anything. Any idea where I might be going wrong? Thanks again!

________________________________
From: Rajini Sivaram <ra...@googlemail.com>
Sent: Monday, November 21, 2016 11:03:14 AM
To: users@kafka.apache.org
Subject: Re: Can Kafka/SSL be terminated at a load balancer?

Rule #1 and Rule #2 cannot co-exist. You are basically configuring your LB
to point to a Kafka broker and you are pointing each Kafka broker to point
to a LB. So you need a pair of ports with a security protocol for the
connection to work. With two rules, Kafka picks up the wrong LB port for
one of the security protocols.

If you want to try without SSL first, the simplest way to try it out would
be to disable Rule #1 and change Rule #2 to use port 9093 instead of 9095.
Then you should be able to connect using PLAINTEXT (the test that is
currently not working).

I think you have the configuration:

advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
,SASL_PLAINTEXT://mykafka01.example.com:9092

And you have a client connecting with PLAINTEXT on mybalancer01:*9095*. The
first connection would work, but subsequent connections would use the
address provided by Kafka from advertised.listeners. The client  will start
connecting with PLAINTEXT on mybalancer01:*9093*, which is expecting SSL.
If you disable Rule #1 and change Rule #2 to use port 9093, you should be
able to test PLAINTEXT without changing Kafka config.

On Mon, Nov 21, 2016 at 3:32 PM, Zac Harvey <za...@welltok.com> wrote:

> In the last email I should have mentioned: don't pay too much attention to
> the code snippet, and after reviewing it, I can see it actually incomplete
> (I forgot to include the section where I configure the topics and broker
> configs to talk to Kafka!).
>
>
> What I'm really concerned about is that before we added all these SSL
> configs, I had plaintext (plaintext:9092 in/out of the load balancer
> to/from Kafka) working fine. Now my consumer code can't even connect to the
> load balancer/Kafka.
>
>
> So I guess what I was really asking was: does that exception
> (ClosedChannelException) indicate bad configs on the Kafka broker?
>
> ________________________________
> From: Zac Harvey <za...@welltok.com>
> Sent: Thursday, November 17, 2016 4:44:06 PM
> To: users@kafka.apache.org
> Subject: Can Kafka/SSL be terminated at a load balancer?
>
> We have two Kafka nodes and for reasons outside of this question, would
> like to set up a load balancer to terminate SSL with producers (clients).
> The SSL cert hosted by the load balancer will be signed by trusted/root CA
> that clients should natively trust.
>
>
> Is this possible to do, or does Kafka somehow require SSL to be setup
> directly on the Kafka servers themselves?
>
>
> Thanks!
>



--
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Rajini Sivaram <ra...@googlemail.com>.
Rule #1 and Rule #2 cannot co-exist. You are basically configuring your LB
to point to a Kafka broker and you are pointing each Kafka broker to point
to a LB. So you need a pair of ports with a security protocol for the
connection to work. With two rules, Kafka picks up the wrong LB port for
one of the security protocols.

If you want to try without SSL first, the simplest way to try it out would
be to disable Rule #1 and change Rule #2 to use port 9093 instead of 9095.
Then you should be able to connect using PLAINTEXT (the test that is
currently not working).

I think you have the configuration:

advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
,SASL_PLAINTEXT://mykafka01.example.com:9092

And you have a client connecting with PLAINTEXT on mybalancer01:*9095*. The
first connection would work, but subsequent connections would use the
address provided by Kafka from advertised.listeners. The client  will start
connecting with PLAINTEXT on mybalancer01:*9093*, which is expecting SSL.
If you disable Rule #1 and change Rule #2 to use port 9093, you should be
able to test PLAINTEXT without changing Kafka config.

On Mon, Nov 21, 2016 at 3:32 PM, Zac Harvey <za...@welltok.com> wrote:

> In the last email I should have mentioned: don't pay too much attention to
> the code snippet, and after reviewing it, I can see it actually incomplete
> (I forgot to include the section where I configure the topics and broker
> configs to talk to Kafka!).
>
>
> What I'm really concerned about is that before we added all these SSL
> configs, I had plaintext (plaintext:9092 in/out of the load balancer
> to/from Kafka) working fine. Now my consumer code can't even connect to the
> load balancer/Kafka.
>
>
> So I guess what I was really asking was: does that exception
> (ClosedChannelException) indicate bad configs on the Kafka broker?
>
> ________________________________
> From: Zac Harvey <za...@welltok.com>
> Sent: Thursday, November 17, 2016 4:44:06 PM
> To: users@kafka.apache.org
> Subject: Can Kafka/SSL be terminated at a load balancer?
>
> We have two Kafka nodes and for reasons outside of this question, would
> like to set up a load balancer to terminate SSL with producers (clients).
> The SSL cert hosted by the load balancer will be signed by trusted/root CA
> that clients should natively trust.
>
>
> Is this possible to do, or does Kafka somehow require SSL to be setup
> directly on the Kafka servers themselves?
>
>
> Thanks!
>



-- 
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Zac Harvey <za...@welltok.com>.
In the last email I should have mentioned: don't pay too much attention to the code snippet, and after reviewing it, I can see it actually incomplete (I forgot to include the section where I configure the topics and broker configs to talk to Kafka!).


What I'm really concerned about is that before we added all these SSL configs, I had plaintext (plaintext:9092 in/out of the load balancer to/from Kafka) working fine. Now my consumer code can't even connect to the load balancer/Kafka.


So I guess what I was really asking was: does that exception (ClosedChannelException) indicate bad configs on the Kafka broker?

________________________________
From: Zac Harvey <za...@welltok.com>
Sent: Thursday, November 17, 2016 4:44:06 PM
To: users@kafka.apache.org
Subject: Can Kafka/SSL be terminated at a load balancer?

We have two Kafka nodes and for reasons outside of this question, would like to set up a load balancer to terminate SSL with producers (clients). The SSL cert hosted by the load balancer will be signed by trusted/root CA that clients should natively trust.


Is this possible to do, or does Kafka somehow require SSL to be setup directly on the Kafka servers themselves?


Thanks!

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Zac Harvey <za...@welltok.com>.
Thanks Rajini,


So I have implemented your solution (the the best of my knowledge) 100% as you have specified.


On the load balancer (AWS ELB) I have two load balancer rules:


Rule #1:

Incoming: SSL:9093

Outgoing (to Kafka): TCP:9093


Rule #2:

Incoming: TCP:9095

Outgoing (toKafka): TCP: 9093


I added the 2nd rule so that I could test this config/setup one step at a time, beginning with a Scala/Spark consumer that tries to connect to the load balancer over plaintext/9095. When I run this consumer/test code:


[code]

object ScalaTestConsumer {
    def main(args: Array[String]): Unit = {
        val jsonMapper : JsonScalaUtils = new JsonScalaUtils()
        var jsonArgs = jsonMapper.toMap[String](args(0))
        var kafkaConfigs = jsonMapper.toMap[String](args(1))

        def messageConsumer(): StreamingContext = {
            val ssc = new StreamingContext(SparkContext.getOrCreate(), Seconds(10))
            val topic : String = jsonArgs("topic")

            createKafkaStream(ssc, topic, kafkaConfigs).foreachRDD(rdd => {
                rdd.collect().foreach { msg =>
                    try {
                      jsonArgs += "inboundMessage" -> msg._2

                      Processor.start(jsonArgs.asJava)
                    } catch {
                        case e @ (_: Exception | _: Error | _: Throwable) => {
                        println("Exception thrown: " + e.getMessage)
                        e.printStackTrace()
                    }
                  }
                }
            })

            ssc
        }

        StreamingContext.getActive.foreach {
            _.stop(stopSparkContext = false)
        }

        val ssc = StreamingContext.getActiveOrCreate(messageConsumer)
        ssc.start()
        ssc.awaitTermination()
    }

    def createKafkaStream(ssc: StreamingContext,
            kafkaTopics: String, kafkaConfigs: Map[String,String]): DStream[(String, String)] = {
        KafkaUtils.createDirectStream[String,String,StringDecoder,StringDecoder](ssc, kafkaConfigs, Set(kafkaTopics))
    }
}
[/code]


I get this exception after 60 seconds:


org.apache.spark.SparkException: java.nio.channels.ClosedChannelException
at org.apache.spark.streaming.kafka.KafkaCluster$$anonfun$checkErrors$1.apply(KafkaCluster.scala:366) at org.apache.spark.streaming.kafka.KafkaCluster$$anonfun$checkErrors$1.apply(KafkaCluster.scala:366) at scala.util.Either.fold(Either.scala:97)
<rest of LARGE stacktrace ommitted for brevity>



If I can get this working and properly connecting to ELB (hence consuming from Kafka) then I can run the kafka-console-producer and send a message to the topic, and see my Scala consumer process it.


Once I have that working, I can have the confidence to tell the other Ruby client (producers) to start trying to connect over port SSL:9093 and I will remove the second load balancer rule.


Any ideas as to why I'm getting this 'ClosedChannelException' exception, or how I could troubleshoot it?


Thanks again!


Best,

Zac

________________________________
From: Rajini Sivaram <ra...@googlemail.com>
Sent: Monday, November 21, 2016 10:11:00 AM
To: users@kafka.apache.org
Subject: Re: Can Kafka/SSL be terminated at a load balancer?

A load balancer that balances the load across the brokers wouldn't work,
but here the LB is being used as a terminating SSL proxy. That should work
if each Kafka is configured with its own proxy.

On Mon, Nov 21, 2016 at 2:57 PM, tao xiao <xi...@gmail.com> wrote:

> I doubt the LB solution will work for Kafka. Client needs to connect to the
> leader of a partition to produce/consume messages. If we put a LB in front
> of all brokers which means all brokers share the same LB how does the LB
> figure out the leader?
> On Mon, Nov 21, 2016 at 10:26 PM Martin Gainty <mg...@hotmail.com>
> wrote:
>
> >
> >
> >
> >
> > ________________________________
> > From: Zac Harvey <za...@welltok.com>
> > Sent: Monday, November 21, 2016 8:59 AM
> > To: users@kafka.apache.org
> > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> >
> > Thanks again Rajini,
> >
> >
> > Using these configs, would clients connect to the load balancer over
> > SSL/9093? And then would I configure the load balancer to forward traffic
> > from SSL/9093 to plaintext/9093?
> >
> > MG>Zach
> >
> > MG>i could be wrong but SSL port != plaintext port ..but consider:
> >
> > MG>consider recent testcase where all traffic around a certain location
> > gets bogged with DOS attacks
> >
> > MG>what are the legitimate role(s) of the LB when SSL Traffic and HTTP1.1
> > Traffic and FTP Traffic are ALL blocked?
> >
> > MG>LB should never be stripping SSL headers to redirect to PlainText
> > because you are not rerouting to a faster route
> >
> > MG>most net engineers worth their salt will configure their routers to
> > static routes to loop around bogged-down routers
> >
> > MG>WDYT?
> >
> > Thanks again, just still a little uncertain about the traffic/ports
> coming
> > into the load balancer!
> >
> >
> > Best,
> >
> > Zac
> >
> > ________________________________
> > From: Rajini Sivaram <ra...@googlemail.com>
> > Sent: Monday, November 21, 2016 8:48:41 AM
> > To: users@kafka.apache.org
> > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> >
> > Zac,
> >
> > Yes, that is correct. Ruby clients will not be authenticated by Kafka.
> They
> > talk SSL to the load balancer and the load balancer uses PLAINTEXT
> without
> > authentication to talk to Kafka.
> >
> > On Mon, Nov 21, 2016 at 1:29 PM, Zac Harvey <za...@welltok.com>
> > wrote:
> >
> > > *Awesome* explanation Rajini - thank you!
> > >
> > >
> > > Just to confirm: the SASL/PLAIN configs would only be for the
> interbroker
> > > communication, correct? Meaning, beyond your recommended changes to
> > > server.properties, and the addition of the new jaas.conf file, the
> > > producers (Ruby clients) wouldn't need to authenticate, correct?
> > >
> > >
> > > Thanks again for all the great help so far, you've already helped me
> more
> > > than you know!
> > >
> > >
> > > Zac
> > >
> > > ________________________________
> > > From: Rajini Sivaram <ra...@googlemail.com>
> > > Sent: Monday, November 21, 2016 3:53:47 AM
> > > To: users@kafka.apache.org
> > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > >
> > > Zac,
> > >
> > > *advertised.listeners* is used to make client connections from
> > > producers/consumers as well as for client-side connections for
> > inter-broker
> > > communication. In your scenario, setting it to *PLAINTEXT://mykafka01*
> > > would work for inter-broker, bypassing the load balancer, but clients
> > would
> > > also then attempt to connect directly to *mykafka01*.  Setting it to
> > > *SSL://mybalancer01* would work for producers/consumers, but brokers
> > would
> > > try to connect to *mybalancer01* using PLAINTEXT. Unfortunately neither
> > > works for both. You need two endpoints, one for inter-broker that
> > bypasses
> > > *mybalancer01* and another for clients that uses *mybalancer01*. With
> the
> > > current Kafka configuration, you would require two security protocols
> to
> > > enable two endpoints.
> > >
> > > You could enable SSL in Kafka (using self-signed certificates if you
> > need)
> > > for one of the two endpoints to overcome this limitation. But
> presumably
> > > you have a secure internal network running Kafka and want to avoid the
> > cost
> > > of encryption in Kafka. The simplest solution I can think of is to use
> > > SASL_PLAINTEXT using SASL/PLAIN for inter-broker as a workaround. The
> > > configuration options in server.properties would look like:
> > >
> > > listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092
> > >
> > > advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
> > > ,SASL_PLAINTEXT://mykafka01.example.com:9092
> > >
> > > security.inter.broker.protocol=SASL_PLAINTEXT
> > >
> > > sasl.enabled.mechanisms=PLAIN
> > >
> > > sasl.mechanism.inter.broker.protocol=PLAIN
> > >
> > >
> > > You also need a JAAS configuration file configured for the broker JVM (
> > > *KAFKA_OPTS="-Djava.security.auth.login.config=/kafka/jaas.conf"*) .
> See
> > > https://kafka.apache.org/documentation#security_sasl for configuring
> > > SASL.*
> > > jaas.conf* would look something like:
> > >
> > > KafkaServer {
> > >
> > >         org.apache.kafka.common.security.plain.PlainLoginModule
> required
> > >
> > >         username="kafka"
> > >
> > >         user_kafka="kafka-password"
> > >
> > >         password="kafka-password";
> > >
> > > };
> > >
> > >
> > > Hope that helps.
> > >
> > >
> > > On Fri, Nov 18, 2016 at 6:39 PM, Zac Harvey <za...@welltok.com>
> > > wrote:
> > >
> > > > Thanks again Rajini!
> > > >
> > > >
> > > > One last followup question, if you don't mind. You said that my
> > > > server.properties file should look something like this:
> > > >
> > > >
> > > > listeners=SSL://:9093
> > > > advertised.listeners=SSL://mybalancer01.example.com:9093
> > > > security.inter.broker.protocol=SSL
> > > >
> > > > However, please remember that I'm looking for the load balancer to
> > > > terminate SSL, meaning that (my desired) communication between the
> load
> > > > balancer and Kafka would be over plaintext (not SSL).  In other
> words:
> > > >
> > > > Ruby Producers/Clients <----SSL:9093----> Load Balancer <----
> > > > Plaintext:9092 ----> Kafka
> > > >
> > > > So producers/client connect to the load balancer over SSL and port
> > 9093,
> > > > but then the load balancer communicates with Kafka over plaintext and
> > > port
> > > > 9092.
> > > >
> > > > I also don't need inter broker communication to be SSL; it can be
> > > > plaintext.
> > > >
> > > > If this is the case, do I still need to change server.properties, or
> > can
> > > I
> > > > leave it like so:
> > > >
> > > > listeners=plaintext://:9092
> > > > advertised.listeners=plaintext://mybalancer01.example.com:9092
> > > >
> > > > Or could it just be:
> > > >
> > > > listeners=plaintext://:9092
> > > > advertised.listeners=plaintext://mykafka01.example.com:9092
> > > >
> > > > Thanks again!
> > > > Zac
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > ________________________________
> > > > From: Rajini Sivaram <ra...@googlemail.com>
> > > > Sent: Friday, November 18, 2016 9:57:22 AM
> > > > To: users@kafka.apache.org
> > > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > > >
> > > > You should set advertised.listeners rather than the older
> > > > advertised.host.name property in server.properties:
> > > >
> > > >
> > > >    - listeners=SSL://:9093
> > > >    - advertised.listeners=SSL://mybalancer01.example.com:9093
> > > >    - security.inter.broker.protocol=SSL
> > > >
> > > >
> > > > If your listeners are on particular interfaces, you can set address
> in
> > > the
> > > > 'listeners' property too.
> > > >
> > > >
> > > > If you want inter-broker communication to bypass the SSL proxy, you
> > would
> > > > need another security protocol that can be used for inter-broker
> > > > communication (PLAINTEXT in the example below).
> > > >
> > > >
> > > >
> > > >    - listeners=SSL://:9093,PLAINTEXT://:9092
> > > >    - advertised.listeners=SSL://mybalancer01.example.com:9093,
> > > PLAINTEXT://
> > > >    mykafka01.example.com:9092
> > > >    - security.inter.broker.protocol=PLAINTEXT
> > > >
> > > >  I haven't used the Ruby clients, so I am not sure about client
> > > > configuration. With Java clients, if you don't specify truststore,
> the
> > > > default trust stores are used, so with trusted CA-signed
> certificates,
> > no
> > > > additional client configuration is required. You can test your
> > > installation
> > > > using the console producer and consumer that are shipped with Kafka
> to
> > > make
> > > > sure it is working before you run with Ruby clients.
> > > >
> > > >
> > > >
> > > > On Fri, Nov 18, 2016 at 1:23 PM, Zac Harvey <za...@welltok.com>
> > > > wrote:
> > > >
> > > > >
> > > > > Thanks Rajini,
> > > > >
> > > > >
> > > > > So currently one of our Kafka nodes is 'mykafka01.example.com',
> and
> > in
> > > > > its server.properties file, I have advertised.host.name=mykafka01
> > > > > .example.com. Our load balancer lives at mybalancer01.example.com,
> > and
> > > > > this what producers will connect to (over SSL) to send messages to
> > > Kafka.
> > > > >
> > > > >
> > > > > It sounds like you're saying I need to change my Kafka node's
> > > > > server.properties to have advertised.host.name=mybalance
> > > r01.example.com,
> > > > > yes? If not, can you perhaps provide a quick snippet of the
> changes I
> > > > would
> > > > > need to make to server.properties?
> > > > >
> > > > >
> > > > > Again, the cert served by the balancer will be a highly-trusted
> (root
> > > > > CA-signed) certificate that all clients will natively trust.
> > > > Interestingly
> > > > > enough, most (if not all) the Kafka producers/clients will be
> written
> > > in
> > > > > Ruby (using the zendesk Kafka-Ruby gem<https://github.com/
> > > > > zendesk/ruby-kafka>), so there wont be any JKS configuration
> options
> > > > > available for those Ruby clients.
> > > > >
> > > > >
> > > > > Besides making the change to server.properties that I mentioned
> > above,
> > > > are
> > > > > there any other client-side configs that will need to be made for
> the
> > > > Ruby
> > > > > clients to connect over SSL?
> > > > >
> > > > >
> > > > > Thank you enormously here!
> > > > >
> > > > >
> > > > > Best,
> > > > >
> > > > > Zac
> > > > >
> > > > >
> > > > > ________________________________
> > > > > From: Rajini Sivaram <ra...@googlemail.com>
> > > > > Sent: Friday, November 18, 2016 5:15:13 AM
> > > > > To: users@kafka.apache.org
> > > > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > > > >
> > > > > Zac,
> > > > >
> > > > > Kafka has its own built-in load-balancing mechanism based on
> > partition
> > > > > assignment. Requests are processed by partition leaders,
> distributing
> > > > load
> > > > > across the brokers in the cluster. If you want to put a proxy like
> > > > HAProxy
> > > > > with SSL termination in front of your brokers for added security,
> you
> > > can
> > > > > do that. You can have completely independent trust chain between
> > > > > clients->proxy and proxy->broker. You need to configure Kafka
> brokers
> > > > with
> > > > > the proxy host as the host in the advertised listeners for the
> > security
> > > > > protocol used by clients.
> > > > >
> > > > > On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <
> zac.harvey@welltok.com>
> > > > > wrote:
> > > > >
> > > > > > We have two Kafka nodes and for reasons outside of this question,
> > > would
> > > > > > like to set up a load balancer to terminate SSL with producers
> > > > (clients).
> > > > > > The SSL cert hosted by the load balancer will be signed by
> > > trusted/root
> > > > > CA
> > > > > > that clients should natively trust.
> > > > > >
> > > > > >
> > > > > > Is this possible to do, or does Kafka somehow require SSL to be
> > setup
> > > > > > directly on the Kafka servers themselves?
> > > > > >
> > > > > >
> > > > > > Thanks!
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Regards,
> > > > >
> > > > > Rajini
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Regards,
> > > >
> > > > Rajini
> > > >
> > >
> > >
> > >
> > > --
> > > Regards,
> > >
> > > Rajini
> > >
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>



--
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Rajini Sivaram <ra...@googlemail.com>.
A load balancer that balances the load across the brokers wouldn't work,
but here the LB is being used as a terminating SSL proxy. That should work
if each Kafka is configured with its own proxy.

On Mon, Nov 21, 2016 at 2:57 PM, tao xiao <xi...@gmail.com> wrote:

> I doubt the LB solution will work for Kafka. Client needs to connect to the
> leader of a partition to produce/consume messages. If we put a LB in front
> of all brokers which means all brokers share the same LB how does the LB
> figure out the leader?
> On Mon, Nov 21, 2016 at 10:26 PM Martin Gainty <mg...@hotmail.com>
> wrote:
>
> >
> >
> >
> >
> > ________________________________
> > From: Zac Harvey <za...@welltok.com>
> > Sent: Monday, November 21, 2016 8:59 AM
> > To: users@kafka.apache.org
> > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> >
> > Thanks again Rajini,
> >
> >
> > Using these configs, would clients connect to the load balancer over
> > SSL/9093? And then would I configure the load balancer to forward traffic
> > from SSL/9093 to plaintext/9093?
> >
> > MG>Zach
> >
> > MG>i could be wrong but SSL port != plaintext port ..but consider:
> >
> > MG>consider recent testcase where all traffic around a certain location
> > gets bogged with DOS attacks
> >
> > MG>what are the legitimate role(s) of the LB when SSL Traffic and HTTP1.1
> > Traffic and FTP Traffic are ALL blocked?
> >
> > MG>LB should never be stripping SSL headers to redirect to PlainText
> > because you are not rerouting to a faster route
> >
> > MG>most net engineers worth their salt will configure their routers to
> > static routes to loop around bogged-down routers
> >
> > MG>WDYT?
> >
> > Thanks again, just still a little uncertain about the traffic/ports
> coming
> > into the load balancer!
> >
> >
> > Best,
> >
> > Zac
> >
> > ________________________________
> > From: Rajini Sivaram <ra...@googlemail.com>
> > Sent: Monday, November 21, 2016 8:48:41 AM
> > To: users@kafka.apache.org
> > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> >
> > Zac,
> >
> > Yes, that is correct. Ruby clients will not be authenticated by Kafka.
> They
> > talk SSL to the load balancer and the load balancer uses PLAINTEXT
> without
> > authentication to talk to Kafka.
> >
> > On Mon, Nov 21, 2016 at 1:29 PM, Zac Harvey <za...@welltok.com>
> > wrote:
> >
> > > *Awesome* explanation Rajini - thank you!
> > >
> > >
> > > Just to confirm: the SASL/PLAIN configs would only be for the
> interbroker
> > > communication, correct? Meaning, beyond your recommended changes to
> > > server.properties, and the addition of the new jaas.conf file, the
> > > producers (Ruby clients) wouldn't need to authenticate, correct?
> > >
> > >
> > > Thanks again for all the great help so far, you've already helped me
> more
> > > than you know!
> > >
> > >
> > > Zac
> > >
> > > ________________________________
> > > From: Rajini Sivaram <ra...@googlemail.com>
> > > Sent: Monday, November 21, 2016 3:53:47 AM
> > > To: users@kafka.apache.org
> > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > >
> > > Zac,
> > >
> > > *advertised.listeners* is used to make client connections from
> > > producers/consumers as well as for client-side connections for
> > inter-broker
> > > communication. In your scenario, setting it to *PLAINTEXT://mykafka01*
> > > would work for inter-broker, bypassing the load balancer, but clients
> > would
> > > also then attempt to connect directly to *mykafka01*.  Setting it to
> > > *SSL://mybalancer01* would work for producers/consumers, but brokers
> > would
> > > try to connect to *mybalancer01* using PLAINTEXT. Unfortunately neither
> > > works for both. You need two endpoints, one for inter-broker that
> > bypasses
> > > *mybalancer01* and another for clients that uses *mybalancer01*. With
> the
> > > current Kafka configuration, you would require two security protocols
> to
> > > enable two endpoints.
> > >
> > > You could enable SSL in Kafka (using self-signed certificates if you
> > need)
> > > for one of the two endpoints to overcome this limitation. But
> presumably
> > > you have a secure internal network running Kafka and want to avoid the
> > cost
> > > of encryption in Kafka. The simplest solution I can think of is to use
> > > SASL_PLAINTEXT using SASL/PLAIN for inter-broker as a workaround. The
> > > configuration options in server.properties would look like:
> > >
> > > listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092
> > >
> > > advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
> > > ,SASL_PLAINTEXT://mykafka01.example.com:9092
> > >
> > > security.inter.broker.protocol=SASL_PLAINTEXT
> > >
> > > sasl.enabled.mechanisms=PLAIN
> > >
> > > sasl.mechanism.inter.broker.protocol=PLAIN
> > >
> > >
> > > You also need a JAAS configuration file configured for the broker JVM (
> > > *KAFKA_OPTS="-Djava.security.auth.login.config=/kafka/jaas.conf"*) .
> See
> > > https://kafka.apache.org/documentation#security_sasl for configuring
> > > SASL.*
> > > jaas.conf* would look something like:
> > >
> > > KafkaServer {
> > >
> > >         org.apache.kafka.common.security.plain.PlainLoginModule
> required
> > >
> > >         username="kafka"
> > >
> > >         user_kafka="kafka-password"
> > >
> > >         password="kafka-password";
> > >
> > > };
> > >
> > >
> > > Hope that helps.
> > >
> > >
> > > On Fri, Nov 18, 2016 at 6:39 PM, Zac Harvey <za...@welltok.com>
> > > wrote:
> > >
> > > > Thanks again Rajini!
> > > >
> > > >
> > > > One last followup question, if you don't mind. You said that my
> > > > server.properties file should look something like this:
> > > >
> > > >
> > > > listeners=SSL://:9093
> > > > advertised.listeners=SSL://mybalancer01.example.com:9093
> > > > security.inter.broker.protocol=SSL
> > > >
> > > > However, please remember that I'm looking for the load balancer to
> > > > terminate SSL, meaning that (my desired) communication between the
> load
> > > > balancer and Kafka would be over plaintext (not SSL).  In other
> words:
> > > >
> > > > Ruby Producers/Clients <----SSL:9093----> Load Balancer <----
> > > > Plaintext:9092 ----> Kafka
> > > >
> > > > So producers/client connect to the load balancer over SSL and port
> > 9093,
> > > > but then the load balancer communicates with Kafka over plaintext and
> > > port
> > > > 9092.
> > > >
> > > > I also don't need inter broker communication to be SSL; it can be
> > > > plaintext.
> > > >
> > > > If this is the case, do I still need to change server.properties, or
> > can
> > > I
> > > > leave it like so:
> > > >
> > > > listeners=plaintext://:9092
> > > > advertised.listeners=plaintext://mybalancer01.example.com:9092
> > > >
> > > > Or could it just be:
> > > >
> > > > listeners=plaintext://:9092
> > > > advertised.listeners=plaintext://mykafka01.example.com:9092
> > > >
> > > > Thanks again!
> > > > Zac
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > ________________________________
> > > > From: Rajini Sivaram <ra...@googlemail.com>
> > > > Sent: Friday, November 18, 2016 9:57:22 AM
> > > > To: users@kafka.apache.org
> > > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > > >
> > > > You should set advertised.listeners rather than the older
> > > > advertised.host.name property in server.properties:
> > > >
> > > >
> > > >    - listeners=SSL://:9093
> > > >    - advertised.listeners=SSL://mybalancer01.example.com:9093
> > > >    - security.inter.broker.protocol=SSL
> > > >
> > > >
> > > > If your listeners are on particular interfaces, you can set address
> in
> > > the
> > > > 'listeners' property too.
> > > >
> > > >
> > > > If you want inter-broker communication to bypass the SSL proxy, you
> > would
> > > > need another security protocol that can be used for inter-broker
> > > > communication (PLAINTEXT in the example below).
> > > >
> > > >
> > > >
> > > >    - listeners=SSL://:9093,PLAINTEXT://:9092
> > > >    - advertised.listeners=SSL://mybalancer01.example.com:9093,
> > > PLAINTEXT://
> > > >    mykafka01.example.com:9092
> > > >    - security.inter.broker.protocol=PLAINTEXT
> > > >
> > > >  I haven't used the Ruby clients, so I am not sure about client
> > > > configuration. With Java clients, if you don't specify truststore,
> the
> > > > default trust stores are used, so with trusted CA-signed
> certificates,
> > no
> > > > additional client configuration is required. You can test your
> > > installation
> > > > using the console producer and consumer that are shipped with Kafka
> to
> > > make
> > > > sure it is working before you run with Ruby clients.
> > > >
> > > >
> > > >
> > > > On Fri, Nov 18, 2016 at 1:23 PM, Zac Harvey <za...@welltok.com>
> > > > wrote:
> > > >
> > > > >
> > > > > Thanks Rajini,
> > > > >
> > > > >
> > > > > So currently one of our Kafka nodes is 'mykafka01.example.com',
> and
> > in
> > > > > its server.properties file, I have advertised.host.name=mykafka01
> > > > > .example.com. Our load balancer lives at mybalancer01.example.com,
> > and
> > > > > this what producers will connect to (over SSL) to send messages to
> > > Kafka.
> > > > >
> > > > >
> > > > > It sounds like you're saying I need to change my Kafka node's
> > > > > server.properties to have advertised.host.name=mybalance
> > > r01.example.com,
> > > > > yes? If not, can you perhaps provide a quick snippet of the
> changes I
> > > > would
> > > > > need to make to server.properties?
> > > > >
> > > > >
> > > > > Again, the cert served by the balancer will be a highly-trusted
> (root
> > > > > CA-signed) certificate that all clients will natively trust.
> > > > Interestingly
> > > > > enough, most (if not all) the Kafka producers/clients will be
> written
> > > in
> > > > > Ruby (using the zendesk Kafka-Ruby gem<https://github.com/
> > > > > zendesk/ruby-kafka>), so there wont be any JKS configuration
> options
> > > > > available for those Ruby clients.
> > > > >
> > > > >
> > > > > Besides making the change to server.properties that I mentioned
> > above,
> > > > are
> > > > > there any other client-side configs that will need to be made for
> the
> > > > Ruby
> > > > > clients to connect over SSL?
> > > > >
> > > > >
> > > > > Thank you enormously here!
> > > > >
> > > > >
> > > > > Best,
> > > > >
> > > > > Zac
> > > > >
> > > > >
> > > > > ________________________________
> > > > > From: Rajini Sivaram <ra...@googlemail.com>
> > > > > Sent: Friday, November 18, 2016 5:15:13 AM
> > > > > To: users@kafka.apache.org
> > > > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > > > >
> > > > > Zac,
> > > > >
> > > > > Kafka has its own built-in load-balancing mechanism based on
> > partition
> > > > > assignment. Requests are processed by partition leaders,
> distributing
> > > > load
> > > > > across the brokers in the cluster. If you want to put a proxy like
> > > > HAProxy
> > > > > with SSL termination in front of your brokers for added security,
> you
> > > can
> > > > > do that. You can have completely independent trust chain between
> > > > > clients->proxy and proxy->broker. You need to configure Kafka
> brokers
> > > > with
> > > > > the proxy host as the host in the advertised listeners for the
> > security
> > > > > protocol used by clients.
> > > > >
> > > > > On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <
> zac.harvey@welltok.com>
> > > > > wrote:
> > > > >
> > > > > > We have two Kafka nodes and for reasons outside of this question,
> > > would
> > > > > > like to set up a load balancer to terminate SSL with producers
> > > > (clients).
> > > > > > The SSL cert hosted by the load balancer will be signed by
> > > trusted/root
> > > > > CA
> > > > > > that clients should natively trust.
> > > > > >
> > > > > >
> > > > > > Is this possible to do, or does Kafka somehow require SSL to be
> > setup
> > > > > > directly on the Kafka servers themselves?
> > > > > >
> > > > > >
> > > > > > Thanks!
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Regards,
> > > > >
> > > > > Rajini
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Regards,
> > > >
> > > > Rajini
> > > >
> > >
> > >
> > >
> > > --
> > > Regards,
> > >
> > > Rajini
> > >
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>



-- 
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by tao xiao <xi...@gmail.com>.
I doubt the LB solution will work for Kafka. Client needs to connect to the
leader of a partition to produce/consume messages. If we put a LB in front
of all brokers which means all brokers share the same LB how does the LB
figure out the leader?
On Mon, Nov 21, 2016 at 10:26 PM Martin Gainty <mg...@hotmail.com> wrote:

>
>
>
>
> ________________________________
> From: Zac Harvey <za...@welltok.com>
> Sent: Monday, November 21, 2016 8:59 AM
> To: users@kafka.apache.org
> Subject: Re: Can Kafka/SSL be terminated at a load balancer?
>
> Thanks again Rajini,
>
>
> Using these configs, would clients connect to the load balancer over
> SSL/9093? And then would I configure the load balancer to forward traffic
> from SSL/9093 to plaintext/9093?
>
> MG>Zach
>
> MG>i could be wrong but SSL port != plaintext port ..but consider:
>
> MG>consider recent testcase where all traffic around a certain location
> gets bogged with DOS attacks
>
> MG>what are the legitimate role(s) of the LB when SSL Traffic and HTTP1.1
> Traffic and FTP Traffic are ALL blocked?
>
> MG>LB should never be stripping SSL headers to redirect to PlainText
> because you are not rerouting to a faster route
>
> MG>most net engineers worth their salt will configure their routers to
> static routes to loop around bogged-down routers
>
> MG>WDYT?
>
> Thanks again, just still a little uncertain about the traffic/ports coming
> into the load balancer!
>
>
> Best,
>
> Zac
>
> ________________________________
> From: Rajini Sivaram <ra...@googlemail.com>
> Sent: Monday, November 21, 2016 8:48:41 AM
> To: users@kafka.apache.org
> Subject: Re: Can Kafka/SSL be terminated at a load balancer?
>
> Zac,
>
> Yes, that is correct. Ruby clients will not be authenticated by Kafka. They
> talk SSL to the load balancer and the load balancer uses PLAINTEXT without
> authentication to talk to Kafka.
>
> On Mon, Nov 21, 2016 at 1:29 PM, Zac Harvey <za...@welltok.com>
> wrote:
>
> > *Awesome* explanation Rajini - thank you!
> >
> >
> > Just to confirm: the SASL/PLAIN configs would only be for the interbroker
> > communication, correct? Meaning, beyond your recommended changes to
> > server.properties, and the addition of the new jaas.conf file, the
> > producers (Ruby clients) wouldn't need to authenticate, correct?
> >
> >
> > Thanks again for all the great help so far, you've already helped me more
> > than you know!
> >
> >
> > Zac
> >
> > ________________________________
> > From: Rajini Sivaram <ra...@googlemail.com>
> > Sent: Monday, November 21, 2016 3:53:47 AM
> > To: users@kafka.apache.org
> > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> >
> > Zac,
> >
> > *advertised.listeners* is used to make client connections from
> > producers/consumers as well as for client-side connections for
> inter-broker
> > communication. In your scenario, setting it to *PLAINTEXT://mykafka01*
> > would work for inter-broker, bypassing the load balancer, but clients
> would
> > also then attempt to connect directly to *mykafka01*.  Setting it to
> > *SSL://mybalancer01* would work for producers/consumers, but brokers
> would
> > try to connect to *mybalancer01* using PLAINTEXT. Unfortunately neither
> > works for both. You need two endpoints, one for inter-broker that
> bypasses
> > *mybalancer01* and another for clients that uses *mybalancer01*. With the
> > current Kafka configuration, you would require two security protocols to
> > enable two endpoints.
> >
> > You could enable SSL in Kafka (using self-signed certificates if you
> need)
> > for one of the two endpoints to overcome this limitation. But presumably
> > you have a secure internal network running Kafka and want to avoid the
> cost
> > of encryption in Kafka. The simplest solution I can think of is to use
> > SASL_PLAINTEXT using SASL/PLAIN for inter-broker as a workaround. The
> > configuration options in server.properties would look like:
> >
> > listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092
> >
> > advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
> > ,SASL_PLAINTEXT://mykafka01.example.com:9092
> >
> > security.inter.broker.protocol=SASL_PLAINTEXT
> >
> > sasl.enabled.mechanisms=PLAIN
> >
> > sasl.mechanism.inter.broker.protocol=PLAIN
> >
> >
> > You also need a JAAS configuration file configured for the broker JVM (
> > *KAFKA_OPTS="-Djava.security.auth.login.config=/kafka/jaas.conf"*) . See
> > https://kafka.apache.org/documentation#security_sasl for configuring
> > SASL.*
> > jaas.conf* would look something like:
> >
> > KafkaServer {
> >
> >         org.apache.kafka.common.security.plain.PlainLoginModule required
> >
> >         username="kafka"
> >
> >         user_kafka="kafka-password"
> >
> >         password="kafka-password";
> >
> > };
> >
> >
> > Hope that helps.
> >
> >
> > On Fri, Nov 18, 2016 at 6:39 PM, Zac Harvey <za...@welltok.com>
> > wrote:
> >
> > > Thanks again Rajini!
> > >
> > >
> > > One last followup question, if you don't mind. You said that my
> > > server.properties file should look something like this:
> > >
> > >
> > > listeners=SSL://:9093
> > > advertised.listeners=SSL://mybalancer01.example.com:9093
> > > security.inter.broker.protocol=SSL
> > >
> > > However, please remember that I'm looking for the load balancer to
> > > terminate SSL, meaning that (my desired) communication between the load
> > > balancer and Kafka would be over plaintext (not SSL).  In other words:
> > >
> > > Ruby Producers/Clients <----SSL:9093----> Load Balancer <----
> > > Plaintext:9092 ----> Kafka
> > >
> > > So producers/client connect to the load balancer over SSL and port
> 9093,
> > > but then the load balancer communicates with Kafka over plaintext and
> > port
> > > 9092.
> > >
> > > I also don't need inter broker communication to be SSL; it can be
> > > plaintext.
> > >
> > > If this is the case, do I still need to change server.properties, or
> can
> > I
> > > leave it like so:
> > >
> > > listeners=plaintext://:9092
> > > advertised.listeners=plaintext://mybalancer01.example.com:9092
> > >
> > > Or could it just be:
> > >
> > > listeners=plaintext://:9092
> > > advertised.listeners=plaintext://mykafka01.example.com:9092
> > >
> > > Thanks again!
> > > Zac
> > >
> > >
> > >
> > >
> > >
> > > ________________________________
> > > From: Rajini Sivaram <ra...@googlemail.com>
> > > Sent: Friday, November 18, 2016 9:57:22 AM
> > > To: users@kafka.apache.org
> > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > >
> > > You should set advertised.listeners rather than the older
> > > advertised.host.name property in server.properties:
> > >
> > >
> > >    - listeners=SSL://:9093
> > >    - advertised.listeners=SSL://mybalancer01.example.com:9093
> > >    - security.inter.broker.protocol=SSL
> > >
> > >
> > > If your listeners are on particular interfaces, you can set address in
> > the
> > > 'listeners' property too.
> > >
> > >
> > > If you want inter-broker communication to bypass the SSL proxy, you
> would
> > > need another security protocol that can be used for inter-broker
> > > communication (PLAINTEXT in the example below).
> > >
> > >
> > >
> > >    - listeners=SSL://:9093,PLAINTEXT://:9092
> > >    - advertised.listeners=SSL://mybalancer01.example.com:9093,
> > PLAINTEXT://
> > >    mykafka01.example.com:9092
> > >    - security.inter.broker.protocol=PLAINTEXT
> > >
> > >  I haven't used the Ruby clients, so I am not sure about client
> > > configuration. With Java clients, if you don't specify truststore, the
> > > default trust stores are used, so with trusted CA-signed certificates,
> no
> > > additional client configuration is required. You can test your
> > installation
> > > using the console producer and consumer that are shipped with Kafka to
> > make
> > > sure it is working before you run with Ruby clients.
> > >
> > >
> > >
> > > On Fri, Nov 18, 2016 at 1:23 PM, Zac Harvey <za...@welltok.com>
> > > wrote:
> > >
> > > >
> > > > Thanks Rajini,
> > > >
> > > >
> > > > So currently one of our Kafka nodes is 'mykafka01.example.com', and
> in
> > > > its server.properties file, I have advertised.host.name=mykafka01
> > > > .example.com. Our load balancer lives at mybalancer01.example.com,
> and
> > > > this what producers will connect to (over SSL) to send messages to
> > Kafka.
> > > >
> > > >
> > > > It sounds like you're saying I need to change my Kafka node's
> > > > server.properties to have advertised.host.name=mybalance
> > r01.example.com,
> > > > yes? If not, can you perhaps provide a quick snippet of the changes I
> > > would
> > > > need to make to server.properties?
> > > >
> > > >
> > > > Again, the cert served by the balancer will be a highly-trusted (root
> > > > CA-signed) certificate that all clients will natively trust.
> > > Interestingly
> > > > enough, most (if not all) the Kafka producers/clients will be written
> > in
> > > > Ruby (using the zendesk Kafka-Ruby gem<https://github.com/
> > > > zendesk/ruby-kafka>), so there wont be any JKS configuration options
> > > > available for those Ruby clients.
> > > >
> > > >
> > > > Besides making the change to server.properties that I mentioned
> above,
> > > are
> > > > there any other client-side configs that will need to be made for the
> > > Ruby
> > > > clients to connect over SSL?
> > > >
> > > >
> > > > Thank you enormously here!
> > > >
> > > >
> > > > Best,
> > > >
> > > > Zac
> > > >
> > > >
> > > > ________________________________
> > > > From: Rajini Sivaram <ra...@googlemail.com>
> > > > Sent: Friday, November 18, 2016 5:15:13 AM
> > > > To: users@kafka.apache.org
> > > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > > >
> > > > Zac,
> > > >
> > > > Kafka has its own built-in load-balancing mechanism based on
> partition
> > > > assignment. Requests are processed by partition leaders, distributing
> > > load
> > > > across the brokers in the cluster. If you want to put a proxy like
> > > HAProxy
> > > > with SSL termination in front of your brokers for added security, you
> > can
> > > > do that. You can have completely independent trust chain between
> > > > clients->proxy and proxy->broker. You need to configure Kafka brokers
> > > with
> > > > the proxy host as the host in the advertised listeners for the
> security
> > > > protocol used by clients.
> > > >
> > > > On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <za...@welltok.com>
> > > > wrote:
> > > >
> > > > > We have two Kafka nodes and for reasons outside of this question,
> > would
> > > > > like to set up a load balancer to terminate SSL with producers
> > > (clients).
> > > > > The SSL cert hosted by the load balancer will be signed by
> > trusted/root
> > > > CA
> > > > > that clients should natively trust.
> > > > >
> > > > >
> > > > > Is this possible to do, or does Kafka somehow require SSL to be
> setup
> > > > > directly on the Kafka servers themselves?
> > > > >
> > > > >
> > > > > Thanks!
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Regards,
> > > >
> > > > Rajini
> > > >
> > >
> > >
> > >
> > > --
> > > Regards,
> > >
> > > Rajini
> > >
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>
>
>
> --
> Regards,
>
> Rajini
>

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Martin Gainty <mg...@hotmail.com>.



________________________________
From: Zac Harvey <za...@welltok.com>
Sent: Monday, November 21, 2016 8:59 AM
To: users@kafka.apache.org
Subject: Re: Can Kafka/SSL be terminated at a load balancer?

Thanks again Rajini,


Using these configs, would clients connect to the load balancer over SSL/9093? And then would I configure the load balancer to forward traffic from SSL/9093 to plaintext/9093?

MG>Zach

MG>i could be wrong but SSL port != plaintext port ..but consider:

MG>consider recent testcase where all traffic around a certain location gets bogged with DOS attacks

MG>what are the legitimate role(s) of the LB when SSL Traffic and HTTP1.1 Traffic and FTP Traffic are ALL blocked?

MG>LB should never be stripping SSL headers to redirect to PlainText because you are not rerouting to a faster route

MG>most net engineers worth their salt will configure their routers to static routes to loop around bogged-down routers

MG>WDYT?

Thanks again, just still a little uncertain about the traffic/ports coming into the load balancer!


Best,

Zac

________________________________
From: Rajini Sivaram <ra...@googlemail.com>
Sent: Monday, November 21, 2016 8:48:41 AM
To: users@kafka.apache.org
Subject: Re: Can Kafka/SSL be terminated at a load balancer?

Zac,

Yes, that is correct. Ruby clients will not be authenticated by Kafka. They
talk SSL to the load balancer and the load balancer uses PLAINTEXT without
authentication to talk to Kafka.

On Mon, Nov 21, 2016 at 1:29 PM, Zac Harvey <za...@welltok.com> wrote:

> *Awesome* explanation Rajini - thank you!
>
>
> Just to confirm: the SASL/PLAIN configs would only be for the interbroker
> communication, correct? Meaning, beyond your recommended changes to
> server.properties, and the addition of the new jaas.conf file, the
> producers (Ruby clients) wouldn't need to authenticate, correct?
>
>
> Thanks again for all the great help so far, you've already helped me more
> than you know!
>
>
> Zac
>
> ________________________________
> From: Rajini Sivaram <ra...@googlemail.com>
> Sent: Monday, November 21, 2016 3:53:47 AM
> To: users@kafka.apache.org
> Subject: Re: Can Kafka/SSL be terminated at a load balancer?
>
> Zac,
>
> *advertised.listeners* is used to make client connections from
> producers/consumers as well as for client-side connections for inter-broker
> communication. In your scenario, setting it to *PLAINTEXT://mykafka01*
> would work for inter-broker, bypassing the load balancer, but clients would
> also then attempt to connect directly to *mykafka01*.  Setting it to
> *SSL://mybalancer01* would work for producers/consumers, but brokers would
> try to connect to *mybalancer01* using PLAINTEXT. Unfortunately neither
> works for both. You need two endpoints, one for inter-broker that bypasses
> *mybalancer01* and another for clients that uses *mybalancer01*. With the
> current Kafka configuration, you would require two security protocols to
> enable two endpoints.
>
> You could enable SSL in Kafka (using self-signed certificates if you need)
> for one of the two endpoints to overcome this limitation. But presumably
> you have a secure internal network running Kafka and want to avoid the cost
> of encryption in Kafka. The simplest solution I can think of is to use
> SASL_PLAINTEXT using SASL/PLAIN for inter-broker as a workaround. The
> configuration options in server.properties would look like:
>
> listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092
>
> advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
> ,SASL_PLAINTEXT://mykafka01.example.com:9092
>
> security.inter.broker.protocol=SASL_PLAINTEXT
>
> sasl.enabled.mechanisms=PLAIN
>
> sasl.mechanism.inter.broker.protocol=PLAIN
>
>
> You also need a JAAS configuration file configured for the broker JVM (
> *KAFKA_OPTS="-Djava.security.auth.login.config=/kafka/jaas.conf"*) . See
> https://kafka.apache.org/documentation#security_sasl for configuring
> SASL.*
> jaas.conf* would look something like:
>
> KafkaServer {
>
>         org.apache.kafka.common.security.plain.PlainLoginModule required
>
>         username="kafka"
>
>         user_kafka="kafka-password"
>
>         password="kafka-password";
>
> };
>
>
> Hope that helps.
>
>
> On Fri, Nov 18, 2016 at 6:39 PM, Zac Harvey <za...@welltok.com>
> wrote:
>
> > Thanks again Rajini!
> >
> >
> > One last followup question, if you don't mind. You said that my
> > server.properties file should look something like this:
> >
> >
> > listeners=SSL://:9093
> > advertised.listeners=SSL://mybalancer01.example.com:9093
> > security.inter.broker.protocol=SSL
> >
> > However, please remember that I'm looking for the load balancer to
> > terminate SSL, meaning that (my desired) communication between the load
> > balancer and Kafka would be over plaintext (not SSL).  In other words:
> >
> > Ruby Producers/Clients <----SSL:9093----> Load Balancer <----
> > Plaintext:9092 ----> Kafka
> >
> > So producers/client connect to the load balancer over SSL and port 9093,
> > but then the load balancer communicates with Kafka over plaintext and
> port
> > 9092.
> >
> > I also don't need inter broker communication to be SSL; it can be
> > plaintext.
> >
> > If this is the case, do I still need to change server.properties, or can
> I
> > leave it like so:
> >
> > listeners=plaintext://:9092
> > advertised.listeners=plaintext://mybalancer01.example.com:9092
> >
> > Or could it just be:
> >
> > listeners=plaintext://:9092
> > advertised.listeners=plaintext://mykafka01.example.com:9092
> >
> > Thanks again!
> > Zac
> >
> >
> >
> >
> >
> > ________________________________
> > From: Rajini Sivaram <ra...@googlemail.com>
> > Sent: Friday, November 18, 2016 9:57:22 AM
> > To: users@kafka.apache.org
> > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> >
> > You should set advertised.listeners rather than the older
> > advertised.host.name property in server.properties:
> >
> >
> >    - listeners=SSL://:9093
> >    - advertised.listeners=SSL://mybalancer01.example.com:9093
> >    - security.inter.broker.protocol=SSL
> >
> >
> > If your listeners are on particular interfaces, you can set address in
> the
> > 'listeners' property too.
> >
> >
> > If you want inter-broker communication to bypass the SSL proxy, you would
> > need another security protocol that can be used for inter-broker
> > communication (PLAINTEXT in the example below).
> >
> >
> >
> >    - listeners=SSL://:9093,PLAINTEXT://:9092
> >    - advertised.listeners=SSL://mybalancer01.example.com:9093,
> PLAINTEXT://
> >    mykafka01.example.com:9092
> >    - security.inter.broker.protocol=PLAINTEXT
> >
> >  I haven't used the Ruby clients, so I am not sure about client
> > configuration. With Java clients, if you don't specify truststore, the
> > default trust stores are used, so with trusted CA-signed certificates, no
> > additional client configuration is required. You can test your
> installation
> > using the console producer and consumer that are shipped with Kafka to
> make
> > sure it is working before you run with Ruby clients.
> >
> >
> >
> > On Fri, Nov 18, 2016 at 1:23 PM, Zac Harvey <za...@welltok.com>
> > wrote:
> >
> > >
> > > Thanks Rajini,
> > >
> > >
> > > So currently one of our Kafka nodes is 'mykafka01.example.com', and in
> > > its server.properties file, I have advertised.host.name=mykafka01
> > > .example.com. Our load balancer lives at mybalancer01.example.com, and
> > > this what producers will connect to (over SSL) to send messages to
> Kafka.
> > >
> > >
> > > It sounds like you're saying I need to change my Kafka node's
> > > server.properties to have advertised.host.name=mybalance
> r01.example.com,
> > > yes? If not, can you perhaps provide a quick snippet of the changes I
> > would
> > > need to make to server.properties?
> > >
> > >
> > > Again, the cert served by the balancer will be a highly-trusted (root
> > > CA-signed) certificate that all clients will natively trust.
> > Interestingly
> > > enough, most (if not all) the Kafka producers/clients will be written
> in
> > > Ruby (using the zendesk Kafka-Ruby gem<https://github.com/
> > > zendesk/ruby-kafka>), so there wont be any JKS configuration options
> > > available for those Ruby clients.
> > >
> > >
> > > Besides making the change to server.properties that I mentioned above,
> > are
> > > there any other client-side configs that will need to be made for the
> > Ruby
> > > clients to connect over SSL?
> > >
> > >
> > > Thank you enormously here!
> > >
> > >
> > > Best,
> > >
> > > Zac
> > >
> > >
> > > ________________________________
> > > From: Rajini Sivaram <ra...@googlemail.com>
> > > Sent: Friday, November 18, 2016 5:15:13 AM
> > > To: users@kafka.apache.org
> > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > >
> > > Zac,
> > >
> > > Kafka has its own built-in load-balancing mechanism based on partition
> > > assignment. Requests are processed by partition leaders, distributing
> > load
> > > across the brokers in the cluster. If you want to put a proxy like
> > HAProxy
> > > with SSL termination in front of your brokers for added security, you
> can
> > > do that. You can have completely independent trust chain between
> > > clients->proxy and proxy->broker. You need to configure Kafka brokers
> > with
> > > the proxy host as the host in the advertised listeners for the security
> > > protocol used by clients.
> > >
> > > On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <za...@welltok.com>
> > > wrote:
> > >
> > > > We have two Kafka nodes and for reasons outside of this question,
> would
> > > > like to set up a load balancer to terminate SSL with producers
> > (clients).
> > > > The SSL cert hosted by the load balancer will be signed by
> trusted/root
> > > CA
> > > > that clients should natively trust.
> > > >
> > > >
> > > > Is this possible to do, or does Kafka somehow require SSL to be setup
> > > > directly on the Kafka servers themselves?
> > > >
> > > >
> > > > Thanks!
> > > >
> > >
> > >
> > >
> > > --
> > > Regards,
> > >
> > > Rajini
> > >
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>
>
>
> --
> Regards,
>
> Rajini
>



--
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Rajini Sivaram <ra...@googlemail.com>.
Zac,

Yes, that is correct.

With the configuration:

listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092

advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
,SASL_PLAINTEXT://mykafka01.example.com:9092



   - Clients talk to port 9093 on load balancer using SSL.
   - Load balancer talks to port 9093 on Kafka brokers using PLAINTEXT
   (that is config you need to add on the load balancer)
   - Brokers talk to each other for inter-broker comms on port 9092 using
   SASL_PLAINTEXT

The connections for the two cases are:
*RubyClient  * <=== SSL ===>  *Load balancer (mybalancer01:9093)* <====
PLAINTEXT ======> *KafkaBroker (mykafka01:9093)*
*KafkaBroker (mykafka02:9092) *<==== SASL_PLAINTEXT ======> *KafkaBroker
(mykafka01:9092)*

You can use different ports on Kafka if you find that using 9093 for SSL on
one side and PLAINTEXT on the other is confusing.


On Mon, Nov 21, 2016 at 1:59 PM, Zac Harvey <za...@welltok.com> wrote:

> Thanks again Rajini,
>
>
> Using these configs, would clients connect to the load balancer over
> SSL/9093? And then would I configure the load balancer to forward traffic
> from SSL/9093 to plaintext/9093?
>
>
> Thanks again, just still a little uncertain about the traffic/ports coming
> into the load balancer!
>
>
> Best,
>
> Zac
>
> ________________________________
> From: Rajini Sivaram <ra...@googlemail.com>
> Sent: Monday, November 21, 2016 8:48:41 AM
> To: users@kafka.apache.org
> Subject: Re: Can Kafka/SSL be terminated at a load balancer?
>
> Zac,
>
> Yes, that is correct. Ruby clients will not be authenticated by Kafka. They
> talk SSL to the load balancer and the load balancer uses PLAINTEXT without
> authentication to talk to Kafka.
>
> On Mon, Nov 21, 2016 at 1:29 PM, Zac Harvey <za...@welltok.com>
> wrote:
>
> > *Awesome* explanation Rajini - thank you!
> >
> >
> > Just to confirm: the SASL/PLAIN configs would only be for the interbroker
> > communication, correct? Meaning, beyond your recommended changes to
> > server.properties, and the addition of the new jaas.conf file, the
> > producers (Ruby clients) wouldn't need to authenticate, correct?
> >
> >
> > Thanks again for all the great help so far, you've already helped me more
> > than you know!
> >
> >
> > Zac
> >
> > ________________________________
> > From: Rajini Sivaram <ra...@googlemail.com>
> > Sent: Monday, November 21, 2016 3:53:47 AM
> > To: users@kafka.apache.org
> > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> >
> > Zac,
> >
> > *advertised.listeners* is used to make client connections from
> > producers/consumers as well as for client-side connections for
> inter-broker
> > communication. In your scenario, setting it to *PLAINTEXT://mykafka01*
> > would work for inter-broker, bypassing the load balancer, but clients
> would
> > also then attempt to connect directly to *mykafka01*.  Setting it to
> > *SSL://mybalancer01* would work for producers/consumers, but brokers
> would
> > try to connect to *mybalancer01* using PLAINTEXT. Unfortunately neither
> > works for both. You need two endpoints, one for inter-broker that
> bypasses
> > *mybalancer01* and another for clients that uses *mybalancer01*. With the
> > current Kafka configuration, you would require two security protocols to
> > enable two endpoints.
> >
> > You could enable SSL in Kafka (using self-signed certificates if you
> need)
> > for one of the two endpoints to overcome this limitation. But presumably
> > you have a secure internal network running Kafka and want to avoid the
> cost
> > of encryption in Kafka. The simplest solution I can think of is to use
> > SASL_PLAINTEXT using SASL/PLAIN for inter-broker as a workaround. The
> > configuration options in server.properties would look like:
> >
> > listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092
> >
> > advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
> > ,SASL_PLAINTEXT://mykafka01.example.com:9092
> >
> > security.inter.broker.protocol=SASL_PLAINTEXT
> >
> > sasl.enabled.mechanisms=PLAIN
> >
> > sasl.mechanism.inter.broker.protocol=PLAIN
> >
> >
> > You also need a JAAS configuration file configured for the broker JVM (
> > *KAFKA_OPTS="-Djava.security.auth.login.config=/kafka/jaas.conf"*) . See
> > https://kafka.apache.org/documentation#security_sasl for configuring
> > SASL.*
> > jaas.conf* would look something like:
> >
> > KafkaServer {
> >
> >         org.apache.kafka.common.security.plain.PlainLoginModule required
> >
> >         username="kafka"
> >
> >         user_kafka="kafka-password"
> >
> >         password="kafka-password";
> >
> > };
> >
> >
> > Hope that helps.
> >
> >
> > On Fri, Nov 18, 2016 at 6:39 PM, Zac Harvey <za...@welltok.com>
> > wrote:
> >
> > > Thanks again Rajini!
> > >
> > >
> > > One last followup question, if you don't mind. You said that my
> > > server.properties file should look something like this:
> > >
> > >
> > > listeners=SSL://:9093
> > > advertised.listeners=SSL://mybalancer01.example.com:9093
> > > security.inter.broker.protocol=SSL
> > >
> > > However, please remember that I'm looking for the load balancer to
> > > terminate SSL, meaning that (my desired) communication between the load
> > > balancer and Kafka would be over plaintext (not SSL).  In other words:
> > >
> > > Ruby Producers/Clients <----SSL:9093----> Load Balancer <----
> > > Plaintext:9092 ----> Kafka
> > >
> > > So producers/client connect to the load balancer over SSL and port
> 9093,
> > > but then the load balancer communicates with Kafka over plaintext and
> > port
> > > 9092.
> > >
> > > I also don't need inter broker communication to be SSL; it can be
> > > plaintext.
> > >
> > > If this is the case, do I still need to change server.properties, or
> can
> > I
> > > leave it like so:
> > >
> > > listeners=plaintext://:9092
> > > advertised.listeners=plaintext://mybalancer01.example.com:9092
> > >
> > > Or could it just be:
> > >
> > > listeners=plaintext://:9092
> > > advertised.listeners=plaintext://mykafka01.example.com:9092
> > >
> > > Thanks again!
> > > Zac
> > >
> > >
> > >
> > >
> > >
> > > ________________________________
> > > From: Rajini Sivaram <ra...@googlemail.com>
> > > Sent: Friday, November 18, 2016 9:57:22 AM
> > > To: users@kafka.apache.org
> > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > >
> > > You should set advertised.listeners rather than the older
> > > advertised.host.name property in server.properties:
> > >
> > >
> > >    - listeners=SSL://:9093
> > >    - advertised.listeners=SSL://mybalancer01.example.com:9093
> > >    - security.inter.broker.protocol=SSL
> > >
> > >
> > > If your listeners are on particular interfaces, you can set address in
> > the
> > > 'listeners' property too.
> > >
> > >
> > > If you want inter-broker communication to bypass the SSL proxy, you
> would
> > > need another security protocol that can be used for inter-broker
> > > communication (PLAINTEXT in the example below).
> > >
> > >
> > >
> > >    - listeners=SSL://:9093,PLAINTEXT://:9092
> > >    - advertised.listeners=SSL://mybalancer01.example.com:9093,
> > PLAINTEXT://
> > >    mykafka01.example.com:9092
> > >    - security.inter.broker.protocol=PLAINTEXT
> > >
> > >  I haven't used the Ruby clients, so I am not sure about client
> > > configuration. With Java clients, if you don't specify truststore, the
> > > default trust stores are used, so with trusted CA-signed certificates,
> no
> > > additional client configuration is required. You can test your
> > installation
> > > using the console producer and consumer that are shipped with Kafka to
> > make
> > > sure it is working before you run with Ruby clients.
> > >
> > >
> > >
> > > On Fri, Nov 18, 2016 at 1:23 PM, Zac Harvey <za...@welltok.com>
> > > wrote:
> > >
> > > >
> > > > Thanks Rajini,
> > > >
> > > >
> > > > So currently one of our Kafka nodes is 'mykafka01.example.com', and
> in
> > > > its server.properties file, I have advertised.host.name=mykafka01
> > > > .example.com. Our load balancer lives at mybalancer01.example.com,
> and
> > > > this what producers will connect to (over SSL) to send messages to
> > Kafka.
> > > >
> > > >
> > > > It sounds like you're saying I need to change my Kafka node's
> > > > server.properties to have advertised.host.name=mybalance
> > r01.example.com,
> > > > yes? If not, can you perhaps provide a quick snippet of the changes I
> > > would
> > > > need to make to server.properties?
> > > >
> > > >
> > > > Again, the cert served by the balancer will be a highly-trusted (root
> > > > CA-signed) certificate that all clients will natively trust.
> > > Interestingly
> > > > enough, most (if not all) the Kafka producers/clients will be written
> > in
> > > > Ruby (using the zendesk Kafka-Ruby gem<https://github.com/
> > > > zendesk/ruby-kafka>), so there wont be any JKS configuration options
> > > > available for those Ruby clients.
> > > >
> > > >
> > > > Besides making the change to server.properties that I mentioned
> above,
> > > are
> > > > there any other client-side configs that will need to be made for the
> > > Ruby
> > > > clients to connect over SSL?
> > > >
> > > >
> > > > Thank you enormously here!
> > > >
> > > >
> > > > Best,
> > > >
> > > > Zac
> > > >
> > > >
> > > > ________________________________
> > > > From: Rajini Sivaram <ra...@googlemail.com>
> > > > Sent: Friday, November 18, 2016 5:15:13 AM
> > > > To: users@kafka.apache.org
> > > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > > >
> > > > Zac,
> > > >
> > > > Kafka has its own built-in load-balancing mechanism based on
> partition
> > > > assignment. Requests are processed by partition leaders, distributing
> > > load
> > > > across the brokers in the cluster. If you want to put a proxy like
> > > HAProxy
> > > > with SSL termination in front of your brokers for added security, you
> > can
> > > > do that. You can have completely independent trust chain between
> > > > clients->proxy and proxy->broker. You need to configure Kafka brokers
> > > with
> > > > the proxy host as the host in the advertised listeners for the
> security
> > > > protocol used by clients.
> > > >
> > > > On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <za...@welltok.com>
> > > > wrote:
> > > >
> > > > > We have two Kafka nodes and for reasons outside of this question,
> > would
> > > > > like to set up a load balancer to terminate SSL with producers
> > > (clients).
> > > > > The SSL cert hosted by the load balancer will be signed by
> > trusted/root
> > > > CA
> > > > > that clients should natively trust.
> > > > >
> > > > >
> > > > > Is this possible to do, or does Kafka somehow require SSL to be
> setup
> > > > > directly on the Kafka servers themselves?
> > > > >
> > > > >
> > > > > Thanks!
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Regards,
> > > >
> > > > Rajini
> > > >
> > >
> > >
> > >
> > > --
> > > Regards,
> > >
> > > Rajini
> > >
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>
>
>
> --
> Regards,
>
> Rajini
>



-- 
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Zac Harvey <za...@welltok.com>.
Thanks again Rajini,


Using these configs, would clients connect to the load balancer over SSL/9093? And then would I configure the load balancer to forward traffic from SSL/9093 to plaintext/9093?


Thanks again, just still a little uncertain about the traffic/ports coming into the load balancer!


Best,

Zac

________________________________
From: Rajini Sivaram <ra...@googlemail.com>
Sent: Monday, November 21, 2016 8:48:41 AM
To: users@kafka.apache.org
Subject: Re: Can Kafka/SSL be terminated at a load balancer?

Zac,

Yes, that is correct. Ruby clients will not be authenticated by Kafka. They
talk SSL to the load balancer and the load balancer uses PLAINTEXT without
authentication to talk to Kafka.

On Mon, Nov 21, 2016 at 1:29 PM, Zac Harvey <za...@welltok.com> wrote:

> *Awesome* explanation Rajini - thank you!
>
>
> Just to confirm: the SASL/PLAIN configs would only be for the interbroker
> communication, correct? Meaning, beyond your recommended changes to
> server.properties, and the addition of the new jaas.conf file, the
> producers (Ruby clients) wouldn't need to authenticate, correct?
>
>
> Thanks again for all the great help so far, you've already helped me more
> than you know!
>
>
> Zac
>
> ________________________________
> From: Rajini Sivaram <ra...@googlemail.com>
> Sent: Monday, November 21, 2016 3:53:47 AM
> To: users@kafka.apache.org
> Subject: Re: Can Kafka/SSL be terminated at a load balancer?
>
> Zac,
>
> *advertised.listeners* is used to make client connections from
> producers/consumers as well as for client-side connections for inter-broker
> communication. In your scenario, setting it to *PLAINTEXT://mykafka01*
> would work for inter-broker, bypassing the load balancer, but clients would
> also then attempt to connect directly to *mykafka01*.  Setting it to
> *SSL://mybalancer01* would work for producers/consumers, but brokers would
> try to connect to *mybalancer01* using PLAINTEXT. Unfortunately neither
> works for both. You need two endpoints, one for inter-broker that bypasses
> *mybalancer01* and another for clients that uses *mybalancer01*. With the
> current Kafka configuration, you would require two security protocols to
> enable two endpoints.
>
> You could enable SSL in Kafka (using self-signed certificates if you need)
> for one of the two endpoints to overcome this limitation. But presumably
> you have a secure internal network running Kafka and want to avoid the cost
> of encryption in Kafka. The simplest solution I can think of is to use
> SASL_PLAINTEXT using SASL/PLAIN for inter-broker as a workaround. The
> configuration options in server.properties would look like:
>
> listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092
>
> advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
> ,SASL_PLAINTEXT://mykafka01.example.com:9092
>
> security.inter.broker.protocol=SASL_PLAINTEXT
>
> sasl.enabled.mechanisms=PLAIN
>
> sasl.mechanism.inter.broker.protocol=PLAIN
>
>
> You also need a JAAS configuration file configured for the broker JVM (
> *KAFKA_OPTS="-Djava.security.auth.login.config=/kafka/jaas.conf"*) . See
> https://kafka.apache.org/documentation#security_sasl for configuring
> SASL.*
> jaas.conf* would look something like:
>
> KafkaServer {
>
>         org.apache.kafka.common.security.plain.PlainLoginModule required
>
>         username="kafka"
>
>         user_kafka="kafka-password"
>
>         password="kafka-password";
>
> };
>
>
> Hope that helps.
>
>
> On Fri, Nov 18, 2016 at 6:39 PM, Zac Harvey <za...@welltok.com>
> wrote:
>
> > Thanks again Rajini!
> >
> >
> > One last followup question, if you don't mind. You said that my
> > server.properties file should look something like this:
> >
> >
> > listeners=SSL://:9093
> > advertised.listeners=SSL://mybalancer01.example.com:9093
> > security.inter.broker.protocol=SSL
> >
> > However, please remember that I'm looking for the load balancer to
> > terminate SSL, meaning that (my desired) communication between the load
> > balancer and Kafka would be over plaintext (not SSL).  In other words:
> >
> > Ruby Producers/Clients <----SSL:9093----> Load Balancer <----
> > Plaintext:9092 ----> Kafka
> >
> > So producers/client connect to the load balancer over SSL and port 9093,
> > but then the load balancer communicates with Kafka over plaintext and
> port
> > 9092.
> >
> > I also don't need inter broker communication to be SSL; it can be
> > plaintext.
> >
> > If this is the case, do I still need to change server.properties, or can
> I
> > leave it like so:
> >
> > listeners=plaintext://:9092
> > advertised.listeners=plaintext://mybalancer01.example.com:9092
> >
> > Or could it just be:
> >
> > listeners=plaintext://:9092
> > advertised.listeners=plaintext://mykafka01.example.com:9092
> >
> > Thanks again!
> > Zac
> >
> >
> >
> >
> >
> > ________________________________
> > From: Rajini Sivaram <ra...@googlemail.com>
> > Sent: Friday, November 18, 2016 9:57:22 AM
> > To: users@kafka.apache.org
> > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> >
> > You should set advertised.listeners rather than the older
> > advertised.host.name property in server.properties:
> >
> >
> >    - listeners=SSL://:9093
> >    - advertised.listeners=SSL://mybalancer01.example.com:9093
> >    - security.inter.broker.protocol=SSL
> >
> >
> > If your listeners are on particular interfaces, you can set address in
> the
> > 'listeners' property too.
> >
> >
> > If you want inter-broker communication to bypass the SSL proxy, you would
> > need another security protocol that can be used for inter-broker
> > communication (PLAINTEXT in the example below).
> >
> >
> >
> >    - listeners=SSL://:9093,PLAINTEXT://:9092
> >    - advertised.listeners=SSL://mybalancer01.example.com:9093,
> PLAINTEXT://
> >    mykafka01.example.com:9092
> >    - security.inter.broker.protocol=PLAINTEXT
> >
> >  I haven't used the Ruby clients, so I am not sure about client
> > configuration. With Java clients, if you don't specify truststore, the
> > default trust stores are used, so with trusted CA-signed certificates, no
> > additional client configuration is required. You can test your
> installation
> > using the console producer and consumer that are shipped with Kafka to
> make
> > sure it is working before you run with Ruby clients.
> >
> >
> >
> > On Fri, Nov 18, 2016 at 1:23 PM, Zac Harvey <za...@welltok.com>
> > wrote:
> >
> > >
> > > Thanks Rajini,
> > >
> > >
> > > So currently one of our Kafka nodes is 'mykafka01.example.com', and in
> > > its server.properties file, I have advertised.host.name=mykafka01
> > > .example.com. Our load balancer lives at mybalancer01.example.com, and
> > > this what producers will connect to (over SSL) to send messages to
> Kafka.
> > >
> > >
> > > It sounds like you're saying I need to change my Kafka node's
> > > server.properties to have advertised.host.name=mybalance
> r01.example.com,
> > > yes? If not, can you perhaps provide a quick snippet of the changes I
> > would
> > > need to make to server.properties?
> > >
> > >
> > > Again, the cert served by the balancer will be a highly-trusted (root
> > > CA-signed) certificate that all clients will natively trust.
> > Interestingly
> > > enough, most (if not all) the Kafka producers/clients will be written
> in
> > > Ruby (using the zendesk Kafka-Ruby gem<https://github.com/
> > > zendesk/ruby-kafka>), so there wont be any JKS configuration options
> > > available for those Ruby clients.
> > >
> > >
> > > Besides making the change to server.properties that I mentioned above,
> > are
> > > there any other client-side configs that will need to be made for the
> > Ruby
> > > clients to connect over SSL?
> > >
> > >
> > > Thank you enormously here!
> > >
> > >
> > > Best,
> > >
> > > Zac
> > >
> > >
> > > ________________________________
> > > From: Rajini Sivaram <ra...@googlemail.com>
> > > Sent: Friday, November 18, 2016 5:15:13 AM
> > > To: users@kafka.apache.org
> > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > >
> > > Zac,
> > >
> > > Kafka has its own built-in load-balancing mechanism based on partition
> > > assignment. Requests are processed by partition leaders, distributing
> > load
> > > across the brokers in the cluster. If you want to put a proxy like
> > HAProxy
> > > with SSL termination in front of your brokers for added security, you
> can
> > > do that. You can have completely independent trust chain between
> > > clients->proxy and proxy->broker. You need to configure Kafka brokers
> > with
> > > the proxy host as the host in the advertised listeners for the security
> > > protocol used by clients.
> > >
> > > On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <za...@welltok.com>
> > > wrote:
> > >
> > > > We have two Kafka nodes and for reasons outside of this question,
> would
> > > > like to set up a load balancer to terminate SSL with producers
> > (clients).
> > > > The SSL cert hosted by the load balancer will be signed by
> trusted/root
> > > CA
> > > > that clients should natively trust.
> > > >
> > > >
> > > > Is this possible to do, or does Kafka somehow require SSL to be setup
> > > > directly on the Kafka servers themselves?
> > > >
> > > >
> > > > Thanks!
> > > >
> > >
> > >
> > >
> > > --
> > > Regards,
> > >
> > > Rajini
> > >
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>
>
>
> --
> Regards,
>
> Rajini
>



--
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Rajini Sivaram <ra...@googlemail.com>.
Zac,

Yes, that is correct. Ruby clients will not be authenticated by Kafka. They
talk SSL to the load balancer and the load balancer uses PLAINTEXT without
authentication to talk to Kafka.

On Mon, Nov 21, 2016 at 1:29 PM, Zac Harvey <za...@welltok.com> wrote:

> *Awesome* explanation Rajini - thank you!
>
>
> Just to confirm: the SASL/PLAIN configs would only be for the interbroker
> communication, correct? Meaning, beyond your recommended changes to
> server.properties, and the addition of the new jaas.conf file, the
> producers (Ruby clients) wouldn't need to authenticate, correct?
>
>
> Thanks again for all the great help so far, you've already helped me more
> than you know!
>
>
> Zac
>
> ________________________________
> From: Rajini Sivaram <ra...@googlemail.com>
> Sent: Monday, November 21, 2016 3:53:47 AM
> To: users@kafka.apache.org
> Subject: Re: Can Kafka/SSL be terminated at a load balancer?
>
> Zac,
>
> *advertised.listeners* is used to make client connections from
> producers/consumers as well as for client-side connections for inter-broker
> communication. In your scenario, setting it to *PLAINTEXT://mykafka01*
> would work for inter-broker, bypassing the load balancer, but clients would
> also then attempt to connect directly to *mykafka01*.  Setting it to
> *SSL://mybalancer01* would work for producers/consumers, but brokers would
> try to connect to *mybalancer01* using PLAINTEXT. Unfortunately neither
> works for both. You need two endpoints, one for inter-broker that bypasses
> *mybalancer01* and another for clients that uses *mybalancer01*. With the
> current Kafka configuration, you would require two security protocols to
> enable two endpoints.
>
> You could enable SSL in Kafka (using self-signed certificates if you need)
> for one of the two endpoints to overcome this limitation. But presumably
> you have a secure internal network running Kafka and want to avoid the cost
> of encryption in Kafka. The simplest solution I can think of is to use
> SASL_PLAINTEXT using SASL/PLAIN for inter-broker as a workaround. The
> configuration options in server.properties would look like:
>
> listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092
>
> advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
> ,SASL_PLAINTEXT://mykafka01.example.com:9092
>
> security.inter.broker.protocol=SASL_PLAINTEXT
>
> sasl.enabled.mechanisms=PLAIN
>
> sasl.mechanism.inter.broker.protocol=PLAIN
>
>
> You also need a JAAS configuration file configured for the broker JVM (
> *KAFKA_OPTS="-Djava.security.auth.login.config=/kafka/jaas.conf"*) . See
> https://kafka.apache.org/documentation#security_sasl for configuring
> SASL.*
> jaas.conf* would look something like:
>
> KafkaServer {
>
>         org.apache.kafka.common.security.plain.PlainLoginModule required
>
>         username="kafka"
>
>         user_kafka="kafka-password"
>
>         password="kafka-password";
>
> };
>
>
> Hope that helps.
>
>
> On Fri, Nov 18, 2016 at 6:39 PM, Zac Harvey <za...@welltok.com>
> wrote:
>
> > Thanks again Rajini!
> >
> >
> > One last followup question, if you don't mind. You said that my
> > server.properties file should look something like this:
> >
> >
> > listeners=SSL://:9093
> > advertised.listeners=SSL://mybalancer01.example.com:9093
> > security.inter.broker.protocol=SSL
> >
> > However, please remember that I'm looking for the load balancer to
> > terminate SSL, meaning that (my desired) communication between the load
> > balancer and Kafka would be over plaintext (not SSL).  In other words:
> >
> > Ruby Producers/Clients <----SSL:9093----> Load Balancer <----
> > Plaintext:9092 ----> Kafka
> >
> > So producers/client connect to the load balancer over SSL and port 9093,
> > but then the load balancer communicates with Kafka over plaintext and
> port
> > 9092.
> >
> > I also don't need inter broker communication to be SSL; it can be
> > plaintext.
> >
> > If this is the case, do I still need to change server.properties, or can
> I
> > leave it like so:
> >
> > listeners=plaintext://:9092
> > advertised.listeners=plaintext://mybalancer01.example.com:9092
> >
> > Or could it just be:
> >
> > listeners=plaintext://:9092
> > advertised.listeners=plaintext://mykafka01.example.com:9092
> >
> > Thanks again!
> > Zac
> >
> >
> >
> >
> >
> > ________________________________
> > From: Rajini Sivaram <ra...@googlemail.com>
> > Sent: Friday, November 18, 2016 9:57:22 AM
> > To: users@kafka.apache.org
> > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> >
> > You should set advertised.listeners rather than the older
> > advertised.host.name property in server.properties:
> >
> >
> >    - listeners=SSL://:9093
> >    - advertised.listeners=SSL://mybalancer01.example.com:9093
> >    - security.inter.broker.protocol=SSL
> >
> >
> > If your listeners are on particular interfaces, you can set address in
> the
> > 'listeners' property too.
> >
> >
> > If you want inter-broker communication to bypass the SSL proxy, you would
> > need another security protocol that can be used for inter-broker
> > communication (PLAINTEXT in the example below).
> >
> >
> >
> >    - listeners=SSL://:9093,PLAINTEXT://:9092
> >    - advertised.listeners=SSL://mybalancer01.example.com:9093,
> PLAINTEXT://
> >    mykafka01.example.com:9092
> >    - security.inter.broker.protocol=PLAINTEXT
> >
> >  I haven't used the Ruby clients, so I am not sure about client
> > configuration. With Java clients, if you don't specify truststore, the
> > default trust stores are used, so with trusted CA-signed certificates, no
> > additional client configuration is required. You can test your
> installation
> > using the console producer and consumer that are shipped with Kafka to
> make
> > sure it is working before you run with Ruby clients.
> >
> >
> >
> > On Fri, Nov 18, 2016 at 1:23 PM, Zac Harvey <za...@welltok.com>
> > wrote:
> >
> > >
> > > Thanks Rajini,
> > >
> > >
> > > So currently one of our Kafka nodes is 'mykafka01.example.com', and in
> > > its server.properties file, I have advertised.host.name=mykafka01
> > > .example.com. Our load balancer lives at mybalancer01.example.com, and
> > > this what producers will connect to (over SSL) to send messages to
> Kafka.
> > >
> > >
> > > It sounds like you're saying I need to change my Kafka node's
> > > server.properties to have advertised.host.name=mybalance
> r01.example.com,
> > > yes? If not, can you perhaps provide a quick snippet of the changes I
> > would
> > > need to make to server.properties?
> > >
> > >
> > > Again, the cert served by the balancer will be a highly-trusted (root
> > > CA-signed) certificate that all clients will natively trust.
> > Interestingly
> > > enough, most (if not all) the Kafka producers/clients will be written
> in
> > > Ruby (using the zendesk Kafka-Ruby gem<https://github.com/
> > > zendesk/ruby-kafka>), so there wont be any JKS configuration options
> > > available for those Ruby clients.
> > >
> > >
> > > Besides making the change to server.properties that I mentioned above,
> > are
> > > there any other client-side configs that will need to be made for the
> > Ruby
> > > clients to connect over SSL?
> > >
> > >
> > > Thank you enormously here!
> > >
> > >
> > > Best,
> > >
> > > Zac
> > >
> > >
> > > ________________________________
> > > From: Rajini Sivaram <ra...@googlemail.com>
> > > Sent: Friday, November 18, 2016 5:15:13 AM
> > > To: users@kafka.apache.org
> > > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> > >
> > > Zac,
> > >
> > > Kafka has its own built-in load-balancing mechanism based on partition
> > > assignment. Requests are processed by partition leaders, distributing
> > load
> > > across the brokers in the cluster. If you want to put a proxy like
> > HAProxy
> > > with SSL termination in front of your brokers for added security, you
> can
> > > do that. You can have completely independent trust chain between
> > > clients->proxy and proxy->broker. You need to configure Kafka brokers
> > with
> > > the proxy host as the host in the advertised listeners for the security
> > > protocol used by clients.
> > >
> > > On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <za...@welltok.com>
> > > wrote:
> > >
> > > > We have two Kafka nodes and for reasons outside of this question,
> would
> > > > like to set up a load balancer to terminate SSL with producers
> > (clients).
> > > > The SSL cert hosted by the load balancer will be signed by
> trusted/root
> > > CA
> > > > that clients should natively trust.
> > > >
> > > >
> > > > Is this possible to do, or does Kafka somehow require SSL to be setup
> > > > directly on the Kafka servers themselves?
> > > >
> > > >
> > > > Thanks!
> > > >
> > >
> > >
> > >
> > > --
> > > Regards,
> > >
> > > Rajini
> > >
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>
>
>
> --
> Regards,
>
> Rajini
>



-- 
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Zac Harvey <za...@welltok.com>.
*Awesome* explanation Rajini - thank you!


Just to confirm: the SASL/PLAIN configs would only be for the interbroker communication, correct? Meaning, beyond your recommended changes to server.properties, and the addition of the new jaas.conf file, the producers (Ruby clients) wouldn't need to authenticate, correct?


Thanks again for all the great help so far, you've already helped me more than you know!


Zac

________________________________
From: Rajini Sivaram <ra...@googlemail.com>
Sent: Monday, November 21, 2016 3:53:47 AM
To: users@kafka.apache.org
Subject: Re: Can Kafka/SSL be terminated at a load balancer?

Zac,

*advertised.listeners* is used to make client connections from
producers/consumers as well as for client-side connections for inter-broker
communication. In your scenario, setting it to *PLAINTEXT://mykafka01*
would work for inter-broker, bypassing the load balancer, but clients would
also then attempt to connect directly to *mykafka01*.  Setting it to
*SSL://mybalancer01* would work for producers/consumers, but brokers would
try to connect to *mybalancer01* using PLAINTEXT. Unfortunately neither
works for both. You need two endpoints, one for inter-broker that bypasses
*mybalancer01* and another for clients that uses *mybalancer01*. With the
current Kafka configuration, you would require two security protocols to
enable two endpoints.

You could enable SSL in Kafka (using self-signed certificates if you need)
for one of the two endpoints to overcome this limitation. But presumably
you have a secure internal network running Kafka and want to avoid the cost
of encryption in Kafka. The simplest solution I can think of is to use
SASL_PLAINTEXT using SASL/PLAIN for inter-broker as a workaround. The
configuration options in server.properties would look like:

listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092

advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
,SASL_PLAINTEXT://mykafka01.example.com:9092

security.inter.broker.protocol=SASL_PLAINTEXT

sasl.enabled.mechanisms=PLAIN

sasl.mechanism.inter.broker.protocol=PLAIN


You also need a JAAS configuration file configured for the broker JVM (
*KAFKA_OPTS="-Djava.security.auth.login.config=/kafka/jaas.conf"*) . See
https://kafka.apache.org/documentation#security_sasl for configuring SASL.*
jaas.conf* would look something like:

KafkaServer {

        org.apache.kafka.common.security.plain.PlainLoginModule required

        username="kafka"

        user_kafka="kafka-password"

        password="kafka-password";

};


Hope that helps.


On Fri, Nov 18, 2016 at 6:39 PM, Zac Harvey <za...@welltok.com> wrote:

> Thanks again Rajini!
>
>
> One last followup question, if you don't mind. You said that my
> server.properties file should look something like this:
>
>
> listeners=SSL://:9093
> advertised.listeners=SSL://mybalancer01.example.com:9093
> security.inter.broker.protocol=SSL
>
> However, please remember that I'm looking for the load balancer to
> terminate SSL, meaning that (my desired) communication between the load
> balancer and Kafka would be over plaintext (not SSL).  In other words:
>
> Ruby Producers/Clients <----SSL:9093----> Load Balancer <----
> Plaintext:9092 ----> Kafka
>
> So producers/client connect to the load balancer over SSL and port 9093,
> but then the load balancer communicates with Kafka over plaintext and port
> 9092.
>
> I also don't need inter broker communication to be SSL; it can be
> plaintext.
>
> If this is the case, do I still need to change server.properties, or can I
> leave it like so:
>
> listeners=plaintext://:9092
> advertised.listeners=plaintext://mybalancer01.example.com:9092
>
> Or could it just be:
>
> listeners=plaintext://:9092
> advertised.listeners=plaintext://mykafka01.example.com:9092
>
> Thanks again!
> Zac
>
>
>
>
>
> ________________________________
> From: Rajini Sivaram <ra...@googlemail.com>
> Sent: Friday, November 18, 2016 9:57:22 AM
> To: users@kafka.apache.org
> Subject: Re: Can Kafka/SSL be terminated at a load balancer?
>
> You should set advertised.listeners rather than the older
> advertised.host.name property in server.properties:
>
>
>    - listeners=SSL://:9093
>    - advertised.listeners=SSL://mybalancer01.example.com:9093
>    - security.inter.broker.protocol=SSL
>
>
> If your listeners are on particular interfaces, you can set address in the
> 'listeners' property too.
>
>
> If you want inter-broker communication to bypass the SSL proxy, you would
> need another security protocol that can be used for inter-broker
> communication (PLAINTEXT in the example below).
>
>
>
>    - listeners=SSL://:9093,PLAINTEXT://:9092
>    - advertised.listeners=SSL://mybalancer01.example.com:9093,PLAINTEXT://
>    mykafka01.example.com:9092
>    - security.inter.broker.protocol=PLAINTEXT
>
>  I haven't used the Ruby clients, so I am not sure about client
> configuration. With Java clients, if you don't specify truststore, the
> default trust stores are used, so with trusted CA-signed certificates, no
> additional client configuration is required. You can test your installation
> using the console producer and consumer that are shipped with Kafka to make
> sure it is working before you run with Ruby clients.
>
>
>
> On Fri, Nov 18, 2016 at 1:23 PM, Zac Harvey <za...@welltok.com>
> wrote:
>
> >
> > Thanks Rajini,
> >
> >
> > So currently one of our Kafka nodes is 'mykafka01.example.com', and in
> > its server.properties file, I have advertised.host.name=mykafka01
> > .example.com. Our load balancer lives at mybalancer01.example.com, and
> > this what producers will connect to (over SSL) to send messages to Kafka.
> >
> >
> > It sounds like you're saying I need to change my Kafka node's
> > server.properties to have advertised.host.name=mybalancer01.example.com,
> > yes? If not, can you perhaps provide a quick snippet of the changes I
> would
> > need to make to server.properties?
> >
> >
> > Again, the cert served by the balancer will be a highly-trusted (root
> > CA-signed) certificate that all clients will natively trust.
> Interestingly
> > enough, most (if not all) the Kafka producers/clients will be written in
> > Ruby (using the zendesk Kafka-Ruby gem<https://github.com/
> > zendesk/ruby-kafka>), so there wont be any JKS configuration options
> > available for those Ruby clients.
> >
> >
> > Besides making the change to server.properties that I mentioned above,
> are
> > there any other client-side configs that will need to be made for the
> Ruby
> > clients to connect over SSL?
> >
> >
> > Thank you enormously here!
> >
> >
> > Best,
> >
> > Zac
> >
> >
> > ________________________________
> > From: Rajini Sivaram <ra...@googlemail.com>
> > Sent: Friday, November 18, 2016 5:15:13 AM
> > To: users@kafka.apache.org
> > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> >
> > Zac,
> >
> > Kafka has its own built-in load-balancing mechanism based on partition
> > assignment. Requests are processed by partition leaders, distributing
> load
> > across the brokers in the cluster. If you want to put a proxy like
> HAProxy
> > with SSL termination in front of your brokers for added security, you can
> > do that. You can have completely independent trust chain between
> > clients->proxy and proxy->broker. You need to configure Kafka brokers
> with
> > the proxy host as the host in the advertised listeners for the security
> > protocol used by clients.
> >
> > On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <za...@welltok.com>
> > wrote:
> >
> > > We have two Kafka nodes and for reasons outside of this question, would
> > > like to set up a load balancer to terminate SSL with producers
> (clients).
> > > The SSL cert hosted by the load balancer will be signed by trusted/root
> > CA
> > > that clients should natively trust.
> > >
> > >
> > > Is this possible to do, or does Kafka somehow require SSL to be setup
> > > directly on the Kafka servers themselves?
> > >
> > >
> > > Thanks!
> > >
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>
>
>
> --
> Regards,
>
> Rajini
>



--
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Rajini Sivaram <ra...@googlemail.com>.
Zac,

*advertised.listeners* is used to make client connections from
producers/consumers as well as for client-side connections for inter-broker
communication. In your scenario, setting it to *PLAINTEXT://mykafka01*
would work for inter-broker, bypassing the load balancer, but clients would
also then attempt to connect directly to *mykafka01*.  Setting it to
*SSL://mybalancer01* would work for producers/consumers, but brokers would
try to connect to *mybalancer01* using PLAINTEXT. Unfortunately neither
works for both. You need two endpoints, one for inter-broker that bypasses
*mybalancer01* and another for clients that uses *mybalancer01*. With the
current Kafka configuration, you would require two security protocols to
enable two endpoints.

You could enable SSL in Kafka (using self-signed certificates if you need)
for one of the two endpoints to overcome this limitation. But presumably
you have a secure internal network running Kafka and want to avoid the cost
of encryption in Kafka. The simplest solution I can think of is to use
SASL_PLAINTEXT using SASL/PLAIN for inter-broker as a workaround. The
configuration options in server.properties would look like:

listeners=PLAINTEXT://:9093,SASL_PLAINTEXT://:9092

advertised.listeners=PLAINTEXT://mybalancer01.example.com:9093
,SASL_PLAINTEXT://mykafka01.example.com:9092

security.inter.broker.protocol=SASL_PLAINTEXT

sasl.enabled.mechanisms=PLAIN

sasl.mechanism.inter.broker.protocol=PLAIN


You also need a JAAS configuration file configured for the broker JVM (
*KAFKA_OPTS="-Djava.security.auth.login.config=/kafka/jaas.conf"*) . See
https://kafka.apache.org/documentation#security_sasl for configuring SASL.*
jaas.conf* would look something like:

KafkaServer {

        org.apache.kafka.common.security.plain.PlainLoginModule required

        username="kafka"

        user_kafka="kafka-password"

        password="kafka-password";

};


Hope that helps.


On Fri, Nov 18, 2016 at 6:39 PM, Zac Harvey <za...@welltok.com> wrote:

> Thanks again Rajini!
>
>
> One last followup question, if you don't mind. You said that my
> server.properties file should look something like this:
>
>
> listeners=SSL://:9093
> advertised.listeners=SSL://mybalancer01.example.com:9093
> security.inter.broker.protocol=SSL
>
> However, please remember that I'm looking for the load balancer to
> terminate SSL, meaning that (my desired) communication between the load
> balancer and Kafka would be over plaintext (not SSL).  In other words:
>
> Ruby Producers/Clients <----SSL:9093----> Load Balancer <----
> Plaintext:9092 ----> Kafka
>
> So producers/client connect to the load balancer over SSL and port 9093,
> but then the load balancer communicates with Kafka over plaintext and port
> 9092.
>
> I also don't need inter broker communication to be SSL; it can be
> plaintext.
>
> If this is the case, do I still need to change server.properties, or can I
> leave it like so:
>
> listeners=plaintext://:9092
> advertised.listeners=plaintext://mybalancer01.example.com:9092
>
> Or could it just be:
>
> listeners=plaintext://:9092
> advertised.listeners=plaintext://mykafka01.example.com:9092
>
> Thanks again!
> Zac
>
>
>
>
>
> ________________________________
> From: Rajini Sivaram <ra...@googlemail.com>
> Sent: Friday, November 18, 2016 9:57:22 AM
> To: users@kafka.apache.org
> Subject: Re: Can Kafka/SSL be terminated at a load balancer?
>
> You should set advertised.listeners rather than the older
> advertised.host.name property in server.properties:
>
>
>    - listeners=SSL://:9093
>    - advertised.listeners=SSL://mybalancer01.example.com:9093
>    - security.inter.broker.protocol=SSL
>
>
> If your listeners are on particular interfaces, you can set address in the
> 'listeners' property too.
>
>
> If you want inter-broker communication to bypass the SSL proxy, you would
> need another security protocol that can be used for inter-broker
> communication (PLAINTEXT in the example below).
>
>
>
>    - listeners=SSL://:9093,PLAINTEXT://:9092
>    - advertised.listeners=SSL://mybalancer01.example.com:9093,PLAINTEXT://
>    mykafka01.example.com:9092
>    - security.inter.broker.protocol=PLAINTEXT
>
>  I haven't used the Ruby clients, so I am not sure about client
> configuration. With Java clients, if you don't specify truststore, the
> default trust stores are used, so with trusted CA-signed certificates, no
> additional client configuration is required. You can test your installation
> using the console producer and consumer that are shipped with Kafka to make
> sure it is working before you run with Ruby clients.
>
>
>
> On Fri, Nov 18, 2016 at 1:23 PM, Zac Harvey <za...@welltok.com>
> wrote:
>
> >
> > Thanks Rajini,
> >
> >
> > So currently one of our Kafka nodes is 'mykafka01.example.com', and in
> > its server.properties file, I have advertised.host.name=mykafka01
> > .example.com. Our load balancer lives at mybalancer01.example.com, and
> > this what producers will connect to (over SSL) to send messages to Kafka.
> >
> >
> > It sounds like you're saying I need to change my Kafka node's
> > server.properties to have advertised.host.name=mybalancer01.example.com,
> > yes? If not, can you perhaps provide a quick snippet of the changes I
> would
> > need to make to server.properties?
> >
> >
> > Again, the cert served by the balancer will be a highly-trusted (root
> > CA-signed) certificate that all clients will natively trust.
> Interestingly
> > enough, most (if not all) the Kafka producers/clients will be written in
> > Ruby (using the zendesk Kafka-Ruby gem<https://github.com/
> > zendesk/ruby-kafka>), so there wont be any JKS configuration options
> > available for those Ruby clients.
> >
> >
> > Besides making the change to server.properties that I mentioned above,
> are
> > there any other client-side configs that will need to be made for the
> Ruby
> > clients to connect over SSL?
> >
> >
> > Thank you enormously here!
> >
> >
> > Best,
> >
> > Zac
> >
> >
> > ________________________________
> > From: Rajini Sivaram <ra...@googlemail.com>
> > Sent: Friday, November 18, 2016 5:15:13 AM
> > To: users@kafka.apache.org
> > Subject: Re: Can Kafka/SSL be terminated at a load balancer?
> >
> > Zac,
> >
> > Kafka has its own built-in load-balancing mechanism based on partition
> > assignment. Requests are processed by partition leaders, distributing
> load
> > across the brokers in the cluster. If you want to put a proxy like
> HAProxy
> > with SSL termination in front of your brokers for added security, you can
> > do that. You can have completely independent trust chain between
> > clients->proxy and proxy->broker. You need to configure Kafka brokers
> with
> > the proxy host as the host in the advertised listeners for the security
> > protocol used by clients.
> >
> > On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <za...@welltok.com>
> > wrote:
> >
> > > We have two Kafka nodes and for reasons outside of this question, would
> > > like to set up a load balancer to terminate SSL with producers
> (clients).
> > > The SSL cert hosted by the load balancer will be signed by trusted/root
> > CA
> > > that clients should natively trust.
> > >
> > >
> > > Is this possible to do, or does Kafka somehow require SSL to be setup
> > > directly on the Kafka servers themselves?
> > >
> > >
> > > Thanks!
> > >
> >
> >
> >
> > --
> > Regards,
> >
> > Rajini
> >
>
>
>
> --
> Regards,
>
> Rajini
>



-- 
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Zac Harvey <za...@welltok.com>.
Thanks again Rajini!


One last followup question, if you don't mind. You said that my server.properties file should look something like this:


listeners=SSL://:9093
advertised.listeners=SSL://mybalancer01.example.com:9093
security.inter.broker.protocol=SSL

However, please remember that I'm looking for the load balancer to terminate SSL, meaning that (my desired) communication between the load balancer and Kafka would be over plaintext (not SSL).  In other words:

Ruby Producers/Clients <----SSL:9093----> Load Balancer <---- Plaintext:9092 ----> Kafka

So producers/client connect to the load balancer over SSL and port 9093, but then the load balancer communicates with Kafka over plaintext and port 9092.

I also don't need inter broker communication to be SSL; it can be plaintext.

If this is the case, do I still need to change server.properties, or can I leave it like so:

listeners=plaintext://:9092
advertised.listeners=plaintext://mybalancer01.example.com:9092

Or could it just be:

listeners=plaintext://:9092
advertised.listeners=plaintext://mykafka01.example.com:9092

Thanks again!
Zac





________________________________
From: Rajini Sivaram <ra...@googlemail.com>
Sent: Friday, November 18, 2016 9:57:22 AM
To: users@kafka.apache.org
Subject: Re: Can Kafka/SSL be terminated at a load balancer?

You should set advertised.listeners rather than the older
advertised.host.name property in server.properties:


   - listeners=SSL://:9093
   - advertised.listeners=SSL://mybalancer01.example.com:9093
   - security.inter.broker.protocol=SSL


If your listeners are on particular interfaces, you can set address in the
'listeners' property too.


If you want inter-broker communication to bypass the SSL proxy, you would
need another security protocol that can be used for inter-broker
communication (PLAINTEXT in the example below).



   - listeners=SSL://:9093,PLAINTEXT://:9092
   - advertised.listeners=SSL://mybalancer01.example.com:9093,PLAINTEXT://
   mykafka01.example.com:9092
   - security.inter.broker.protocol=PLAINTEXT

 I haven't used the Ruby clients, so I am not sure about client
configuration. With Java clients, if you don't specify truststore, the
default trust stores are used, so with trusted CA-signed certificates, no
additional client configuration is required. You can test your installation
using the console producer and consumer that are shipped with Kafka to make
sure it is working before you run with Ruby clients.



On Fri, Nov 18, 2016 at 1:23 PM, Zac Harvey <za...@welltok.com> wrote:

>
> Thanks Rajini,
>
>
> So currently one of our Kafka nodes is 'mykafka01.example.com', and in
> its server.properties file, I have advertised.host.name=mykafka01
> .example.com. Our load balancer lives at mybalancer01.example.com, and
> this what producers will connect to (over SSL) to send messages to Kafka.
>
>
> It sounds like you're saying I need to change my Kafka node's
> server.properties to have advertised.host.name=mybalancer01.example.com,
> yes? If not, can you perhaps provide a quick snippet of the changes I would
> need to make to server.properties?
>
>
> Again, the cert served by the balancer will be a highly-trusted (root
> CA-signed) certificate that all clients will natively trust. Interestingly
> enough, most (if not all) the Kafka producers/clients will be written in
> Ruby (using the zendesk Kafka-Ruby gem<https://github.com/
> zendesk/ruby-kafka>), so there wont be any JKS configuration options
> available for those Ruby clients.
>
>
> Besides making the change to server.properties that I mentioned above, are
> there any other client-side configs that will need to be made for the Ruby
> clients to connect over SSL?
>
>
> Thank you enormously here!
>
>
> Best,
>
> Zac
>
>
> ________________________________
> From: Rajini Sivaram <ra...@googlemail.com>
> Sent: Friday, November 18, 2016 5:15:13 AM
> To: users@kafka.apache.org
> Subject: Re: Can Kafka/SSL be terminated at a load balancer?
>
> Zac,
>
> Kafka has its own built-in load-balancing mechanism based on partition
> assignment. Requests are processed by partition leaders, distributing load
> across the brokers in the cluster. If you want to put a proxy like HAProxy
> with SSL termination in front of your brokers for added security, you can
> do that. You can have completely independent trust chain between
> clients->proxy and proxy->broker. You need to configure Kafka brokers with
> the proxy host as the host in the advertised listeners for the security
> protocol used by clients.
>
> On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <za...@welltok.com>
> wrote:
>
> > We have two Kafka nodes and for reasons outside of this question, would
> > like to set up a load balancer to terminate SSL with producers (clients).
> > The SSL cert hosted by the load balancer will be signed by trusted/root
> CA
> > that clients should natively trust.
> >
> >
> > Is this possible to do, or does Kafka somehow require SSL to be setup
> > directly on the Kafka servers themselves?
> >
> >
> > Thanks!
> >
>
>
>
> --
> Regards,
>
> Rajini
>



--
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Rajini Sivaram <ra...@googlemail.com>.
You should set advertised.listeners rather than the older
advertised.host.name property in server.properties:


   - listeners=SSL://:9093
   - advertised.listeners=SSL://mybalancer01.example.com:9093
   - security.inter.broker.protocol=SSL


If your listeners are on particular interfaces, you can set address in the
'listeners' property too.


If you want inter-broker communication to bypass the SSL proxy, you would
need another security protocol that can be used for inter-broker
communication (PLAINTEXT in the example below).



   - listeners=SSL://:9093,PLAINTEXT://:9092
   - advertised.listeners=SSL://mybalancer01.example.com:9093,PLAINTEXT://
   mykafka01.example.com:9092
   - security.inter.broker.protocol=PLAINTEXT

 I haven't used the Ruby clients, so I am not sure about client
configuration. With Java clients, if you don't specify truststore, the
default trust stores are used, so with trusted CA-signed certificates, no
additional client configuration is required. You can test your installation
using the console producer and consumer that are shipped with Kafka to make
sure it is working before you run with Ruby clients.



On Fri, Nov 18, 2016 at 1:23 PM, Zac Harvey <za...@welltok.com> wrote:

>
> Thanks Rajini,
>
>
> So currently one of our Kafka nodes is 'mykafka01.example.com', and in
> its server.properties file, I have advertised.host.name=mykafka01
> .example.com. Our load balancer lives at mybalancer01.example.com, and
> this what producers will connect to (over SSL) to send messages to Kafka.
>
>
> It sounds like you're saying I need to change my Kafka node's
> server.properties to have advertised.host.name=mybalancer01.example.com,
> yes? If not, can you perhaps provide a quick snippet of the changes I would
> need to make to server.properties?
>
>
> Again, the cert served by the balancer will be a highly-trusted (root
> CA-signed) certificate that all clients will natively trust. Interestingly
> enough, most (if not all) the Kafka producers/clients will be written in
> Ruby (using the zendesk Kafka-Ruby gem<https://github.com/
> zendesk/ruby-kafka>), so there wont be any JKS configuration options
> available for those Ruby clients.
>
>
> Besides making the change to server.properties that I mentioned above, are
> there any other client-side configs that will need to be made for the Ruby
> clients to connect over SSL?
>
>
> Thank you enormously here!
>
>
> Best,
>
> Zac
>
>
> ________________________________
> From: Rajini Sivaram <ra...@googlemail.com>
> Sent: Friday, November 18, 2016 5:15:13 AM
> To: users@kafka.apache.org
> Subject: Re: Can Kafka/SSL be terminated at a load balancer?
>
> Zac,
>
> Kafka has its own built-in load-balancing mechanism based on partition
> assignment. Requests are processed by partition leaders, distributing load
> across the brokers in the cluster. If you want to put a proxy like HAProxy
> with SSL termination in front of your brokers for added security, you can
> do that. You can have completely independent trust chain between
> clients->proxy and proxy->broker. You need to configure Kafka brokers with
> the proxy host as the host in the advertised listeners for the security
> protocol used by clients.
>
> On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <za...@welltok.com>
> wrote:
>
> > We have two Kafka nodes and for reasons outside of this question, would
> > like to set up a load balancer to terminate SSL with producers (clients).
> > The SSL cert hosted by the load balancer will be signed by trusted/root
> CA
> > that clients should natively trust.
> >
> >
> > Is this possible to do, or does Kafka somehow require SSL to be setup
> > directly on the Kafka servers themselves?
> >
> >
> > Thanks!
> >
>
>
>
> --
> Regards,
>
> Rajini
>



-- 
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Zac Harvey <za...@welltok.com>.
Thanks Rajini,


So currently one of our Kafka nodes is 'mykafka01.example.com', and in its server.properties file, I have advertised.host.name=mykafka01.example.com. Our load balancer lives at mybalancer01.example.com, and this what producers will connect to (over SSL) to send messages to Kafka.


It sounds like you're saying I need to change my Kafka node's server.properties to have advertised.host.name=mybalancer01.example.com, yes? If not, can you perhaps provide a quick snippet of the changes I would need to make to server.properties?


Again, the cert served by the balancer will be a highly-trusted (root CA-signed) certificate that all clients will natively trust. Interestingly enough, most (if not all) the Kafka producers/clients will be written in Ruby (using the zendesk Kafka-Ruby gem<https://github.com/zendesk/ruby-kafka>), so there wont be any JKS configuration options available for those Ruby clients.


Besides making the change to server.properties that I mentioned above, are there any other client-side configs that will need to be made for the Ruby clients to connect over SSL?


Thank you enormously here!


Best,

Zac


________________________________
From: Rajini Sivaram <ra...@googlemail.com>
Sent: Friday, November 18, 2016 5:15:13 AM
To: users@kafka.apache.org
Subject: Re: Can Kafka/SSL be terminated at a load balancer?

Zac,

Kafka has its own built-in load-balancing mechanism based on partition
assignment. Requests are processed by partition leaders, distributing load
across the brokers in the cluster. If you want to put a proxy like HAProxy
with SSL termination in front of your brokers for added security, you can
do that. You can have completely independent trust chain between
clients->proxy and proxy->broker. You need to configure Kafka brokers with
the proxy host as the host in the advertised listeners for the security
protocol used by clients.

On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <za...@welltok.com> wrote:

> We have two Kafka nodes and for reasons outside of this question, would
> like to set up a load balancer to terminate SSL with producers (clients).
> The SSL cert hosted by the load balancer will be signed by trusted/root CA
> that clients should natively trust.
>
>
> Is this possible to do, or does Kafka somehow require SSL to be setup
> directly on the Kafka servers themselves?
>
>
> Thanks!
>



--
Regards,

Rajini

Re: Can Kafka/SSL be terminated at a load balancer?

Posted by Rajini Sivaram <ra...@googlemail.com>.
Zac,

Kafka has its own built-in load-balancing mechanism based on partition
assignment. Requests are processed by partition leaders, distributing load
across the brokers in the cluster. If you want to put a proxy like HAProxy
with SSL termination in front of your brokers for added security, you can
do that. You can have completely independent trust chain between
clients->proxy and proxy->broker. You need to configure Kafka brokers with
the proxy host as the host in the advertised listeners for the security
protocol used by clients.

On Thu, Nov 17, 2016 at 9:44 PM, Zac Harvey <za...@welltok.com> wrote:

> We have two Kafka nodes and for reasons outside of this question, would
> like to set up a load balancer to terminate SSL with producers (clients).
> The SSL cert hosted by the load balancer will be signed by trusted/root CA
> that clients should natively trust.
>
>
> Is this possible to do, or does Kafka somehow require SSL to be setup
> directly on the Kafka servers themselves?
>
>
> Thanks!
>



-- 
Regards,

Rajini