Thanks a lot, Guozhang! That clarifies it to a great extent. I'll try to figure out who the leader of the group is.
On Wed, Mar 27, 2019 at 9:59 PM Guozhang Wang <[email protected]> wrote: > Hello Anirudh, > > The config `upgrade.from` is recommended for safe and smooth upgrade. In > your case it is possible that when rolling bounce the instances the first > upgraded instance happen to be the leader of the group and hence even > without the config it can recognize other instances; but if you are in bad > luck and the leader is bounced later, then it would not be able to > recognize other instances and hence cause it to crash. So in some words, > setting is config would be a safe choice but it does not mean you are > doomed to fail the upgrade if you do not execute this way. > > > Guozhang > > On Tue, Mar 26, 2019 at 10:38 PM Anirudh Vyas <[email protected]> > wrote: > > > Hey, > > We run 3 instances. > > > > Anirudh > > > > On Tue, Mar 26, 2019 at 9:28 PM Matthias J. Sax <[email protected]> > > wrote: > > > > > Not sure. How many instances to do you run? If it's only one, you don't > > > need the config. > > > > > > -Matthias > > > > > > On 3/26/19 5:17 AM, Anirudh Vyas wrote: > > > > Hi, > > > > I am in the process of upgrading my Kafka streams services from 1.1 > to > > > > 2.1.0. I am following the upgrade guide: > > > > https://kafka.apache.org/20/documentation/streams/upgrade-guide . > > > > > > > > My service is running on kafka version 2.0 and using kafka streams > > > 1.1.1. I > > > > updated my kafka-streams to 2.1.0 but DID NOT pass the config value > > > > `upgrade.from` (it is null), as can be verified from the logs > > > > ``` > > > > [INFO ] 2019-03-26 18:05:49,550 [main] > > > > org.apache.kafka.streams.StreamsConfig:logAll: StreamsConfig values: > > > > application.id = application_id > > > > application.server = > > > > bootstrap.servers = [bootstrap-server-01:6667, > > > > bootstrap-server-02:6667] > > > > buffered.records.per.partition = 10000 > > > > cache.max.bytes.buffering = 10485760 > > > > client.id = > > > > commit.interval.ms = 15000 > > > > connections.max.idle.ms = 540000 > > > > default.deserialization.exception.handler = class > > > > org.apache.kafka.streams.errors.LogAndFailExceptionHandler > > > > default.key.serde = class > > > > org.apache.kafka.common.serialization.Serdes$ByteArraySerde > > > > default.production.exception.handler = class > > > > org.apache.kafka.streams.errors.DefaultProductionExceptionHandler > > > > default.timestamp.extractor = class > > > > ziggurat.timestamp_transformer.IngestionTimeExtractor > > > > default.value.serde = class > > > > org.apache.kafka.common.serialization.Serdes$ByteArraySerde > > > > max.task.idle.ms = 0 > > > > metadata.max.age.ms = 300000 > > > > metric.reporters = [] > > > > metrics.num.samples = 2 > > > > metrics.recording.level = INFO > > > > metrics.sample.window.ms = 30000 > > > > num.standby.replicas = 0 > > > > num.stream.threads = 3 > > > > partition.grouper = class > > > > org.apache.kafka.streams.processor.DefaultPartitionGrouper > > > > poll.ms = 100 > > > > processing.guarantee = at_least_once > > > > receive.buffer.bytes = 32768 > > > > reconnect.backoff.max.ms = 1000 > > > > reconnect.backoff.ms = 50 > > > > replication.factor = 1 > > > > request.timeout.ms = 40000 > > > > retries = 0 > > > > retry.backoff.ms = 100 > > > > rocksdb.config.setter = null > > > > security.protocol = PLAINTEXT > > > > send.buffer.bytes = 131072 > > > > state.cleanup.delay.ms = 600000 > > > > state.dir = /tmp/kafka-streams > > > > topology.optimization = none > > > > upgrade.from = null > > > > windowstore.changelog.additional.retention.ms = 86400000 > > > > ``` > > > > > > > > The application is consuming messages as expected and is not failing. > > > > > > > > I even went through the steps mentioned in the upgrade guide and did > 2 > > > > rolling bounces with the correct config in place. Then I rolled back > my > > > > application to Kafka streams 1.1.1 and it was running as expected. > > > > > > > > I went through KIP-268 > > > > < > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-268%3A+Simplify+Kafka+Streams+Rebalance+Metadata+Upgrade > > > > > > > > and > > > > it mentions `Kafka Streams need to be configured with > > > > upgrade.from="<old.version>"for startup`, but the service runs fine > > even > > > if > > > > I don't configure it. > > > > It also mentions `user prepares a second round of rebalance; this > time, > > > the > > > > configuration parameter upgrade.from must be removed for new startup` > > but > > > > the application runs as expected even if I don't remove the config > > > > parameter. > > > > > > > > So, my question is, what is the significance of the variable > > > `upgrade.from` > > > > as it does not seem to have any effect on our kafka streams service. > > > > > > > > Regards, > > > > Anirudh > > > > > > > > > > > > > > > -- > -- Guozhang >
