Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KIP-392: Allow consumers to fetch from closest replica #1696

Closed
wants to merge 1 commit into from

Conversation

dneralla
Copy link
Contributor

@dneralla dneralla commented May 11, 2020

@dnwe
Copy link
Collaborator

dnwe commented May 11, 2020

@dneralla thanks for contributing!

I've taken a first pass over the changes and they look good. Please can you sign the CLA?

Have you tried them out against a real 2.4/2.5 cluster with rack aware support enabled?

@dneralla
Copy link
Contributor Author

@dnwe Signed the CLA. Not sure how this CI is integrated it seems to be not running again.
I haven't tried out yet on 2.4 or 2.5 cluster. Will do some time later this week.

@bai
Copy link
Contributor

bai commented May 11, 2020

Thanks for contributing 🙏 I've re-triggered CI.

@ghost ghost removed the cla-needed label May 11, 2020
@bai
Copy link
Contributor

bai commented May 11, 2020

Do you think you could rebase your branch off of latest master please?

@@ -359,21 +362,29 @@ func (child *partitionConsumer) dispatcher() {
close(child.feeder)
}

func (child *partitionConsumer) preferedBroker() (*Broker, error) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

preferred, not prefered.

@@ -617,6 +626,12 @@ func (child *partitionConsumer) parseResponse(response *FetchResponse) ([]*Consu
child.fetchSize = child.conf.Consumer.Fetch.Default
atomic.StoreInt64(&child.highWaterMarkOffset, block.HighWaterMarkOffset)

if response.Version == 11 && len(child.consumer.conf.RackID) > 0 {
// we got a valid response with messages. update child's preferredReadReplica from the FetchResponseBlock
child.replicaInited = true
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this name is not descriptive enough; suggest readReplicaConfigured or similar.

@d1egoaz d1egoaz changed the title KIP-392: Consumer support for sarama KIP-392: Allow consumers to fetch from closest replica May 11, 2020
@@ -299,6 +299,9 @@ type partitionConsumer struct {
errors chan *ConsumerError
feeder chan *FetchResponse

replicaInited bool
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

initiated? what is this going to be used for?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I will rename it...its used to fetch from read replica instead of master.

@@ -834,6 +855,7 @@ func (bc *brokerConsumer) handleResponses() {
// not an error, but does need redispatching
Logger.Printf("consumer/broker/%d abandoned subscription to %s/%d because %s\n",
bc.broker.ID(), child.topic, child.partition, result)
child.replicaInited = false
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder if this needs a locking mechanism? seems unsafe
are we sure there are no multiple go routines accessing this?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Checked. it it's fine to be accessed like this.

if !child.replicaInited {
return
}
if bc.broker.ID() != child.preferredReadReplica {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this would also do the same:

if child.replicaInited && bc.broker.ID() != child.preferredReadReplica

not need to do the return there

also the no-op message might be important to leave it, maybe with a modified comment

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ack. will do this

broker, err := child.consumer.client.Broker(child.preferredReadReplica)
if err == nil {
return broker, nil
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

does the error not important? could we log it?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not important at all..but could log it.

@dnwe
Copy link
Collaborator

dnwe commented Jun 2, 2020

@dneralla did you get around to testing your changes against a 2.4/2.5 cluster yet?

@marc-ostrow
Copy link

The positive test case, to verify the feature, though, the replica.selector.class broker configuration must be set to the fully qualified class name that implements ReplicaSelector. This configuration item should be set to org.apache.kafka.common.replica.RackAwareReplicaSelector

@marc-ostrow
Copy link

What's the ETA towards merge?

@dnwe
Copy link
Collaborator

dnwe commented Jun 11, 2020

@marc-ostrow still waiting for some confirmation that this has been tested against a properly configured cluster and shown to work before we can merge the feature. I haven’t had time to do this testing myself sadly

@dneralla
Copy link
Contributor Author

dneralla commented Jun 12, 2020

I haven't found time to test with new kafka cluster as well sadly. Will try to see if I can setup this weekend.

@marc-ostrow
Copy link

marc-ostrow commented Jun 13, 2020

I have a small 3-az MSK cluster running Kafka 2.4.1. On May 20th, I realized I needed to set the cluster configuration to include replica.selector.class set to org.apache.kafka.common.replica.RackAwareReplicaSelector so I did that.

Today, I cloned https://github.com/dneralla/sarama.git and added a little additional log output to the kip-392 branch. I then wrote a small test driver app in ./examples/kip392verify. I ran it on a utility instance for that MSK cluster.

  • The utility instance is in us-east-1a (rack: use1-az1),
  • and the particular topic's 0 partition's leader is on a broker in us-east-1b (rack: use1-az2).

My consumer started out on the leader broker and stayed there. FetchResponseBlock.decode was receiving PreferredReadReplica set to -1. Yes, I verified the FetchRequest had my rack ID (use1-az1) too. ;-(

I searched Cloudwatch broker logs to see what the cluster configuration was set to. The following snippet confirmed to me that I had successfully updated the cluster configuration; this from the broker log just after the restart. Notice the values for replica.selector.class and broker.rack.

CloudWatch Logs Insights
region: us-east-1
log-group-names: /aws/msk/system-events-dev-blue-ref20
start-time: 2020-05-21T01:46:00.000Z
end-time: 2020-05-21T01:47:00.000Z
query-string:

filter @logStream = 'system-events-dev-blue-ref20-e2a3ab84-b5da-4f6b-b434-f0b0a91dfcca-5-Broker-1' and @timestamp > 1590025560000 and @timestamp < 1590025620000
 | fields @timestamp, @message
Click to expand
@timestamp
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,333] INFO Registered kafka:type=kafka.Log4jController MBean (kafka.utils.Log4jControllerRegistration$)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,805] INFO Registered signal handlers for TERM, INT, HUP (org.apache.kafka.common.utils.LoggingSignalHandler)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,805] INFO starting (kafka.server.KafkaServer)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,806] INFO Connecting to zookeeper on INTERNAL_ZK_DNS,INTERNAL_ZK_DNS,INTERNAL_ZK_DNS, (kafka.server.KafkaServer)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,821] INFO [ZooKeeperClient Kafka server] Initializing a new session to INTERNAL_ZK_DNS,INTERNAL_ZK_DNS,INTERNAL_ZK_DNS,. (kafka.zookeeper.ZooKeeperClient)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,826] INFO Client environment:zookeeper.version=3.5.7-f0fdd52973d373ffd9c86b81d99842dc2c7f660e, built on 02/10/2020 11:30 GMT (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,826] INFO Client environment:host.name=HOST_NAME (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,826] INFO Client environment:java.version=1.8.0_242 (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,826] INFO Client environment:java.vendor=Oracle Corporation (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,826] INFO Client environment:java.home=JAVA_HOME (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:java.class.path=CLASSPATH (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:java.io.tmpdir=/tmp (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:java.compiler= (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:os.name=Linux (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:os.arch=amd64 (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:os.version=4.14.171-136.231.amzn2.x86_64 (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:user.name=kafka-user (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:user.home=/home/kafka-user (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:user.dir=/ (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:os.memory.free=1996MB (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:os.memory.max=6144MB (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,827] INFO Client environment:os.memory.total=2048MB (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,829] INFO Initiating client connection, connectString=INTERNAL_ZK_DNS,INTERNAL_ZK_DNS,INTERNAL_ZK_DNS, sessionTimeout=18000 watcher=kafka.zookeeper.ZooKeeperClient$ZooKeeperClientWatcher$@59fd97a8 (org.apache.zookeeper.ZooKeeper)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,833] INFO Setting -D jdk.tls.rejectClientInitiatedRenegotiation=true to disable client-initiated TLS renegotiation (org.apache.zookeeper.common.X509Util)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,837] INFO jute.maxbuffer value is 10485759 Bytes (org.apache.zookeeper.ClientCnxnSocket)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,842] INFO zookeeper.request.timeout value is 0. feature enabled= (org.apache.zookeeper.ClientCnxn)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,843] INFO [ZooKeeperClient Kafka server] Waiting until connected. (kafka.zookeeper.ZooKeeperClient)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,904] INFO Opening socket connection to server INTERNAL_ZK_DNS/INTERNAL_IP. Will not attempt to authenticate using SASL (unknown error) (org.apache.zookeeper.ClientCnxn)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,910] INFO Socket connection established, initiating session, client: /INTERNAL_IP, server: INTERNAL_ZK_DNS/INTERNAL_IP (org.apache.zookeeper.ClientCnxn)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,916] INFO Session establishment complete on server INTERNAL_ZK_DNS/INTERNAL_IP, sessionid = 0x100000953f00019, negotiated timeout = 18000 (org.apache.zookeeper.ClientCnxn)
2020-05-21 01:46:01.000 [2020-05-21 01:46:00,919] INFO [ZooKeeperClient Kafka server] Connected. (kafka.zookeeper.ZooKeeperClient)
2020-05-21 01:46:01.000 [2020-05-21 01:46:01,130] INFO Cluster ID = _HDXTbbgS4eGlQYqtScwBg (kafka.server.KafkaServer)
2020-05-21 01:46:01.000 [2020-05-21 01:46:01,195] INFO KafkaConfig values:
2020-05-21 01:46:01.000 advertised.host.name = null
2020-05-21 01:46:01.000 advertised.listeners = null
2020-05-21 01:46:01.000 advertised.port = null
2020-05-21 01:46:01.000 alter.config.policy.class.name = null
2020-05-21 01:46:01.000 alter.log.dirs.replication.quota.window.num = 11
2020-05-21 01:46:01.000 alter.log.dirs.replication.quota.window.size.seconds = 1
2020-05-21 01:46:01.000 authorizer.class.name = kafka.security.auth.SimpleAclAuthorizer
2020-05-21 01:46:01.000 auto.create.topics.enable = false
2020-05-21 01:46:01.000 auto.leader.rebalance.enable = true
2020-05-21 01:46:01.000 background.threads = 10
2020-05-21 01:46:01.000 broker.id = 1
2020-05-21 01:46:01.000 broker.id.generation.enable = false
2020-05-21 01:46:01.000 broker.rack = use1-az2
2020-05-21 01:46:01.000 client.quota.callback.class = null
2020-05-21 01:46:01.000 compression.type = producer
2020-05-21 01:46:01.000 connection.failed.authentication.delay.ms = 100
2020-05-21 01:46:01.000 connections.max.idle.ms = 600000
2020-05-21 01:46:01.000 connections.max.reauth.ms = 0
2020-05-21 01:46:01.000 control.plane.listener.name = null
2020-05-21 01:46:01.000 controlled.shutdown.enable = true
2020-05-21 01:46:01.000 controlled.shutdown.max.retries = 3
2020-05-21 01:46:01.000 controlled.shutdown.retry.backoff.ms = 5000
2020-05-21 01:46:01.000 controller.socket.timeout.ms = 30000
2020-05-21 01:46:01.000 create.topic.policy.class.name = null
2020-05-21 01:46:01.000 default.replication.factor = 3
2020-05-21 01:46:01.000 delegation.token.expiry.check.interval.ms = 3600000
2020-05-21 01:46:01.000 delegation.token.expiry.time.ms = 86400000
2020-05-21 01:46:01.000 delegation.token.master.key = null
2020-05-21 01:46:01.000 delegation.token.max.lifetime.ms = 604800000
2020-05-21 01:46:01.000 delete.records.purgatory.purge.interval.requests = 1
2020-05-21 01:46:01.000 delete.topic.enable = true
2020-05-21 01:46:01.000 fetch.purgatory.purge.interval.requests = 1000
2020-05-21 01:46:01.000 group.initial.rebalance.delay.ms = 3000
2020-05-21 01:46:01.000 group.max.session.timeout.ms = 1800000
2020-05-21 01:46:01.000 group.max.size = 2147483647
2020-05-21 01:46:01.000 group.min.session.timeout.ms = 6000
2020-05-21 01:46:01.000 host.name =
2020-05-21 01:46:01.000 inter.broker.listener.name = REPLICATION
2020-05-21 01:46:01.000 inter.broker.protocol.version = 2.4.1
2020-05-21 01:46:01.000 kafka.metrics.polling.interval.secs = 10
2020-05-21 01:46:01.000 kafka.metrics.reporters = []
2020-05-21 01:46:01.000 leader.imbalance.check.interval.seconds = 300
2020-05-21 01:46:01.000 leader.imbalance.per.broker.percentage = 10
2020-05-21 01:46:01.000 listener.security.protocol.map = CLIENT:PLAINTEXT,CLIENT_SECURE:SSL,REPLICATION:PLAINTEXT,REPLICATION_SECURE:SSL
2020-05-21 01:46:01.000 listeners = CLIENT://b-1.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com:9092,REPLICATION://INTERNAL_BROKER_DNS
2020-05-21 01:46:01.000 log.cleaner.backoff.ms = 15000
2020-05-21 01:46:01.000 log.cleaner.dedupe.buffer.size = 134217728
2020-05-21 01:46:01.000 log.cleaner.delete.retention.ms = 86400000
2020-05-21 01:46:01.000 log.cleaner.enable = true
2020-05-21 01:46:01.000 log.cleaner.io.buffer.load.factor = 0.9
2020-05-21 01:46:01.000 log.cleaner.io.buffer.size = 524288
2020-05-21 01:46:01.000 log.cleaner.io.max.bytes.per.second = 1.7976931348623157E308
2020-05-21 01:46:01.000 log.cleaner.max.compaction.lag.ms = 9223372036854775807
2020-05-21 01:46:01.000 log.cleaner.min.cleanable.ratio = 0.5
2020-05-21 01:46:01.000 log.cleaner.min.compaction.lag.ms = 0
2020-05-21 01:46:01.000 log.cleaner.threads = 1
2020-05-21 01:46:01.000 log.cleanup.policy = [delete]
2020-05-21 01:46:01.000 log.flush.interval.messages = 9223372036854775807
2020-05-21 01:46:01.000 log.flush.interval.ms = null
2020-05-21 01:46:01.000 log.flush.offset.checkpoint.interval.ms = 60000
2020-05-21 01:46:01.000 log.flush.scheduler.interval.ms = 9223372036854775807
2020-05-21 01:46:01.000 log.flush.start.offset.checkpoint.interval.ms = 60000
2020-05-21 01:46:01.000 log.index.interval.bytes = 4096
2020-05-21 01:46:01.000 log.index.size.max.bytes = 10485760
2020-05-21 01:46:01.000 log.message.downconversion.enable = true
2020-05-21 01:46:01.000 log.message.format.version = 2.4.1
2020-05-21 01:46:01.000 log.message.timestamp.difference.max.ms = 9223372036854775807
2020-05-21 01:46:01.000 log.message.timestamp.type = CreateTime
2020-05-21 01:46:01.000 log.preallocate = false
2020-05-21 01:46:01.000 log.retention.bytes = -1
2020-05-21 01:46:01.000 log.retention.check.interval.ms = 300000
2020-05-21 01:46:01.000 log.retention.hours = 168
2020-05-21 01:46:01.000 log.retention.minutes = null
2020-05-21 01:46:01.000 log.retention.ms = null
2020-05-21 01:46:01.000 log.roll.hours = 168
2020-05-21 01:46:01.000 log.roll.jitter.hours = 0
2020-05-21 01:46:01.000 log.roll.jitter.ms = null
2020-05-21 01:46:01.000 log.roll.ms = null
2020-05-21 01:46:01.000 log.segment.bytes = 1073741824
2020-05-21 01:46:01.000 log.segment.delete.delay.ms = 60000
2020-05-21 01:46:01.000 max.connections = 2147483647
2020-05-21 01:46:01.000 max.connections.per.ip = 2147483647
2020-05-21 01:46:01.000 max.connections.per.ip.overrides =
2020-05-21 01:46:01.000 max.incremental.fetch.session.cache.slots = 1000
2020-05-21 01:46:01.000 message.max.bytes = 1000012
2020-05-21 01:46:01.000 metric.reporters = []
2020-05-21 01:46:01.000 metrics.num.samples = 2
2020-05-21 01:46:01.000 metrics.recording.level = INFO
2020-05-21 01:46:01.000 metrics.sample.window.ms = 30000
2020-05-21 01:46:01.000 min.insync.replicas = 2
2020-05-21 01:46:01.000 num.io.threads = 8
2020-05-21 01:46:01.000 num.network.threads = 5
2020-05-21 01:46:01.000 num.partitions = 1
2020-05-21 01:46:01.000 num.recovery.threads.per.data.dir = 2
2020-05-21 01:46:01.000 num.replica.alter.log.dirs.threads = null
2020-05-21 01:46:01.000 num.replica.fetchers = 2
2020-05-21 01:46:01.000 offset.metadata.max.bytes = 4096
2020-05-21 01:46:01.000 offsets.commit.required.acks = -1
2020-05-21 01:46:01.000 offsets.commit.timeout.ms = 5000
2020-05-21 01:46:01.000 offsets.load.buffer.size = 5242880
2020-05-21 01:46:01.000 offsets.retention.check.interval.ms = 600000
2020-05-21 01:46:01.000 offsets.retention.minutes = 10080
2020-05-21 01:46:01.000 offsets.topic.compression.codec = 0
2020-05-21 01:46:01.000 offsets.topic.num.partitions = 50
2020-05-21 01:46:01.000 offsets.topic.replication.factor = 3
2020-05-21 01:46:01.000 offsets.topic.segment.bytes = 104857600
2020-05-21 01:46:01.000 password.encoder.cipher.algorithm = AES/CBC/PKCS5Padding
2020-05-21 01:46:01.000 password.encoder.iterations = 4096
2020-05-21 01:46:01.000 password.encoder.key.length = 128
2020-05-21 01:46:01.000 password.encoder.keyfactory.algorithm = null
2020-05-21 01:46:01.000 password.encoder.old.secret = null
2020-05-21 01:46:01.000 password.encoder.secret = null
2020-05-21 01:46:01.000 port = 9092
2020-05-21 01:46:01.000 principal.builder.class = null
2020-05-21 01:46:01.000 producer.purgatory.purge.interval.requests = 1000
2020-05-21 01:46:01.000 queued.max.request.bytes = -1
2020-05-21 01:46:01.000 queued.max.requests = 500
2020-05-21 01:46:01.000 quota.consumer.default = 9223372036854775807
2020-05-21 01:46:01.000 quota.producer.default = 9223372036854775807
2020-05-21 01:46:01.000 quota.window.num = 11
2020-05-21 01:46:01.000 quota.window.size.seconds = 1
2020-05-21 01:46:01.000 replica.fetch.backoff.ms = 1000
2020-05-21 01:46:01.000 replica.fetch.max.bytes = 1048576
2020-05-21 01:46:01.000 replica.fetch.min.bytes = 1
2020-05-21 01:46:01.000 replica.fetch.response.max.bytes = 10485760
2020-05-21 01:46:01.000 replica.fetch.wait.max.ms = 500
2020-05-21 01:46:01.000 replica.high.watermark.checkpoint.interval.ms = 5000
2020-05-21 01:46:01.000 replica.lag.time.max.ms = 30000
2020-05-21 01:46:01.000 replica.selector.class = org.apache.kafka.common.replica.RackAwareReplicaSelector
2020-05-21 01:46:01.000 replica.socket.receive.buffer.bytes = 65536
2020-05-21 01:46:01.000 replica.socket.timeout.ms = 30000
2020-05-21 01:46:01.000 replication.quota.window.num = 11
2020-05-21 01:46:01.000 replication.quota.window.size.seconds = 1
2020-05-21 01:46:01.000 request.timeout.ms = 30000
2020-05-21 01:46:01.000 reserved.broker.max.id = 2000
2020-05-21 01:46:01.000 sasl.client.callback.handler.class = null
2020-05-21 01:46:01.000 sasl.enabled.mechanisms = [GSSAPI]
2020-05-21 01:46:01.000 sasl.jaas.config = null
2020-05-21 01:46:01.000 sasl.kerberos.kinit.cmd = /usr/bin/kinit
2020-05-21 01:46:01.000 sasl.kerberos.min.time.before.relogin = 60000
2020-05-21 01:46:01.000 sasl.kerberos.principal.to.local.rules = [DEFAULT]
2020-05-21 01:46:01.000 sasl.kerberos.service.name = null
2020-05-21 01:46:01.000 sasl.kerberos.ticket.renew.jitter = 0.05
2020-05-21 01:46:01.000 sasl.kerberos.ticket.renew.window.factor = 0.8
2020-05-21 01:46:01.000 sasl.login.callback.handler.class = null
2020-05-21 01:46:01.000 sasl.login.class = null
2020-05-21 01:46:01.000 sasl.login.refresh.buffer.seconds = 300
2020-05-21 01:46:01.000 sasl.login.refresh.min.period.seconds = 60
2020-05-21 01:46:01.000 sasl.login.refresh.window.factor = 0.8
2020-05-21 01:46:01.000 sasl.login.refresh.window.jitter = 0.05
2020-05-21 01:46:01.000 sasl.mechanism.inter.broker.protocol = GSSAPI
2020-05-21 01:46:01.000 sasl.server.callback.handler.class = null
2020-05-21 01:46:01.000 security.inter.broker.protocol = PLAINTEXT
2020-05-21 01:46:01.000 security.providers = null
2020-05-21 01:46:01.000 socket.receive.buffer.bytes = 102400
2020-05-21 01:46:01.000 socket.request.max.bytes = 104857600
2020-05-21 01:46:01.000 socket.send.buffer.bytes = 102400
2020-05-21 01:46:01.000 ssl.cipher.suites = []
2020-05-21 01:46:01.000 ssl.client.auth = none
2020-05-21 01:46:01.000 ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
2020-05-21 01:46:01.000 ssl.endpoint.identification.algorithm = https
2020-05-21 01:46:01.000 ssl.key.password = null
2020-05-21 01:46:01.000 ssl.keymanager.algorithm = SunX509
2020-05-21 01:46:01.000 ssl.keystore.password = null
2020-05-21 01:46:01.000 ssl.keystore.type = JKS
2020-05-21 01:46:01.000 ssl.principal.mapping.rules = DEFAULT
2020-05-21 01:46:01.000 ssl.protocol = TLS
2020-05-21 01:46:01.000 ssl.provider = null
2020-05-21 01:46:01.000 ssl.secure.random.implementation = null
2020-05-21 01:46:01.000 ssl.trustmanager.algorithm = PKIX
2020-05-21 01:46:01.000 ssl.truststore.password = null
2020-05-21 01:46:01.000 ssl.truststore.type = JKS
2020-05-21 01:46:01.000 transaction.abort.timed.out.transaction.cleanup.interval.ms = 60000
2020-05-21 01:46:01.000 transaction.max.timeout.ms = 900000
2020-05-21 01:46:01.000 transaction.remove.expired.transaction.cleanup.interval.ms = 3600000
2020-05-21 01:46:01.000 transaction.state.log.load.buffer.size = 5242880
2020-05-21 01:46:01.000 transaction.state.log.min.isr = 2
2020-05-21 01:46:01.000 transaction.state.log.num.partitions = 50
2020-05-21 01:46:01.000 transaction.state.log.replication.factor = 3
2020-05-21 01:46:01.000 transaction.state.log.segment.bytes = 104857600
2020-05-21 01:46:01.000 transactional.id.expiration.ms = 604800000
2020-05-21 01:46:01.000 unclean.leader.election.enable = true
2020-05-21 01:46:01.000 zookeeper.connect = INTERNAL_ZK_DNS,INTERNAL_ZK_DNS,INTERNAL_ZK_DNS,
2020-05-21 01:46:01.000 zookeeper.connection.timeout.ms = null
2020-05-21 01:46:01.000 zookeeper.max.in.flight.requests = 10
2020-05-21 01:46:01.000 zookeeper.session.timeout.ms = 18000
2020-05-21 01:46:01.000 zookeeper.set.acl = false
2020-05-21 01:46:01.000 zookeeper.sync.time.ms = 2000
2020-05-21 01:46:01.000 (kafka.server.KafkaConfig)

@dneralla
Copy link
Contributor Author

@marc-ostrow Did u set the rack id in the consumer config ?

@marc-ostrow
Copy link

@dneralla Yes, I did. Here's some initial log output showing the brokers, their rack id's, the Leader broker's rack ID, and the rack id in the encode and the preferred replica in the decode of fetch request/response.

[ec2-user@ip-10-232-166-105 ~]$ ./kip392verify -brokers $BROKERS -topic CCR-Signal-Quality -verbose true
[kip392_verify] 2020/06/15 14:35:12 My Rack: use1-az1 (us-east-1a)
[kip392_verify] 2020/06/15 14:35:12 MSK Broker Rack: use1-az2 (us-east-1b), Address: b-1.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com
[kip392_verify] 2020/06/15 14:35:12 MSK Broker Rack: use1-az4 (us-east-1c), Address: b-2.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com
[kip392_verify] 2020/06/15 14:35:12 MSK Broker Rack: use1-az1 (us-east-1a), Address: b-3.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com
[sarama] 2020/06/15 14:35:12 Initializing new client
[sarama] 2020/06/15 14:35:12 client/metadata fetching metadata for all topics from broker b-2.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com:9092
[sarama] 2020/06/15 14:35:12 Connected to broker at b-2.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com:9092 (unregistered)
[sarama] 2020/06/15 14:35:12 client/brokers registered new broker #2 at b-2.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com:9092
[sarama] 2020/06/15 14:35:12 client/brokers registered new broker #3 at b-3.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com:9092
[sarama] 2020/06/15 14:35:12 client/brokers registered new broker #1 at b-1.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com:9092
[sarama] 2020/06/15 14:35:12 Successfully initialized new client
[kip392_verify] 2020/06/15 14:35:12 Sarama Broker: ID: 2, Rack: use1-az4 Address: b-2.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com:9092
[kip392_verify] 2020/06/15 14:35:12 Sarama Broker: ID: 3, Rack: use1-az1 Address: b-3.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com:9092
[kip392_verify] 2020/06/15 14:35:12 Sarama Broker: ID: 1, Rack: use1-az2 Address: b-1.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com:9092
[kip392_verify] 2020/06/15 14:35:12 Leader Broker for topic "CCR-Signal-Quality": ID: 1, Rack: use1-az2, Address: b-1.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com:9092
[sarama] 2020/06/15 14:35:12 Connected to broker at b-1.system-events-dev-blue.uwjleq.c5.kafka.us-east-1.amazonaws.com:9092 (registered as #1)
[sarama] 2020/06/15 14:35:12 consumer/broker/1 added subscription to CCR-Signal-Quality/0
[sarama] 2020/06/15 14:35:12 encode version 11,  RackID use1-az1
[sarama] 2020/06/15 14:35:12 encode version 11,  RackID use1-az1
[sarama] 2020/06/15 14:35:12 decoded version 11, PreferredReadReplica -1

@marc-ostrow
Copy link

marc-ostrow commented Jun 16, 2020

@dneralla Hi, so I also tried running a docker network cluster of three ZK, and three Kafka nodes using confluentinc/cp-kafka:5.4.1 which I believe incorporates Kafka 2.4.1.

I actually modified the fetch handler to log the client metadata so I could verify we were indeed sending the rackId; and we are. I cloned the confluentinc/kafka repo and checked out the 5.4.1 tag, made my mods, and built an unsigned tarball. I extracted kafka_2.12-5.4.1-ccs/libs/kafka_2.12-5.4.1-ccs.jar from the kafka_2.12-5.4.1-ccs.tgz tarball. I then built a new image from the confluentinc/cp-kafka:5.4.1 base image rewriting the VOLUME and CMD dockerfile commands to match the cp-docker-images dockerfile.

FROM confluentinc/cp-kafka:5.4.1
COPY kafka_2.12-5.4.1-ccs.jar /usr/share/java/kafka/
VOLUME ["/var/lib/kafka/data", "/etc/kafka/secrets"]
CMD ["/etc/confluent/docker/run"]

My diffs were as follows

diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 124b2e356..6018303a3 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -609,6 +609,10 @@ class KafkaApis(val requestChannel: RequestChannel,
       None
     }
 
+    if (versionId >= 11) {
+      info("handleFetchRequest: clientMetadata is " + clientMetadata.getOrElse("not set"))
+    }
+
     def errorResponse[T >: MemoryRecords <: BaseRecords](error: Errors): FetchResponse.PartitionData[T] = {
       new FetchResponse.PartitionData[T](error, FetchResponse.INVALID_HIGHWATERMARK, FetchResponse.INVALID_LAST_STABLE_OFFSET,
         FetchResponse.INVALID_LOG_START_OFFSET, null, MemoryRecords.EMPTY)

Btw, I highly suggest we modify the pull request and change any protocol version checks against version 11 api, to be true for any version greater than or equal to 11.

I ran my docker network following the confluent.io clustered-deployment example as follows:

Click to expand

create-zoo-cluster

#!/bin/sh

docker run -d \
    --rm \
    --net=host \
    --name=zk-1 \
    -e ZOOKEEPER_SERVER_ID=1 \
    -e ZOOKEEPER_CLIENT_PORT=22181 \
    -e ZOOKEEPER_TICK_TIME=2000 \
    -e ZOOKEEPER_INIT_LIMIT=5 \
    -e ZOOKEEPER_SYNC_LIMIT=2 \
    -e ZOOKEEPER_SERVERS="localhost:22888:23888;localhost:32888:33888;localhost:42888:43888" \
    -e KAFKA_OPTS="-Dzookeeper.4lw.commands.whitelist=*" \
    confluentinc/cp-zookeeper:5.4.1

docker run -d \
    --rm \
    --net=host \
    --name=zk-2 \
    -e ZOOKEEPER_SERVER_ID=2 \
    -e ZOOKEEPER_CLIENT_PORT=32181 \
    -e ZOOKEEPER_TICK_TIME=2000 \
    -e ZOOKEEPER_INIT_LIMIT=5 \
    -e ZOOKEEPER_SYNC_LIMIT=2 \
    -e ZOOKEEPER_SERVERS="localhost:22888:23888;localhost:32888:33888;localhost:42888:43888" \
    -e KAFKA_OPTS="-Dzookeeper.4lw.commands.whitelist=*" \
    confluentinc/cp-zookeeper:5.4.1

docker run -d \
    --rm \
    --net=host \
    --name=zk-3 \
    -e ZOOKEEPER_SERVER_ID=3 \
    -e ZOOKEEPER_CLIENT_PORT=42181 \
    -e ZOOKEEPER_TICK_TIME=2000 \
    -e ZOOKEEPER_INIT_LIMIT=5 \
    -e ZOOKEEPER_SYNC_LIMIT=2 \
    -e ZOOKEEPER_SERVERS="localhost:22888:23888;localhost:32888:33888;localhost:42888:43888" \
    -e KAFKA_OPTS="-Dzookeeper.4lw.commands.whitelist=*" \
    confluentinc/cp-zookeeper:5.4.1

check-zoo-cluster

#!/bin/sh

for i in 22181 32181 42181; do
    docker run --net=host --rm confluentinc/cp-zookeeper:5.4.1 bash -c "echo stat | nc localhost $i | grep Mode"
done

create-kafka-cluster

#!/bin/sh

docker run -d \
    --rm \
    --net=host \
    --name=kafka-1 \
    -e KAFKA_ZOOKEEPER_CONNECT=localhost:22181,localhost:32181,localhost:42181 \
    -e KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://localhost:29092 \
    -e KAFKA_BROKER_ID=1 \
    -e KAFKA_BROKER_RACK=Rack-1 \
    -e KAFKA_REPLICA_SELECTOR_CLASS=org.apache.kafka.common.replica.RackAwareReplicaSelector \
    local/kafka

#    confluentinc/cp-kafka:5.4.1

docker run -d \
    --rm \
    --net=host \
    --name=kafka-2 \
    -e KAFKA_ZOOKEEPER_CONNECT=localhost:22181,localhost:32181,localhost:42181 \
    -e KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://localhost:39092 \
    -e KAFKA_BROKER_ID=2 \
    -e KAFKA_BROKER_RACK=Rack-2 \
    -e KAFKA_REPLICA_SELECTOR_CLASS=org.apache.kafka.common.replica.RackAwareReplicaSelector \
    local/kafka

#    confluentinc/cp-kafka:5.4.1

docker run -d \
    --rm \
    --net=host \
    --name=kafka-3 \
    -e KAFKA_ZOOKEEPER_CONNECT=localhost:22181,localhost:32181,localhost:42181 \
    -e KAFKA_ADVERTISED_LISTENERS=PLAINTEXT://localhost:49092 \
    -e KAFKA_BROKER_ID=3 \
    -e KAFKA_BROKER_RACK=Rack-3 \
    -e KAFKA_REPLICA_SELECTOR_CLASS=org.apache.kafka.common.replica.RackAwareReplicaSelector \
    local/kafka

#    confluentinc/cp-kafka:5.4.1

create-topic

#!/bin/sh

docker run \
    --rm \
    --net=host \
    confluentinc/cp-kafka:5.4.1 kafka-topics \
        --bootstrap-server localhost:29092,localhost:39092,localhost:49092 \
        --create --topic bar --partitions 1 --replication-factor 3

docker run \
    --rm \
    --net=host \
    confluentinc/cp-kafka:5.4.1 kafka-topics \
        --bootstrap-server localhost:29092,localhost:39092,localhost:49092 \
        --describe --topic bar

With each broker having a broker.id, broker.rack and replica.selector.class configuration setting configured, and a topic with no activity, I finally saw the FetchResponse come back with the expected broker.id of the replica broker having a matching RackID as my client.

Btw, I further modified my instrumentation as follows to reduce log output:

diff --git a/fetch_request.go b/fetch_request.go
index cffc339..9802ae2 100644
--- a/fetch_request.go
+++ b/fetch_request.go
@@ -65,6 +65,8 @@ const (
        ReadCommitted
 )
 
+var encodeCount = uint64(0)
+
 func (r *FetchRequest) encode(pe packetEncoder) (err error) {
        pe.putInt32(-1) // replica ID is always -1 for clients
        pe.putInt32(r.MaxWaitTime)
@@ -124,7 +126,10 @@ func (r *FetchRequest) encode(pe packetEncoder) (err error) {
                if err != nil {
                        return err
                }
-               Logger.Printf("encode version %d,  RackID %s\n", r.Version, r.RackID)
+               if encodeCount%100 == 0 {
+                       Logger.Printf("encode version %d,  RackID %s\n", r.Version, r.RackID)
+               }
+               encodeCount++
        }
 
        return nil
diff --git a/fetch_response.go b/fetch_response.go
index a60e948..291ed9f 100644
--- a/fetch_response.go
+++ b/fetch_response.go
@@ -41,6 +41,8 @@ type FetchResponseBlock struct {
        Partial              bool
 }
 
+var decodeCount = uint64(0)
+
 func (b *FetchResponseBlock) decode(pd packetDecoder, version int16) (err error) {
        tmp, err := pd.getInt16()
        if err != nil {
@@ -89,7 +91,10 @@ func (b *FetchResponseBlock) decode(pd packetDecoder, version int16) (err error)
                if err != nil {
                        return err
                }
-               Logger.Printf("decoded version %d, PreferredReadReplica %d\n", version, b.PreferredReadReplica)
+               if decodeCount%100 == 0 {
+                       Logger.Printf("decoded version %d, PreferredReadReplica %d\n", version, b.PreferredReadReplica)
+               }
+               decodeCount++
        }
 
        recordsSize, err := pd.getInt32()

I experienced the following:

  • With no message traffic, I did see PreferredReadReplica set to the broker id of the broker matching my client's RackID.
  • With constant message traffic, I saw PreferredReadReplica set to -1; I don't remember if my client was reporting messages.
  • With constant message traffic, I saw PreferredReadReplica set to the expected broker id, but my client was not reporting consumed messages.
  • When I was testing in AWS on MSK with Kafka 2.4.1, as stated above, I only saw PreferredReadReplica set to -1, but my client was reporting consumed messages. Also, if I didn't mention before, I had 0 under replicated partitions for each broker.

Sorry... will have to try again next weekend if I have time.

AusIV added a commit to openrelayxyz/sarama that referenced this pull request Jun 19, 2020
Rather than always using the leader partition, this lets the consumer
application specify a function to select which broker they wish to
consume from. The function has the signature

    fn(topic string, partition int32, client Client) (*Broker, error)

Allowing the client to implement their own logic for broker slection.
If none is specified, the default behavior is backwards compatible
and selects the leader.

This pulls some changes from IBM#1696, but in our use case we want to
spread the load across multiple replicas, not pick based on geographic
considerations. I didn't think our specific use case made sense for
the general library, but it also wasn't possible without changes to
Sarama, so I made a generalizable solution. It might be possible to
implement the changes for KIP-392 as a ReplicaSelector function,
though that may still require some additional code changes to support
communications regarding which broker should be used for a given rack.
@@ -617,6 +626,12 @@ func (child *partitionConsumer) parseResponse(response *FetchResponse) ([]*Consu
child.fetchSize = child.conf.Consumer.Fetch.Default
atomic.StoreInt64(&child.highWaterMarkOffset, block.HighWaterMarkOffset)

if response.Version == 11 && len(child.consumer.conf.RackID) > 0 {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I accidently removed my suggestion. Doing it again.

To support additional Fetch protocol changes, the comparison with version 11 should allow this.
change
response.Version == 11
to
response.Version >= 11

@marc-ostrow
Copy link

marc-ostrow commented Jul 19, 2020

Our major concern with cross-az traffic is with bringing in upwards to 7000 mpeg transport streams. Not really an idiomatic use of Kafka, but it works using single partition topics, one for each stream. If we had this feature, assuming the cluster was performing appropriately, we wouldn't have to worry so much about the cost of cross-az traffic with our consumers of these streams. However, we would still have to watch the leader brokers per partition/topic, and move the producers into the same AZ to eliminate cross-az traffic.

We've basically opted to go the route of gRPC for these streams which means all consumers will have to operate within the same AZ as their gRPC server to reduce cross-az traffic.

BUT, has anyone gotten this pull request to work, and verify the bulk of traffic is between a client and its nearest replica (that being a replica in its rack)?

@marc-ostrow
Copy link

Anyone get this working?

@angeloskaltsikis
Copy link

Hello,
Is there any ETA for this to be merged?
Is anything that i could do to help with this?
Angelos

@dnwe
Copy link
Collaborator

dnwe commented Oct 2, 2020

@angeloskaltsikis i believe it’s still waiting for someone to test the changes and confirm they function as expected by analysing the traffic. The earlier comments seemed to suggest it wasn’t working as it should do we wouldn’t want to merge and advertise the functionality until it was

danp added a commit to danp/sarama that referenced this pull request Oct 3, 2020
Address comments on IBM#1696.

Change to only adding and using a new preferredReadReplica field.

Move setting of the preferredReadReplica field up in
parseResponse. This causes the returned preferred read replica to be
used even if there are no records in the fetch response.

Set FetchResponse.PreferredReadReplica to -1 when decoding versions
prior to 11.

Add more tests.
danp added a commit to danp/sarama that referenced this pull request Oct 4, 2020
Address comments on IBM#1696.

Change to only adding and using a new preferredReadReplica field.

Move setting of the preferredReadReplica field up in
parseResponse. This causes the returned preferred read replica to be
used even if there are no records in the fetch response.

Set FetchResponse.PreferredReadReplica to -1 when decoding versions
prior to 11.

Add more tests.
danp added a commit to danp/sarama that referenced this pull request Oct 4, 2020
Address comments on IBM#1696.

Change to only adding and using a new preferredReadReplica field.

Move setting of the preferredReadReplica field up in
parseResponse. This causes the returned preferred read replica to be
used even if there are no records in the fetch response.

Set FetchResponse.PreferredReadReplica to -1 when decoding versions
prior to 11.

Add more tests.
danp added a commit to danp/sarama that referenced this pull request Oct 4, 2020
Address comments on IBM#1696.

Change to only adding and using a new preferredReadReplica field.

Move setting of the preferredReadReplica field up in
parseResponse. This causes the returned preferred read replica to be
used even if there are no records in the fetch response.

Set FetchResponse.PreferredReadReplica to -1 when decoding versions
prior to 11.

Add more tests.

Change consumeMsgs in functional tests to use subtests to reduce
number of outstanding connections.
@danp
Copy link
Contributor

danp commented Oct 4, 2020

Hi there,

I spent some time validating this and, hopefully, improving it. Hope you don't mind me taking it a bit further, @dneralla!

My branch is here. These are the main changes from this PR:

  • Addressed feedback
  • Tightened up diff a bit (removed unrelated whitespace changes, etc)
  • Changed to only using a preferredReadReplica field, -1 indicates no preference
  • Moved up setting preferredReadReplica in parseResponse, which sets if if there are no records in the response
  • Set FetchResponse.PreferredReadReplica to -1 when decoding versions prior to 11
  • Add more tests to cover error cases and capture the behavior I observed that -1 is returned when no change should be made
  • Change consumeMsgs in functional tests to use subtests to reduce number of outstanding connections

I also set up a test rig at https://github.com/danp/sarama-rack. It let me send some messages to a cluster in docker-compose and see things working. Maybe that will help others validate this as well.

Feel free to integrate my changes however you like, or let me know if there's anything else I can do.

@ssorren
Copy link

ssorren commented Oct 28, 2020

Any ETA on this merge? One thing to note - without this change, if you make the mistake of setting the RackId on the consumer client config with a rack aware kafka cluster, you may or may not consume messages. Luck of the draw

@dneralla
Copy link
Contributor Author

dneralla commented Oct 28, 2020 via email

@angeloskaltsikis
Copy link

Hi there,

I spent some time validating this and, hopefully, improving it. Hope you don't mind me taking it a bit further, @dneralla!

My branch is here. These are the main changes from this PR:

  • Addressed feedback
  • Tightened up diff a bit (removed unrelated whitespace changes, etc)
  • Changed to only using a preferredReadReplica field, -1 indicates no preference
  • Moved up setting preferredReadReplica in parseResponse, which sets if if there are no records in the response
  • Set FetchResponse.PreferredReadReplica to -1 when decoding versions prior to 11
  • Add more tests to cover error cases and capture the behavior I observed that -1 is returned when no change should be made
  • Change consumeMsgs in functional tests to use subtests to reduce number of outstanding connections

I also set up a test rig at https://github.com/danp/sarama-rack. It let me send some messages to a cluster in docker-compose and see things working. Maybe that will help others validate this as well.

Feel free to integrate my changes however you like, or let me know if there's anything else I can do.

After some very basic testing with (sarama-rack)[https://github.com/danp/sarama-rack] some weeks ago, it seemed that the fork of @danp worked as expected. Can someone else try the changes? It's very easy to setup with docker.
I can try to test this feature more extensively as well.

Any certain tests you suggest that we run @dnwe ?

The thing i believe that most people will agree with, is that as soon as this feature is tested and released in sarama it is going to save a LOT of Inter-AZ/Cross-AZ Traffic Costs to a lot of applications using sarama as the base library.

@bai
Copy link
Contributor

bai commented Oct 29, 2020

@danp could you please submit a PR with changes you've implemented on top of this one?

@danp
Copy link
Contributor

danp commented Oct 29, 2020

Sure, opened #1822.

@dnwe
Copy link
Collaborator

dnwe commented Nov 6, 2020

Superceded by #1822

@dnwe dnwe closed this Nov 6, 2020
dnwe pushed a commit that referenced this pull request Nov 6, 2020
Support for [KIP-392](https://cwiki.apache.org/confluence/display/KAFKA/KIP-392%3A+Allow+consumers+to+fetch+from+closest+replica) in Sarama consumers.

This is a continuation of #1696 with changes mentioned [here](#1696 (comment)).

Co-authored-by: Deepak.Neralla <Deepak.Neralla@mixpanel.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

9 participants