You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Joel Koshy <jj...@gmail.com> on 2014/02/12 20:50:53 UTC

Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/
-----------------------------------------------------------

(Updated Feb. 12, 2014, 7:50 p.m.)


Review request for kafka.


Bugs: KAFKA-1012
    https://issues.apache.org/jira/browse/KAFKA-1012


Repository: kafka


Description
-------

I picked up most of Tejas' patch and made various edits for review here as I
would like this to be completed and closed.

Here is a link to the original implementation wiki:
https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management

A lot of it is the same in this revision of the patch, but there is a bunch
of refactoring. This patch does not use an "embedded producer" in the
consumer. i.e., the consumer issues offset commit/fetch requests directly to
the broker. Also, I decided against doing any kind of request forwarding and
added a "ConsumerMetadataRequest" that will be used to determine the offset
coordinator (and subsequently group coordinator that may be useful for the
client rewrite - see
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
Also, there were some questions on how to support multiple offset manager
implementations cleanly. After thinking about it I think it makes the code
simpler and clearer if we just have one good offset storage mechanism (i.e.,
Kafka-based). Consumers that want to store offsets elsewhere can do so on
their own. (However, if we do want to do this somewhat cleanly, see the
discussion on separation of APIs below.)

Here is a quick recap of how offset management within Kafka works:
- A special __offsets topic holds consumer offsets.
- The consumer group serves as the partition key for offsets committed to
  the __offsets topic. i.e., all offsets for all topics that a group
  consumes will be in a single partition of the offsets topic.
- The "group-topic-partition" is the actual (stored) key in each message of
  the offsets topic.  This facilitates de-duplication (and thus removal) of
  older offsets.
- The offset manager also contains an in-memory cache of offsets so it can
  serve offset fetch requests quickly.
- Think of commits as a little more than a produce request. If and only if
  the commit is appended to the __offsets log as a regular produce request
  we update the offsets cache. So the semantics are identical to a produce
  request.  Offset fetches return whatever is in the cache. If it is absent,
  and offsets have not yet been loaded from the logs into the cache (on
  becoming a leader), then we return an "OffsetsLoading" error code.

(Tejas' wiki has pretty good diagrams that describe the above.)

Some more details:

- Atomicity per-commit: One drawback of the Zookeeper-based offset commits
  is that we when we commit multiple offsets (since we don't use
  multi-write) we have to write offsets serially so it is not atomic.  In
  this implementation I went with Jun's suggestion on using a compressed
  message set. This ensures that we will disallow partial commits of a bulk
  commit. I have hard-coded this to GZIP but maybe it is better to just
  expose a config. Another option is to introduce an identity compression
  codec.
- The main corner cases to consider are when there is leader movement due to
  broker failures and simultaneous offset commits/fetches. Offset fetches
  would only occur if there are consumer-side rebalances or shutdowns. The
  guarantees we want to provide are: (i) successfully acknowledged offset
  commits should be returned on the next offset fetch - i.e., should not be
  lost (ii) offset fetches should never return a stale offset.
  - On becoming a follower of an offsets topic partition:
    - Partition.makeFollower clears the offset cache of entries belonging to
      this partition of __offsets.
    - Any subsequent offset fetch request will find out that the partition
      is no longer a leader and fail. There is one problem in the existing
      patch which I will highlight in the RB along with a suggested fix.
    - Likewise, any subsequent offset commit request will fail (since the
      underlying producer request will fail). It is okay if the underlying
      producer request succeeds and the broker becomes a follower for that
      partition just before the offset cache is updated (since the broker
      will not serve any OffsetFetchRequests for that partition until it
      becomes a leader again).
  - On becoming a leader of an offsets topic partition:
    - Partition.makeLeader: will load the offsets from the log
      (asynchronously). While this is in progress, the broker rejects offset
      fetches to this partition. Offset commits may continue to arrive -
      i.e., will be appended to the log and then written to the cache. The
      load loop might actually overwrite it with an earlier offset from the
      log but that is okay - since it will eventually reach the more recent
      update in the log and load that into the cache.

Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
- The broker config should set offsets.backup.enabled=true
- Upgrade the brokers to the latest jar. (Consumers still commit
  directly to ZooKeeper).
- Start migrating the consumers over.
- Consumers will now start sending offset commits to the broker. Since the
  backup setting is enabled, offsets will also be committed to ZooKeeper.
  This is necessary when migrating consumers.
- After _all_ consumers have moved over you can turn off the backup.

I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
as a note to myself and others reviewing).

Questions/comments for discussion
- Should we explicitly disallow changes to the number of offset topic partitions?
  This is necessary (or at least prompt with a warning) since changing the number
  of partitions would affect the partitioning strategy.
- Should we remove per-partition error codes for offset commits and use just
  a global error code for the entire request? I'm using compressed message
  sets for commits.  i.e., the log append for a given commit will either
  fail entirely or succeed entirely. The OffsetCommitResponse contains
  per-partition error codes. So if the log append fails for any reason the
  same error code would apply for all partitions. i.e., it is sufficient to
  have a global error code. I think we currently have per-partition error
  codes due to the fact that offset commit requests can include metadata for
  each offset. The per-partition error code is set to MetadataTooLarge if
  the metadata entry exceeds the MaxMetadataLength. However, in this case I
  would prefer to just fail the entire request as opposed to doing partial
  commits (as I am in the current patch). Anyone have thoughts on this?
- Error codes: right now I'm using existing error codes (with the exception
  of OffsetsLoading). It may be better to return more specific error codes
  but I'm not sure if it matters - since the client-side implementation
  needs to check for _any_ error and if any error exists (other than
  MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
  i.e., the client should not really care about the actual error. If people
  have any strong preference on this let me know.
- Separation of APIs: Right now, the offset manager, replica manager are
  intertwined which is less than ideal. It is okay if offset manager depends
  on replica manager but not the other way around. Ideally, I would like to
  have KafkaApis hand off offset commit/fetch requests to the offset manager
  which then handles it. However, the inter-dependence comes about due to
  the need to clear out the offset cache on becoming a follower and the need
  to load offsets on becoming a leader. I think we can improve the
  separation as follows:
  - Don't optimistically load offsets/clear offsets on a leader/follower
    transition. Instead, load offsets only when an offset fetch request
    arrives for a partition that had not been loaded yet.
  - The OffsetManager will need to maintain a Map[partition ->
    lastKnownLeaderEpoch] to determine whether to load offsets or not.
  - The above will eliminate the reference to OffsetManager from
    ReplicaManager. KafkaApis still needs to reference the OffsetManager and
    will need to create the offset commit message to append to the __offsets
    log.
  - We can actually avoid the need for KafkaApis to know about offset commit
    messsages as well: in order to do that, we will need to create a
    "DurableLog" layer on top of LogManager and move all the purgatory stuff
    in there. The LogManager supports appends/reads from the local log, but
    does not know anything about the replicas. Instead, we can have a
    DurableLog layer that depends on ReplicaManager and LogManager and
    contains the Producer/Fetch-Request purgatories. So OffsetManager will
    need to depend on this DurableLog component. So KafkaApis can just hand
    off ProducerRequests, FetchRequests to the DurableLog layer directly. It
    will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
    which will then hand it off to the DurableLog layer.
  - Is the above worth it? I'm not sure it is, especially if we are sticking
    to only one offset management implementation.


Diffs
-----

  core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
  core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
  core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
  core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
  core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
  core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
  core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
  core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
  core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
  core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
  core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
  core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
  core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
  core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
  core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
  core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
  core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
  core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
  core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
  core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
  core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
  core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
  core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
  core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
  core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
  core/src/main/scala/kafka/utils/Utils.scala a89b046 
  core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
  core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
  core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
  core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
  sbt 944ebf8 
  system_test/mirror_maker/README da53c14 
  system_test/mirror_maker/bin/expected.out 0a1bbaf 
  system_test/mirror_maker/bin/run-test.sh e5e6c08 
  system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
  system_test/mirror_maker/config/mirror_producer.properties aa8be65 
  system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
  system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
  system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
  system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
  system_test/mirror_maker/config/server_target_1_1.properties d37955a 
  system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
  system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
  system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
  system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
  system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
  system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
  system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
  system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
  system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
  system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
  system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
  system_test/testcase_to_run.json 8252860 
  system_test/utils/kafka_system_test_utils.py fb4a9c0 
  system_test/utils/testcase_env.py bee8716 

Diff: https://reviews.apache.org/r/18022/diff/


Testing
-------


Thanks,

Joel Koshy


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Joel Koshy <jj...@gmail.com>.

> On Feb. 19, 2014, 1:25 a.m., Guozhang Wang wrote:
> > Some general comments:
> > 
> > 1. sbt: do we really want to delete this file in this patch?
> > 
> > 2. offset retention minutes: 
> > 
> > I was not aware of the retention before. I think this is like the consumer heartbeat scheme: the retention minutes is defined globally on broker and the offset commit behavior is customizable on consumers, so we need to be very conservative on the server-side global retention parameter since this needs to be the MAX of all consumers. Or we can make this parameter as group-specific, but that may slightly complicate the logic on server side.
> > 
> > 3. LoadingOffsetException seems not used anywhere. We should check this when getting offset from cache.
> > 
> > 4. Currently the append-to-log operation, which is the synchronous-point of the commit-operation is not actually in the offset manager, the offset manager actually only takes care of the cache operations. That is a little awkward. Following Joel and Jun's suggestion I would propose the following:
> > 
> > KafkaApi : 
> > 
> > 1. handle produce request and commit offset request separately. In handling commit offset request, hand it to offset manager. Keep handling produce request as it is and hand to log manager.
> > 
> > 2. In handling leader and ISR, call makeLeader/Follower of log manager, then optionally call migrate offset manager of Offset Manager. Delaying the migration of offset manager is actually fine since the offset manager would check leader replica Id for fetching/committing anyways, and after makerLeader/Follower has finished subsequent fetch/commit would all be rejected since leader replica Id has changed. The consumer would just keep retrying to get the new coordinator info when it receives the error code.
> > 
> > Offset manager : imports log manager, calls its append operation for commit and cleaning.
> > 
> > Log manager : do not import offset manager, expose the append function to offset manager and Kafka Api.

1. That was unintentional. I was just getting tired of accidentally typing ./sbt
2. The retention setting is mainly to deal with short-lived consumers (typically console consumers). We don't want those offsets sitting around indefinitely on a long-running broker process. A default of one day should be good. Another alternative is to set the retention to the maximum retention period of any topic. (Since offsets earlier than that would be invalid anyway).
3. I could not find this class anywhere. Can you confirm?
4.1/2 - Yes this is essentially what I proposed in the summary, but it is a major refactoring of KafkaApis which I will defer to a subsequent jira.

I didn't quite follow your last two points on Offset manager/log manager.


> On Feb. 19, 2014, 1:25 a.m., Guozhang Wang wrote:
> > core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala, lines 41-42
> > <https://reviews.apache.org/r/18022/diff/1/?file=483574#file483574line41>
> >
> >     Also, do we want just a broker Id or the broker object reference? And if it will be a broker object should we enforce it to be not optional, i.e., if it is not available return error code instead?

Broker id - since this will be serialized in the response and the id is really what the client needs.


> On Feb. 19, 2014, 1:25 a.m., Guozhang Wang wrote:
> > core/src/main/scala/kafka/api/OffsetCommitRequest.scala, line 129
> > <https://reviews.apache.org/r/18022/diff/1/?file=483575#file483575line129>
> >
> >     Is "details = " necessary?

no - a lot of this is due to suggestions in the ide. i.e., the code is clearer this way. e.g., if someone reads describe(true), what is true? It is clearer if the parameter name is explicitly named.


> On Feb. 19, 2014, 1:25 a.m., Guozhang Wang wrote:
> > core/src/main/scala/kafka/cluster/Partition.scala, line 219
> > <https://reviews.apache.org/r/18022/diff/1/?file=483580#file483580line219>
> >
> >     Why use foreach on Option[]?

It is an idiom. The code in braces will execute if the option is defined.


> On Feb. 19, 2014, 1:25 a.m., Guozhang Wang wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 157
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line157>
> >
> >     Should we check if it is loading to cache and possibly use the OffsetLoadInProgressException here? Otherwise this exception is not used anywhere.

Oh ok so you were referring to OffsetLoadInProgress. We don't really need the exception per se on the server side, but clients can use it. ErrorMapping needs an exception for the code.


> On Feb. 19, 2014, 1:25 a.m., Guozhang Wang wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 186
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line186>
> >
> >     If we can separate the ProduceRequest with the OffsetCommitRequest, probably we should put the append operation here. Otherwise we should change the comment: the offset has already been committed before this function, and this function is only used to update the cache.

Yes I will change the comment.


- Joel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review34803
-----------------------------------------------------------


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Guozhang Wang <gu...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review34803
-----------------------------------------------------------


Some general comments:

1. sbt: do we really want to delete this file in this patch?

2. offset retention minutes: 

I was not aware of the retention before. I think this is like the consumer heartbeat scheme: the retention minutes is defined globally on broker and the offset commit behavior is customizable on consumers, so we need to be very conservative on the server-side global retention parameter since this needs to be the MAX of all consumers. Or we can make this parameter as group-specific, but that may slightly complicate the logic on server side.

3. LoadingOffsetException seems not used anywhere. We should check this when getting offset from cache.

4. Currently the append-to-log operation, which is the synchronous-point of the commit-operation is not actually in the offset manager, the offset manager actually only takes care of the cache operations. That is a little awkward. Following Joel and Jun's suggestion I would propose the following:

KafkaApi : 

1. handle produce request and commit offset request separately. In handling commit offset request, hand it to offset manager. Keep handling produce request as it is and hand to log manager.

2. In handling leader and ISR, call makeLeader/Follower of log manager, then optionally call migrate offset manager of Offset Manager. Delaying the migration of offset manager is actually fine since the offset manager would check leader replica Id for fetching/committing anyways, and after makerLeader/Follower has finished subsequent fetch/commit would all be rejected since leader replica Id has changed. The consumer would just keep retrying to get the new coordinator info when it receives the error code.

Offset manager : imports log manager, calls its append operation for commit and cleaning.

Log manager : do not import offset manager, expose the append function to offset manager and Kafka Api.


core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
<https://reviews.apache.org/r/18022/#comment65145>

    Also, do we want just a broker Id or the broker object reference? And if it will be a broker object should we enforce it to be not optional, i.e., if it is not available return error code instead?



core/src/main/scala/kafka/api/OffsetCommitRequest.scala
<https://reviews.apache.org/r/18022/#comment65157>

    correlationId = correlationId?



core/src/main/scala/kafka/api/OffsetCommitRequest.scala
<https://reviews.apache.org/r/18022/#comment65155>

    Is "details = " necessary?



core/src/main/scala/kafka/cluster/Partition.scala
<https://reviews.apache.org/r/18022/#comment65164>

    Why use foreach on Option[]?



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment65168>

    How about using "case (topicAndPartition, Id)" here?



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment65177>

    Should we check if it is loading to cache and possibly use the OffsetLoadInProgressException here? Otherwise this exception is not used anywhere.



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment65179>

    If we can separate the ProduceRequest with the OffsetCommitRequest, probably we should put the append operation here. Otherwise we should change the comment: the offset has already been committed before this function, and this function is only used to update the cache.



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment65180>

    Ditto as above: probably we should change the function name.


- Guozhang Wang


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Joel Koshy <jj...@gmail.com>.

> On March 13, 2014, 11:25 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/consumer/ConsoleConsumer.scala, line 132
> > <https://reviews.apache.org/r/18022/diff/2-3/?file=511547#file511547line132>
> >
> >     Should this default to true if offsets-storage is set to kafka?

Right now it is just a flag and does not take an argument although I can change that - but I would prefer not to default to true because after migration the consumer shouldn't need to explicitly set it to false.


- Joel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review37130
-----------------------------------------------------------


On March 13, 2014, 11:03 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated March 13, 2014, 11:03 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/admin/TopicCommand.scala dc9b092 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 882b6da 
>   core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/Topic.scala c1b9f65 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/ConsumerConnector.scala 13c3f77 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala fa7caa7 
>   core/src/main/scala/kafka/consumer/TopicCount.scala e332633 
>   core/src/main/scala/kafka/consumer/TopicFilter.scala 4f20823 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/controller/KafkaController.scala 4deff9d 
>   core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java c45c803 
>   core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala 58e83f6 
>   core/src/main/scala/kafka/log/FileMessageSet.scala e1f8b97 
>   core/src/main/scala/kafka/producer/ProducerConfig.scala 7947b18 
>   core/src/main/scala/kafka/server/KafkaApis.scala 215ac36 
>   core/src/main/scala/kafka/server/KafkaConfig.scala b871843 
>   core/src/main/scala/kafka/server/KafkaServer.scala feb2093 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala fb759d9 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/DumpLogSegments.scala 14f44d9 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 5e8c56d 
>   core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala eac9af2 
>   core/src/main/scala/kafka/utils/VerifiableProperties.scala b070bb4 
>   core/src/main/scala/kafka/utils/ZkUtils.scala a198628 
>   core/src/test/scala/other/kafka/TestOffsetManager.scala PRE-CREATION 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala dbe078c 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala cf2724b 
>   core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala 8fe7259 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   system_test/migration_tool_testsuite/migration_tool_test.py 2fecd19 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/mirror_maker_testsuite/mirror_maker_test.py fd18088 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/replication_testsuite/replica_basic_test.py 5d3d93e 
>   system_test/utils/kafka_system_test_utils.py 29ab2ba 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Neha Narkhede <ne...@gmail.com>.

> On March 13, 2014, 11:25 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/consumer/ConsoleConsumer.scala, line 132
> > <https://reviews.apache.org/r/18022/diff/2-3/?file=511547#file511547line132>
> >
> >     Should this default to true if offsets-storage is set to kafka?
> 
> Joel Koshy wrote:
>     Right now it is just a flag and does not take an argument although I can change that - but I would prefer not to default to true because after migration the consumer shouldn't need to explicitly set it to false.

Ya, that's fine too.


- Neha


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review37130
-----------------------------------------------------------


On March 13, 2014, 11:03 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated March 13, 2014, 11:03 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/admin/TopicCommand.scala dc9b092 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 882b6da 
>   core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/Topic.scala c1b9f65 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/ConsumerConnector.scala 13c3f77 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala fa7caa7 
>   core/src/main/scala/kafka/consumer/TopicCount.scala e332633 
>   core/src/main/scala/kafka/consumer/TopicFilter.scala 4f20823 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/controller/KafkaController.scala 4deff9d 
>   core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java c45c803 
>   core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala 58e83f6 
>   core/src/main/scala/kafka/log/FileMessageSet.scala e1f8b97 
>   core/src/main/scala/kafka/producer/ProducerConfig.scala 7947b18 
>   core/src/main/scala/kafka/server/KafkaApis.scala 215ac36 
>   core/src/main/scala/kafka/server/KafkaConfig.scala b871843 
>   core/src/main/scala/kafka/server/KafkaServer.scala feb2093 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala fb759d9 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/DumpLogSegments.scala 14f44d9 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 5e8c56d 
>   core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala eac9af2 
>   core/src/main/scala/kafka/utils/VerifiableProperties.scala b070bb4 
>   core/src/main/scala/kafka/utils/ZkUtils.scala a198628 
>   core/src/test/scala/other/kafka/TestOffsetManager.scala PRE-CREATION 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala dbe078c 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala cf2724b 
>   core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala 8fe7259 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   system_test/migration_tool_testsuite/migration_tool_test.py 2fecd19 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/mirror_maker_testsuite/mirror_maker_test.py fd18088 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/replication_testsuite/replica_basic_test.py 5d3d93e 
>   system_test/utils/kafka_system_test_utils.py 29ab2ba 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Neha Narkhede <ne...@gmail.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review37130
-----------------------------------------------------------

Ship it!


Minor review comment below


core/src/main/scala/kafka/consumer/ConsoleConsumer.scala
<https://reviews.apache.org/r/18022/#comment68478>

    Should this default to true if offsets-storage is set to kafka?


- Neha Narkhede


On March 13, 2014, 11:03 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated March 13, 2014, 11:03 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/admin/TopicCommand.scala dc9b092 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 882b6da 
>   core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/Topic.scala c1b9f65 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/ConsumerConnector.scala 13c3f77 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala fa7caa7 
>   core/src/main/scala/kafka/consumer/TopicCount.scala e332633 
>   core/src/main/scala/kafka/consumer/TopicFilter.scala 4f20823 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/controller/KafkaController.scala 4deff9d 
>   core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java c45c803 
>   core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala 58e83f6 
>   core/src/main/scala/kafka/log/FileMessageSet.scala e1f8b97 
>   core/src/main/scala/kafka/producer/ProducerConfig.scala 7947b18 
>   core/src/main/scala/kafka/server/KafkaApis.scala 215ac36 
>   core/src/main/scala/kafka/server/KafkaConfig.scala b871843 
>   core/src/main/scala/kafka/server/KafkaServer.scala feb2093 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala fb759d9 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/DumpLogSegments.scala 14f44d9 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 5e8c56d 
>   core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala eac9af2 
>   core/src/main/scala/kafka/utils/VerifiableProperties.scala b070bb4 
>   core/src/main/scala/kafka/utils/ZkUtils.scala a198628 
>   core/src/test/scala/other/kafka/TestOffsetManager.scala PRE-CREATION 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala dbe078c 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala cf2724b 
>   core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala 8fe7259 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   system_test/migration_tool_testsuite/migration_tool_test.py 2fecd19 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/mirror_maker_testsuite/mirror_maker_test.py fd18088 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/replication_testsuite/replica_basic_test.py 5d3d93e 
>   system_test/utils/kafka_system_test_utils.py 29ab2ba 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Joel Koshy <jj...@gmail.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/
-----------------------------------------------------------

(Updated March 13, 2014, 11:03 p.m.)


Review request for kafka.


Changes
-------

Key changes in this revision:
- I moved the dual-commit (offsets.backup) concept to the consumer. The
  reason for this is that if a consumer that consumes several partitions
  migrates to Kafka-based offset storage then the initial lookup in Kafka
  can take a long time (since the broker will have to look up those
  partitions serially from ZooKeeper). If we have several consumers doing
  the same thing it could cause a DoS on the socket server.  So consumers
  will have to specify offsets.storage=kafka and set
  dual.commit.enabled=true to start migrating. During this phase, commits
  will go to both ZooKeeper and Kafka. For offset fetches, the consumer will
  look up ZooKeeper and query Kafka and pick the latest (greater) offset.
  After all instances in the group have moved over, dual.commit.enabled can
  be set to false.
- Added a kafka/zookeeper commit meter to help operations determine if
  consumers have moved over to Kafka for offsets storage.
- I also changed the offset load process to avoid an unnecessary copy for
  faster load times.


Bugs: KAFKA-1012
    https://issues.apache.org/jira/browse/KAFKA-1012


Repository: kafka


Description
-------

I picked up most of Tejas' patch and made various edits for review here as I
would like this to be completed and closed.

Here is a link to the original implementation wiki:
https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management

A lot of it is the same in this revision of the patch, but there is a bunch
of refactoring. This patch does not use an "embedded producer" in the
consumer. i.e., the consumer issues offset commit/fetch requests directly to
the broker. Also, I decided against doing any kind of request forwarding and
added a "ConsumerMetadataRequest" that will be used to determine the offset
coordinator (and subsequently group coordinator that may be useful for the
client rewrite - see
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
Also, there were some questions on how to support multiple offset manager
implementations cleanly. After thinking about it I think it makes the code
simpler and clearer if we just have one good offset storage mechanism (i.e.,
Kafka-based). Consumers that want to store offsets elsewhere can do so on
their own. (However, if we do want to do this somewhat cleanly, see the
discussion on separation of APIs below.)

Here is a quick recap of how offset management within Kafka works:
- A special __offsets topic holds consumer offsets.
- The consumer group serves as the partition key for offsets committed to
  the __offsets topic. i.e., all offsets for all topics that a group
  consumes will be in a single partition of the offsets topic.
- The "group-topic-partition" is the actual (stored) key in each message of
  the offsets topic.  This facilitates de-duplication (and thus removal) of
  older offsets.
- The offset manager also contains an in-memory cache of offsets so it can
  serve offset fetch requests quickly.
- Think of commits as a little more than a produce request. If and only if
  the commit is appended to the __offsets log as a regular produce request
  we update the offsets cache. So the semantics are identical to a produce
  request.  Offset fetches return whatever is in the cache. If it is absent,
  and offsets have not yet been loaded from the logs into the cache (on
  becoming a leader), then we return an "OffsetsLoading" error code.

(Tejas' wiki has pretty good diagrams that describe the above.)

Some more details:

- Atomicity per-commit: One drawback of the Zookeeper-based offset commits
  is that we when we commit multiple offsets (since we don't use
  multi-write) we have to write offsets serially so it is not atomic.  In
  this implementation I went with Jun's suggestion on using a compressed
  message set. This ensures that we will disallow partial commits of a bulk
  commit. I have hard-coded this to GZIP but maybe it is better to just
  expose a config. Another option is to introduce an identity compression
  codec.
- The main corner cases to consider are when there is leader movement due to
  broker failures and simultaneous offset commits/fetches. Offset fetches
  would only occur if there are consumer-side rebalances or shutdowns. The
  guarantees we want to provide are: (i) successfully acknowledged offset
  commits should be returned on the next offset fetch - i.e., should not be
  lost (ii) offset fetches should never return a stale offset.
  - On becoming a follower of an offsets topic partition:
    - Partition.makeFollower clears the offset cache of entries belonging to
      this partition of __offsets.
    - Any subsequent offset fetch request will find out that the partition
      is no longer a leader and fail. There is one problem in the existing
      patch which I will highlight in the RB along with a suggested fix.
    - Likewise, any subsequent offset commit request will fail (since the
      underlying producer request will fail). It is okay if the underlying
      producer request succeeds and the broker becomes a follower for that
      partition just before the offset cache is updated (since the broker
      will not serve any OffsetFetchRequests for that partition until it
      becomes a leader again).
  - On becoming a leader of an offsets topic partition:
    - Partition.makeLeader: will load the offsets from the log
      (asynchronously). While this is in progress, the broker rejects offset
      fetches to this partition. Offset commits may continue to arrive -
      i.e., will be appended to the log and then written to the cache. The
      load loop might actually overwrite it with an earlier offset from the
      log but that is okay - since it will eventually reach the more recent
      update in the log and load that into the cache.

Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
- The broker config should set offsets.backup.enabled=true
- Upgrade the brokers to the latest jar. (Consumers still commit
  directly to ZooKeeper).
- Start migrating the consumers over.
- Consumers will now start sending offset commits to the broker. Since the
  backup setting is enabled, offsets will also be committed to ZooKeeper.
  This is necessary when migrating consumers.
- After _all_ consumers have moved over you can turn off the backup.

I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
as a note to myself and others reviewing).

Questions/comments for discussion
- Should we explicitly disallow changes to the number of offset topic partitions?
  This is necessary (or at least prompt with a warning) since changing the number
  of partitions would affect the partitioning strategy.
- Should we remove per-partition error codes for offset commits and use just
  a global error code for the entire request? I'm using compressed message
  sets for commits.  i.e., the log append for a given commit will either
  fail entirely or succeed entirely. The OffsetCommitResponse contains
  per-partition error codes. So if the log append fails for any reason the
  same error code would apply for all partitions. i.e., it is sufficient to
  have a global error code. I think we currently have per-partition error
  codes due to the fact that offset commit requests can include metadata for
  each offset. The per-partition error code is set to MetadataTooLarge if
  the metadata entry exceeds the MaxMetadataLength. However, in this case I
  would prefer to just fail the entire request as opposed to doing partial
  commits (as I am in the current patch). Anyone have thoughts on this?
- Error codes: right now I'm using existing error codes (with the exception
  of OffsetsLoading). It may be better to return more specific error codes
  but I'm not sure if it matters - since the client-side implementation
  needs to check for _any_ error and if any error exists (other than
  MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
  i.e., the client should not really care about the actual error. If people
  have any strong preference on this let me know.
- Separation of APIs: Right now, the offset manager, replica manager are
  intertwined which is less than ideal. It is okay if offset manager depends
  on replica manager but not the other way around. Ideally, I would like to
  have KafkaApis hand off offset commit/fetch requests to the offset manager
  which then handles it. However, the inter-dependence comes about due to
  the need to clear out the offset cache on becoming a follower and the need
  to load offsets on becoming a leader. I think we can improve the
  separation as follows:
  - Don't optimistically load offsets/clear offsets on a leader/follower
    transition. Instead, load offsets only when an offset fetch request
    arrives for a partition that had not been loaded yet.
  - The OffsetManager will need to maintain a Map[partition ->
    lastKnownLeaderEpoch] to determine whether to load offsets or not.
  - The above will eliminate the reference to OffsetManager from
    ReplicaManager. KafkaApis still needs to reference the OffsetManager and
    will need to create the offset commit message to append to the __offsets
    log.
  - We can actually avoid the need for KafkaApis to know about offset commit
    messsages as well: in order to do that, we will need to create a
    "DurableLog" layer on top of LogManager and move all the purgatory stuff
    in there. The LogManager supports appends/reads from the local log, but
    does not know anything about the replicas. Instead, we can have a
    DurableLog layer that depends on ReplicaManager and LogManager and
    contains the Producer/Fetch-Request purgatories. So OffsetManager will
    need to depend on this DurableLog component. So KafkaApis can just hand
    off ProducerRequests, FetchRequests to the DurableLog layer directly. It
    will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
    which will then hand it off to the DurableLog layer.
  - Is the above worth it? I'm not sure it is, especially if we are sticking
    to only one offset management implementation.


Diffs (updated)
-----

  core/src/main/scala/kafka/admin/TopicCommand.scala dc9b092 
  core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
  core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
  core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
  core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
  core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
  core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
  core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
  core/src/main/scala/kafka/cluster/Partition.scala 882b6da 
  core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala PRE-CREATION 
  core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
  core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala PRE-CREATION 
  core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
  core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala PRE-CREATION 
  core/src/main/scala/kafka/common/Topic.scala c1b9f65 
  core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
  core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
  core/src/main/scala/kafka/consumer/ConsumerConnector.scala 13c3f77 
  core/src/main/scala/kafka/consumer/SimpleConsumer.scala fa7caa7 
  core/src/main/scala/kafka/consumer/TopicCount.scala e332633 
  core/src/main/scala/kafka/consumer/TopicFilter.scala 4f20823 
  core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
  core/src/main/scala/kafka/controller/KafkaController.scala 4deff9d 
  core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala PRE-CREATION 
  core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
  core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
  core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java c45c803 
  core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala 58e83f6 
  core/src/main/scala/kafka/log/FileMessageSet.scala e1f8b97 
  core/src/main/scala/kafka/producer/ProducerConfig.scala 7947b18 
  core/src/main/scala/kafka/server/KafkaApis.scala 215ac36 
  core/src/main/scala/kafka/server/KafkaConfig.scala b871843 
  core/src/main/scala/kafka/server/KafkaServer.scala feb2093 
  core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
  core/src/main/scala/kafka/server/ReplicaManager.scala fb759d9 
  core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
  core/src/main/scala/kafka/tools/DumpLogSegments.scala 14f44d9 
  core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
  core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 5e8c56d 
  core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala eac9af2 
  core/src/main/scala/kafka/utils/VerifiableProperties.scala b070bb4 
  core/src/main/scala/kafka/utils/ZkUtils.scala a198628 
  core/src/test/scala/other/kafka/TestOffsetManager.scala PRE-CREATION 
  core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
  core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala dbe078c 
  core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
  core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala cf2724b 
  core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala 8fe7259 
  core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
  core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
  system_test/migration_tool_testsuite/migration_tool_test.py 2fecd19 
  system_test/mirror_maker/README da53c14 
  system_test/mirror_maker/bin/expected.out 0a1bbaf 
  system_test/mirror_maker/bin/run-test.sh e5e6c08 
  system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
  system_test/mirror_maker/config/mirror_producer.properties aa8be65 
  system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
  system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
  system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
  system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
  system_test/mirror_maker/config/server_target_1_1.properties d37955a 
  system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
  system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
  system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
  system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
  system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
  system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
  system_test/mirror_maker_testsuite/mirror_maker_test.py fd18088 
  system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
  system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
  system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
  system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
  system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
  system_test/replication_testsuite/replica_basic_test.py 5d3d93e 
  system_test/utils/kafka_system_test_utils.py 29ab2ba 
  system_test/utils/testcase_env.py bee8716 

Diff: https://reviews.apache.org/r/18022/diff/


Testing
-------


Thanks,

Joel Koshy


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Joel Koshy <jj...@gmail.com>.

> On March 6, 2014, 1:47 a.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala, line 307
> > <https://reviews.apache.org/r/18022/diff/1-2/?file=483587#file483587line307>
> >
> >     ZookeeperConsumerConnector is getting really large. I think it might be worth moving this logic out of here, but I realize that will be some refactoring work. Can we file a JIRA to improve this in the future?
> 
> Joel Koshy wrote:
>     Yeah it's not my favorite file in our code-base. Do you think it is worth filing a jira given that we are beginning a consumer rewrite?

After thinking about this I think it is worth doing anyway - will file a follow-up jira if I don't get to it before checking in (wanted to leave some time for others to review as well).

So we can add a ConsumerOffsetsChannel class that manages fetching and committing offsets. This will be useful in tools (and we can also get rid of the currently broken commitOffsets API in SimpleConsumer).


- Joel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review36305
-----------------------------------------------------------


On March 5, 2014, 11:53 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated March 5, 2014, 11:53 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   config/server.properties c9e923a 
>   core/src/main/scala/kafka/admin/TopicCommand.scala dc9b092 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 882b6da 
>   core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/Topic.scala c1b9f65 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/ConsumerConnector.scala 13c3f77 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 098d6e4 
>   core/src/main/scala/kafka/consumer/TopicCount.scala e332633 
>   core/src/main/scala/kafka/consumer/TopicFilter.scala 4f20823 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/controller/KafkaController.scala 4deff9d 
>   core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java c45c803 
>   core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala 58e83f6 
>   core/src/main/scala/kafka/producer/ProducerConfig.scala 7947b18 
>   core/src/main/scala/kafka/server/KafkaApis.scala 215ac36 
>   core/src/main/scala/kafka/server/KafkaConfig.scala b871843 
>   core/src/main/scala/kafka/server/KafkaServer.scala feb2093 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala fb759d9 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/DumpLogSegments.scala 14f44d9 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 5e8c56d 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala eac9af2 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/main/scala/kafka/utils/VerifiableProperties.scala b070bb4 
>   core/src/main/scala/kafka/utils/ZkUtils.scala a198628 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala dbe078c 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala cf2724b 
>   core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala 8fe7259 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/utils/kafka_system_test_utils.py 5d2b7df 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Joel Koshy <jj...@gmail.com>.

> On March 6, 2014, 1:47 a.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/client/ClientUtils.scala, line 120
> > <https://reviews.apache.org/r/18022/diff/1-2/?file=483579#file483579line120>
> >
> >     Now that you refactored the loop to use while(), is it necessary to use find? It seems like foreach would suffice

The find is needed to search for an available broker. i.e., if I use foreach I would need to "break" out of the foreach after connecting to some broker.


> On March 6, 2014, 1:47 a.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala, line 307
> > <https://reviews.apache.org/r/18022/diff/1-2/?file=483587#file483587line307>
> >
> >     ZookeeperConsumerConnector is getting really large. I think it might be worth moving this logic out of here, but I realize that will be some refactoring work. Can we file a JIRA to improve this in the future?

Yeah it's not my favorite file in our code-base. Do you think it is worth filing a jira given that we are beginning a consumer rewrite?


> On March 6, 2014, 1:47 a.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/server/KafkaServer.scala, line 226
> > <https://reviews.apache.org/r/18022/diff/1-2/?file=483592#file483592line226>
> >
> >     this is also not part of your patch right?

weird - this looks like an RB rendering bug. I looked at the raw patch and it does not have a change here although it is in the vicinity of another change.


> On March 6, 2014, 1:47 a.m., Neha Narkhede wrote:
> > config/server.properties, line 106
> > <https://reviews.apache.org/r/18022/diff/2/?file=511531#file511531line106>
> >
> >     Is this change required? We already default to delete.

I can revert that. I just wanted to be sure while testing.


> On March 6, 2014, 1:47 a.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/controller/KafkaController.scala, line 1119
> > <https://reviews.apache.org/r/18022/diff/2/?file=511554#file511554line1119>
> >
> >     is this supposed to be part of your patch?

No - but I found it while testing. It's an easy fix. The issue is that after running preferred replica election, the admin path is not deleted (even if successful) and subsequent attempts to run it will fail.


- Joel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review36305
-----------------------------------------------------------


On March 5, 2014, 11:53 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated March 5, 2014, 11:53 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   config/server.properties c9e923a 
>   core/src/main/scala/kafka/admin/TopicCommand.scala dc9b092 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 882b6da 
>   core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/Topic.scala c1b9f65 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/ConsumerConnector.scala 13c3f77 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 098d6e4 
>   core/src/main/scala/kafka/consumer/TopicCount.scala e332633 
>   core/src/main/scala/kafka/consumer/TopicFilter.scala 4f20823 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/controller/KafkaController.scala 4deff9d 
>   core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java c45c803 
>   core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala 58e83f6 
>   core/src/main/scala/kafka/producer/ProducerConfig.scala 7947b18 
>   core/src/main/scala/kafka/server/KafkaApis.scala 215ac36 
>   core/src/main/scala/kafka/server/KafkaConfig.scala b871843 
>   core/src/main/scala/kafka/server/KafkaServer.scala feb2093 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala fb759d9 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/DumpLogSegments.scala 14f44d9 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 5e8c56d 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala eac9af2 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/main/scala/kafka/utils/VerifiableProperties.scala b070bb4 
>   core/src/main/scala/kafka/utils/ZkUtils.scala a198628 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala dbe078c 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala cf2724b 
>   core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala 8fe7259 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/utils/kafka_system_test_utils.py 5d2b7df 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Neha Narkhede <ne...@gmail.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review36305
-----------------------------------------------------------

Ship it!


I think this patch is in reasonable shape and is pretty large at the same time. I actually think it may not be a bad idea to check it in as is and file JIRAs for further improvements and bug fixes. It is getting harder to review the patch given it's size.

Few minor review comments/questions below.


core/src/main/scala/kafka/client/ClientUtils.scala
<https://reviews.apache.org/r/18022/#comment67267>

    Now that you refactored the loop to use while(), is it necessary to use find? It seems like foreach would suffice



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment67282>

    ZookeeperConsumerConnector is getting really large. I think it might be worth moving this logic out of here, but I realize that will be some refactoring work. Can we file a JIRA to improve this in the future?



core/src/main/scala/kafka/server/KafkaServer.scala
<https://reviews.apache.org/r/18022/#comment67284>

    this is also not part of your patch right?



config/server.properties
<https://reviews.apache.org/r/18022/#comment67262>

    Is this change required? We already default to delete. 



core/src/main/scala/kafka/controller/KafkaController.scala
<https://reviews.apache.org/r/18022/#comment67283>

    is this supposed to be part of your patch?


- Neha Narkhede


On March 5, 2014, 11:53 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated March 5, 2014, 11:53 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   config/server.properties c9e923a 
>   core/src/main/scala/kafka/admin/TopicCommand.scala dc9b092 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 882b6da 
>   core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/Topic.scala c1b9f65 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/ConsumerConnector.scala 13c3f77 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 098d6e4 
>   core/src/main/scala/kafka/consumer/TopicCount.scala e332633 
>   core/src/main/scala/kafka/consumer/TopicFilter.scala 4f20823 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/controller/KafkaController.scala 4deff9d 
>   core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java c45c803 
>   core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala 58e83f6 
>   core/src/main/scala/kafka/producer/ProducerConfig.scala 7947b18 
>   core/src/main/scala/kafka/server/KafkaApis.scala 215ac36 
>   core/src/main/scala/kafka/server/KafkaConfig.scala b871843 
>   core/src/main/scala/kafka/server/KafkaServer.scala feb2093 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala fb759d9 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/DumpLogSegments.scala 14f44d9 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 5e8c56d 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala eac9af2 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/main/scala/kafka/utils/VerifiableProperties.scala b070bb4 
>   core/src/main/scala/kafka/utils/ZkUtils.scala a198628 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala dbe078c 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala cf2724b 
>   core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala 8fe7259 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/utils/kafka_system_test_utils.py 5d2b7df 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Joel Koshy <jj...@gmail.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/
-----------------------------------------------------------

(Updated March 5, 2014, 11:53 p.m.)


Review request for kafka.


Changes
-------

I believe I have addressed/incorporated all of the comments from the
previous review in this updated patch. Let me know if I have missed
anything. A couple of additional comments:

- I will file follow-up issues for these:
  - Fix commitOffsets in SimpleConsumer (not sure if SimpleConsumer needs a
    commit offsets though). It may be better to have a generic
    "OffsetsCommitter" to do commits/fetches.
  - LogCleaner does not work with compressed topics:
    - First, the iteration over compressed message sets is incorrect. The
      position is advanced by the size of the decompressed messages within
      each compressed message-set.
    - Second, it only writes out uncompressed messages. We should probably
      have a cleaner-config that specifies the compression codec to use for
      the compacted log.
  - I should probably clear offsets cache on StopReplica (say, during
    partition reassignment) but I have not looked carefully enough to be
    sure and it is non-critical.
  - Import/export zk offsets/updateoffsetsinzk needs to be updated -
    preferably with an uber-consumer-admin tool.
- Given that the log cleaner does not work with compressed topics I have set
  the default for the offsets topic to no-compression.
- Since this is a new feature, I'm making consumers commit to zookeeper by
  default.
- I used the new protocol schema utilities for versioning the on-disk
  messages of the offsets topic. Those classes should probably move to a
  more general place since it is useful for more than just protocol schemas.
- I disallowed changes to the partition count of offsets topic, but maybe we
  should expose a --force option. If anyone has thoughts on this, let me
  know. I also outlined  possible approaches to changing the partition count
  on the fly (see Neha's review).
- Here's what I ended up doing for offset commit/fetch error codes in the
  consumer connector (there was a shutdown issue in the previous patch):
  - If offset commit fails while rebalancing or start-up, then retry
    indefinitely (since we need availability during normal operation).
  - If offset commit fails during shutdown, then retry only up to max
    retries.
  - If offset fetch fails during shutdown, abort and fail the rebalance.
- In my tests, offset loads are taking longer than I expect - I'm
  investigating that separately.


Bugs: KAFKA-1012
    https://issues.apache.org/jira/browse/KAFKA-1012


Repository: kafka


Description
-------

I picked up most of Tejas' patch and made various edits for review here as I
would like this to be completed and closed.

Here is a link to the original implementation wiki:
https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management

A lot of it is the same in this revision of the patch, but there is a bunch
of refactoring. This patch does not use an "embedded producer" in the
consumer. i.e., the consumer issues offset commit/fetch requests directly to
the broker. Also, I decided against doing any kind of request forwarding and
added a "ConsumerMetadataRequest" that will be used to determine the offset
coordinator (and subsequently group coordinator that may be useful for the
client rewrite - see
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
Also, there were some questions on how to support multiple offset manager
implementations cleanly. After thinking about it I think it makes the code
simpler and clearer if we just have one good offset storage mechanism (i.e.,
Kafka-based). Consumers that want to store offsets elsewhere can do so on
their own. (However, if we do want to do this somewhat cleanly, see the
discussion on separation of APIs below.)

Here is a quick recap of how offset management within Kafka works:
- A special __offsets topic holds consumer offsets.
- The consumer group serves as the partition key for offsets committed to
  the __offsets topic. i.e., all offsets for all topics that a group
  consumes will be in a single partition of the offsets topic.
- The "group-topic-partition" is the actual (stored) key in each message of
  the offsets topic.  This facilitates de-duplication (and thus removal) of
  older offsets.
- The offset manager also contains an in-memory cache of offsets so it can
  serve offset fetch requests quickly.
- Think of commits as a little more than a produce request. If and only if
  the commit is appended to the __offsets log as a regular produce request
  we update the offsets cache. So the semantics are identical to a produce
  request.  Offset fetches return whatever is in the cache. If it is absent,
  and offsets have not yet been loaded from the logs into the cache (on
  becoming a leader), then we return an "OffsetsLoading" error code.

(Tejas' wiki has pretty good diagrams that describe the above.)

Some more details:

- Atomicity per-commit: One drawback of the Zookeeper-based offset commits
  is that we when we commit multiple offsets (since we don't use
  multi-write) we have to write offsets serially so it is not atomic.  In
  this implementation I went with Jun's suggestion on using a compressed
  message set. This ensures that we will disallow partial commits of a bulk
  commit. I have hard-coded this to GZIP but maybe it is better to just
  expose a config. Another option is to introduce an identity compression
  codec.
- The main corner cases to consider are when there is leader movement due to
  broker failures and simultaneous offset commits/fetches. Offset fetches
  would only occur if there are consumer-side rebalances or shutdowns. The
  guarantees we want to provide are: (i) successfully acknowledged offset
  commits should be returned on the next offset fetch - i.e., should not be
  lost (ii) offset fetches should never return a stale offset.
  - On becoming a follower of an offsets topic partition:
    - Partition.makeFollower clears the offset cache of entries belonging to
      this partition of __offsets.
    - Any subsequent offset fetch request will find out that the partition
      is no longer a leader and fail. There is one problem in the existing
      patch which I will highlight in the RB along with a suggested fix.
    - Likewise, any subsequent offset commit request will fail (since the
      underlying producer request will fail). It is okay if the underlying
      producer request succeeds and the broker becomes a follower for that
      partition just before the offset cache is updated (since the broker
      will not serve any OffsetFetchRequests for that partition until it
      becomes a leader again).
  - On becoming a leader of an offsets topic partition:
    - Partition.makeLeader: will load the offsets from the log
      (asynchronously). While this is in progress, the broker rejects offset
      fetches to this partition. Offset commits may continue to arrive -
      i.e., will be appended to the log and then written to the cache. The
      load loop might actually overwrite it with an earlier offset from the
      log but that is okay - since it will eventually reach the more recent
      update in the log and load that into the cache.

Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
- The broker config should set offsets.backup.enabled=true
- Upgrade the brokers to the latest jar. (Consumers still commit
  directly to ZooKeeper).
- Start migrating the consumers over.
- Consumers will now start sending offset commits to the broker. Since the
  backup setting is enabled, offsets will also be committed to ZooKeeper.
  This is necessary when migrating consumers.
- After _all_ consumers have moved over you can turn off the backup.

I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
as a note to myself and others reviewing).

Questions/comments for discussion
- Should we explicitly disallow changes to the number of offset topic partitions?
  This is necessary (or at least prompt with a warning) since changing the number
  of partitions would affect the partitioning strategy.
- Should we remove per-partition error codes for offset commits and use just
  a global error code for the entire request? I'm using compressed message
  sets for commits.  i.e., the log append for a given commit will either
  fail entirely or succeed entirely. The OffsetCommitResponse contains
  per-partition error codes. So if the log append fails for any reason the
  same error code would apply for all partitions. i.e., it is sufficient to
  have a global error code. I think we currently have per-partition error
  codes due to the fact that offset commit requests can include metadata for
  each offset. The per-partition error code is set to MetadataTooLarge if
  the metadata entry exceeds the MaxMetadataLength. However, in this case I
  would prefer to just fail the entire request as opposed to doing partial
  commits (as I am in the current patch). Anyone have thoughts on this?
- Error codes: right now I'm using existing error codes (with the exception
  of OffsetsLoading). It may be better to return more specific error codes
  but I'm not sure if it matters - since the client-side implementation
  needs to check for _any_ error and if any error exists (other than
  MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
  i.e., the client should not really care about the actual error. If people
  have any strong preference on this let me know.
- Separation of APIs: Right now, the offset manager, replica manager are
  intertwined which is less than ideal. It is okay if offset manager depends
  on replica manager but not the other way around. Ideally, I would like to
  have KafkaApis hand off offset commit/fetch requests to the offset manager
  which then handles it. However, the inter-dependence comes about due to
  the need to clear out the offset cache on becoming a follower and the need
  to load offsets on becoming a leader. I think we can improve the
  separation as follows:
  - Don't optimistically load offsets/clear offsets on a leader/follower
    transition. Instead, load offsets only when an offset fetch request
    arrives for a partition that had not been loaded yet.
  - The OffsetManager will need to maintain a Map[partition ->
    lastKnownLeaderEpoch] to determine whether to load offsets or not.
  - The above will eliminate the reference to OffsetManager from
    ReplicaManager. KafkaApis still needs to reference the OffsetManager and
    will need to create the offset commit message to append to the __offsets
    log.
  - We can actually avoid the need for KafkaApis to know about offset commit
    messsages as well: in order to do that, we will need to create a
    "DurableLog" layer on top of LogManager and move all the purgatory stuff
    in there. The LogManager supports appends/reads from the local log, but
    does not know anything about the replicas. Instead, we can have a
    DurableLog layer that depends on ReplicaManager and LogManager and
    contains the Producer/Fetch-Request purgatories. So OffsetManager will
    need to depend on this DurableLog component. So KafkaApis can just hand
    off ProducerRequests, FetchRequests to the DurableLog layer directly. It
    will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
    which will then hand it off to the DurableLog layer.
  - Is the above worth it? I'm not sure it is, especially if we are sticking
    to only one offset management implementation.


Diffs (updated)
-----

  config/server.properties c9e923a 
  core/src/main/scala/kafka/admin/TopicCommand.scala dc9b092 
  core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
  core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
  core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
  core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
  core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
  core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
  core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
  core/src/main/scala/kafka/cluster/Partition.scala 882b6da 
  core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala PRE-CREATION 
  core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
  core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala PRE-CREATION 
  core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
  core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala PRE-CREATION 
  core/src/main/scala/kafka/common/Topic.scala c1b9f65 
  core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
  core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
  core/src/main/scala/kafka/consumer/ConsumerConnector.scala 13c3f77 
  core/src/main/scala/kafka/consumer/SimpleConsumer.scala 098d6e4 
  core/src/main/scala/kafka/consumer/TopicCount.scala e332633 
  core/src/main/scala/kafka/consumer/TopicFilter.scala 4f20823 
  core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
  core/src/main/scala/kafka/controller/KafkaController.scala 4deff9d 
  core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala PRE-CREATION 
  core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
  core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
  core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java c45c803 
  core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala 58e83f6 
  core/src/main/scala/kafka/producer/ProducerConfig.scala 7947b18 
  core/src/main/scala/kafka/server/KafkaApis.scala 215ac36 
  core/src/main/scala/kafka/server/KafkaConfig.scala b871843 
  core/src/main/scala/kafka/server/KafkaServer.scala feb2093 
  core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
  core/src/main/scala/kafka/server/ReplicaManager.scala fb759d9 
  core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
  core/src/main/scala/kafka/tools/DumpLogSegments.scala 14f44d9 
  core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
  core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 5e8c56d 
  core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
  core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala eac9af2 
  core/src/main/scala/kafka/utils/Utils.scala a89b046 
  core/src/main/scala/kafka/utils/VerifiableProperties.scala b070bb4 
  core/src/main/scala/kafka/utils/ZkUtils.scala a198628 
  core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
  core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala dbe078c 
  core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
  core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala cf2724b 
  core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala 8fe7259 
  core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
  core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
  system_test/mirror_maker/README da53c14 
  system_test/mirror_maker/bin/expected.out 0a1bbaf 
  system_test/mirror_maker/bin/run-test.sh e5e6c08 
  system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
  system_test/mirror_maker/config/mirror_producer.properties aa8be65 
  system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
  system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
  system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
  system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
  system_test/mirror_maker/config/server_target_1_1.properties d37955a 
  system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
  system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
  system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
  system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
  system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
  system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
  system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
  system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
  system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
  system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
  system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
  system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
  system_test/utils/kafka_system_test_utils.py 5d2b7df 
  system_test/utils/testcase_env.py bee8716 

Diff: https://reviews.apache.org/r/18022/diff/


Testing
-------


Thanks,

Joel Koshy


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Joel Koshy <jj...@gmail.com>.

> On Feb. 18, 2014, 3:31 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 338
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line338>
> >
> >     This is used only for DEBUG. Waste of memory for higher logging levels. Why not allocate / populate this only is DEBUG (or below logging mode) is turned on ?

I don't think it matters. However, I can remove it and just use a counter.


- Joel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review34673
-----------------------------------------------------------


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Tejas Patil <te...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review34673
-----------------------------------------------------------



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64884>

    This is used only for DEBUG. Waste of memory for higher logging levels. Why not allocate / populate this only is DEBUG (or below logging mode) is turned on ?


- Tejas Patil


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Tejas Patil <te...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review34676
-----------------------------------------------------------


- Tejas Patil


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Joel Koshy <jj...@gmail.com>.

> On Feb. 17, 2014, 6:04 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/server/KafkaApis.scala, line 210
> > <https://reviews.apache.org/r/18022/diff/1/?file=483590#file483590line210>
> >
> >     For small batch of offset message(s), compression would increase the message size. For a setup, where large % of consumers read from few topic-partitions, this would be bad.

Hey Tejas! Thanks for taking the time to review this.

Most consumers would consume more than a couple partitions. If so, it actually compresses just fine - you can do a simple test by sending offset commit requests with various compression codecs. If you have only a few partitions then compression may increase the size a bit but the overall size is still small. It's not a big deal. Another alternative was to introduce an "identity codec" that does not compress but just adds the message-set envelope, but I don't think we need to bother with these over-optimizations.


> On Feb. 17, 2014, 6:04 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 76
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line76>
> >
> >     What was the reason behind choosing 10000 ? If there was any, it would be awesome to have it in comment.

I actually don't recollect a good reason for that. I will remove it.


> On Feb. 17, 2014, 6:04 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 77
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line77>
> >
> >     scheduled task needs a better name

Will do - not sure how I ended up with 0 there.


> On Feb. 17, 2014, 6:04 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 93
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line93>
> >
> >     Is there any naming convention across the codebase for naming the start time of some operation ?

Not that I'm aware of. Do you feel this is unclear?


> On Feb. 17, 2014, 6:04 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 104
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line104>
> >
> >     This is a corner case: 
> >     - The cleaner thread populates the offset entry to be cleaned (say K, V1)
> >     - Broker's request handler thread does an offset commit with (K, V2) 
> >     - Cleaner thread removes K from the offset cache.
> >     - (K, V2) is lost

I don't think we need to address this. i.e., the staleness threshold should be set to a reasonable period so that if an offset is older than a certain age then it means we can safely declare that consumer as retired so there won't be further offset commits.


> On Feb. 17, 2014, 6:04 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 123
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line123>
> >
> >     This is a corner case: 
> >     - The cleaner thread populates the tombstone message (K, null)
> >     - Broker receives a offset commit message and appends it to the log (K, V) 
> >     - Cleaner thread appends the tombstone message to the log (K, null)
> >     - (K, V) is lost

See above.


> On Feb. 17, 2014, 6:04 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 190
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line190>
> >
> >     A rollback would be really hard at this point after the message was written to logs.
> >     Shouldn't the consumer be notified about this ?

This was more for debugging. We may actually see this message while loading offsets if an actual offset commit comes through. The commit would update the cache, but the loading process would overwrite it (with a lower offset) but would eventually put the correct offset back in the cache.


> On Feb. 17, 2014, 6:04 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 425
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line425>
> >
> >     currently not used anywhere in the code

It is meant for use in the console consumer - specified through props.


> On Feb. 17, 2014, 6:04 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 313
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line313>
> >
> >     contains unsafe check-then-act

Can you elaborate on this?


- Joel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review34615
-----------------------------------------------------------


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Tejas Patil <te...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review34615
-----------------------------------------------------------


Although I am NOT knowledgeable enough to review Joel's code, did a partial walk-over. Would add more comments after putting more thought.


core/src/main/scala/kafka/server/KafkaApis.scala
<https://reviews.apache.org/r/18022/#comment64790>

    For small batch of offset message(s), compression would increase the message size. For a setup, where large % of consumers read from few topic-partitions, this would be bad.



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64793>

    What was the reason behind choosing 10000 ? If there was any, it would be awesome to have it in comment. 



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64794>

    scheduled task needs a better name



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64796>

    Is there any naming convention across the codebase for naming the start time of some operation ?



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64797>

    This is a corner case: 
    - The cleaner thread populates the offset entry to be cleaned (say K, V1)
    - Broker's request handler thread does an offset commit with (K, V2) 
    - Cleaner thread removes K from the offset cache.
    - (K, V2) is lost



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64798>

    This is a corner case: 
    - The cleaner thread populates the tombstone message (K, null)
    - Broker receives a offset commit message and appends it to the log (K, V) 
    - Cleaner thread appends the tombstone message to the log (K, null)
    - (K, V) is lost



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64799>

    A rollback would be really hard at this point after the message was written to logs.
    Shouldn't the consumer be notified about this ?



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64795>

    contains unsafe check-then-act



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64792>

    currently not used anywhere in the code


- Tejas Patil


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Joel Koshy <jj...@gmail.com>.

> On Feb. 18, 2014, 3:48 a.m., Tejas Patil wrote:
> > My views (which hardly matter !!) about the points in 'Questions/comments for discussion' section:
> > 
> > - Change in partition assignment would need sync of offsets across brokers and subsequent bootstrap. Would be better to address in a separate jira.
> > - +1 for single error code for all offsets. There must be versioning for OffsetCommitResponse as in future there might be need to have separate error codes.
> > - It would be good to have separate error codes so that clients are aware of what went wrong.
> > - Lazy offset loading:
> >   - "clear out the offset cache on becoming a follower" : The cleanup thread would eventually clean this stuff after the retention period. As per on-paper-math, having extra offsets won't eat up much space in the offsets cache. There must be a mechanism (Map[partition -> lastKnownLeaderEpoch] ??) to figure out that those offsets in the cache are stale and a bootstrap is needed for them. As a downside, the old offsets would sit for long in the old generation space in heap eating up few MBs worth of space.
> >   - How would offset manager populate Map[partition -> lastKnownLeaderEpoch] ? Its clear that it can't do that all by itself thus implying some coupling with other module.
> >   - DurableLog: I liked the concept. This might touch a lot of places in the codebase and would be better to address in a separate jira.

Tejas:- Change in partition assignment would need sync of offsets across brokers and subsequent bootstrap. Would be better to address in a separate jira

Can you elaborate on the above?

"clear out eh offset cache" - yes that is not strictly needed, but it is cheap. However, we do need to load offsets on a leader change (at least in the current approach which I prefer to doing a lazy load on offset fetch).

How would offset manager populate the Map[partition->epoch]. The offset manager still needs to have access to the replica manager. However, the approach I described removes the dependency of the ReplicaManager to OffsetManager (which is an undesirable dependency).


> On Feb. 18, 2014, 3:48 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala, line 92
> > <https://reviews.apache.org/r/18022/diff/1/?file=483587#file483587line92>
> >
> >     1. This could be made a 'val' instead of 'var'
> >     2. Are entries ever cleared off this pool ? I see that it will keep growing after rebalances.

It could be a val.

Yes you are right that the entries are not cleared. If a consumer regains ownership of a partition it could potentially avoid recommitting offsets that have not moved since it last owned it. The memory requirements are probably not a big concern. The main reason we had this was to reduce the write load on ZooKeeper. Since we are moving to a new implementation we can even remove this altogether although it is still a useful optimization.


> On Feb. 18, 2014, 3:48 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala, line 168
> > <https://reviews.apache.org/r/18022/diff/1/?file=483587#file483587line168>
> >
> >     There are two different names used possibly for the same thing: offset coordinator and offset manager. Might be confusing to someone new top this feature.

We could collapse the two - coordinator crept in because there is some overlap with the consumer rewrite (and its use of the term consumer coordinator). My thinking was "coordinator" means broker, and offset manager is a component inside the broker that actually manages offsets.


> On Feb. 18, 2014, 3:48 a.m., Tejas Patil wrote:
> > core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala, line 273
> > <https://reviews.apache.org/r/18022/diff/1/?file=483587#file483587line273>
> >
> >     I am wondering why is an infinite loop needed inside offset commit. A finite retry counter might help for getting few reattempts.

Yes I'm a bit undecided on whether we should retry until a successful commit or use a retry limit.


- Joel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review34677
-----------------------------------------------------------


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Tejas Patil <te...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review34677
-----------------------------------------------------------


My views (which hardly matter !!) about the points in 'Questions/comments for discussion' section:

- Change in partition assignment would need sync of offsets across brokers and subsequent bootstrap. Would be better to address in a separate jira.
- +1 for single error code for all offsets. There must be versioning for OffsetCommitResponse as in future there might be need to have separate error codes.
- It would be good to have separate error codes so that clients are aware of what went wrong.
- Lazy offset loading:
  - "clear out the offset cache on becoming a follower" : The cleanup thread would eventually clean this stuff after the retention period. As per on-paper-math, having extra offsets won't eat up much space in the offsets cache. There must be a mechanism (Map[partition -> lastKnownLeaderEpoch] ??) to figure out that those offsets in the cache are stale and a bootstrap is needed for them. As a downside, the old offsets would sit for long in the old generation space in heap eating up few MBs worth of space.
  - How would offset manager populate Map[partition -> lastKnownLeaderEpoch] ? Its clear that it can't do that all by itself thus implying some coupling with other module.
  - DurableLog: I liked the concept. This might touch a lot of places in the codebase and would be better to address in a separate jira.


core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment64895>

    1. This could be made a 'val' instead of 'var'
    2. Are entries ever cleared off this pool ? I see that it will keep growing after rebalances.



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment64897>

    There are two different names used possibly for the same thing: offset coordinator and offset manager. Might be confusing to someone new top this feature.



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment64902>

    I am wondering why is an infinite loop needed inside offset commit. A finite retry counter might help for getting few reattempts.


- Tejas Patil


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Joel Koshy <jj...@gmail.com>.

> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > Thanks for the patch. Some comments.
> > 
> > 1. KafkaApis: The way that we handle OffsetCommit request pollutes handleProducerRequest() and DelayedProduce. It would be great if we can isolate the logic better. I was thinking that we can do the following. First, only support ack=-1 when writing to the offset topic. This way, there is only one place where the response for a produce request to offset topic will be sent--DelayedProduce. Second, introduce an OffsetCommitDelayedProduce that extends DelayedProduce and overrides the respond() function. Then, when handling OffsetCommit request, we first create a ProduceRequest and pass it to handleProducerRequest() with a flag to indicate whether we should use DelayedProduce or OffsetCommitDelayedProduce.
> > 
> > 2. ConsumerOffsetChecker: Do we need to provide an option to read the consumed offset from ZK so that for those consumers that haven't migrated off ZK, we can still check its lag?
> > 
> > 3. OffsetMetadataAndError.scala should be renamed to OffsetAndMetadata.scala.
> > 
> > 4. TestOffsetManager: Could you add some comments on what this test does? Also, the file is missing the license header.
> > 
> > 5. offset_management_testsuite: The new suite seems to be just running a few consumers, bouncing them and validating the consumed data matching the produced. Instead of introducing a new test suite, could we just reuse the existing mirror maker test suite and maybe add a new test case there? If we do need this test suite, could we add some comments on what the test suite does?
> > 
> > 6. OffsetFetchRequest: On the server side, if no partition is specified in the request, we fetch the offset for all partitions in this consumer group. It's not clear where we use this feature on the client side. If this feature is indeed needed, could we document it in the comment?

For (1) - yeah I think we can mandate ack = -1. The main reason I thought we might want to leave it configurable is if a user wants a stronger guarantee. i.e., acks > 1. WRT the suggestion on OffsetCommitDelayedProduce - that was actually how I first implemented it, but ended up removing it (https://gist.github.com/anonymous/9082577). I just felt that the current code is a bit clearer than what I had previously. However, I can revisit this.

(2) yes that is a good point.

(3) Ok - although it does have a OffsetMetadataAndError class in it as well. OffsetAndMetadata is used for commits and OffsetMetadataAndError is used for responses to offset fetches. I felt it was weird to have an error code in the commit request even if we provide a default that the user does not need to specify.

(4) I'm not planning to check this in. I had it in there to do some stress testing.

(5) We could - I just wanted to isolate this test for now. I need to do simultaneous bounces of the broker and the consumers. I also wanted to have more consumer instances than we have in the mirror maker suite. Finally I didn't really need a target cluster - i.e., this test would run faster than an equivalent mirror-maker suite test.

(6) Yes I had originally intended this for use by tools (e.g., export offsets) but I think it is useful to have in general.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala, lines 41-42
> > <https://reviews.apache.org/r/18022/diff/1/?file=483574#file483574line41>
> >
> >     Do we really need two coordinators?

I kept these separate even though we are planning to use the offset coordinator as the consumer coordinator because I feel it is very unintuitive that the consumer coordinator decision is driven off the offsets topic. That said, I will make this expose only a single coordinator and if we ever want to use a separate consumer coordinator then we can revisit.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/client/ClientUtils.scala, lines 142-146
> > <https://reviews.apache.org/r/18022/diff/1/?file=483579#file483579line142>
> >
> >     Stream.continually seems to be an un-intuitive way to implement a while loop.

As I wrote - I actually think it is intuitive and convenient once you use it a couple of times. It also reduces the use of vars and forces you to validate all code paths that assign a value to that val. However, I'm not sure about its overheads so I'm planning to remove it.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/client/ClientUtils.scala, lines 200-203
> > <https://reviews.apache.org/r/18022/diff/1/?file=483579#file483579line200>
> >
> >     The check on channel.isConnected in line 201 is already done inside BlockingChannel.disconnect().


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/consumer/ConsumerConfig.scala, lines 44-45
> > <https://reviews.apache.org/r/18022/diff/1/?file=483585#file483585line44>
> >
> >     We can probably use a larger backoff and socket timeout. The intuition is that if the server is somehow slow in processing the request, resending the requests too quickly is just going to make the server worse. How about 20secs timeout and 500ms backoff?

Makes sense.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala, lines 279-280
> > <https://reviews.apache.org/r/18022/diff/1/?file=483587#file483587line279>
> >
> >     Shouldn't we specify the offset commit timestamp here?

We could. However, I'm setting the timestamp on receiving the request at the broker. Part of the reason for this was that an earlier version of this patch depended on a correct timestamp while loading offsets (which I don't depend on anymore). With client-side timestamps we are exposed to client-side errors and clock skews so I preferred having the broker set the same timestamp on all entries in a given OffsetCommitRequest.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala, lines 286-287
> > <https://reviews.apache.org/r/18022/diff/1/?file=483587#file483587line286>
> >
> >     We can probably rename connectOffsetManager() to sth like ensureOffsetManagerConnected() and get rid of the if test here.

Makes sense.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/KafkaConfig.scala, lines 255-256
> > <https://reviews.apache.org/r/18022/diff/1/?file=483591#file483591line255>
> >
> >     Could we add a comment to explain how offsetsBackupDisabledGroups will be used? Do we expect to upgrade all consumers in those consumer groups first and then add those groups to this list?

Yes - and it is intended mainly for heavy-hitters such as mirror makers. Will add a comment.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/KafkaConfig.scala, lines 266-267
> > <https://reviews.apache.org/r/18022/diff/1/?file=483591#file483591line266>
> >
> >     Do we really need to make this configurable? It seems that we should always use ack=-1.

Had a comment on this above in the overall summary.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, lines 79-85
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line79>
> >
> >     The registered mbean already includes the class name. Do we need to add the "OffsetManager" prefix in the sensor name?

Will remove.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, lines 352-353
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line352>
> >
> >     Would it be better to call this topic "__consumer_offsets"?

Yes - I think that is better.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/cluster/Partition.scala, lines 188-189
> > <https://reviews.apache.org/r/18022/diff/1/?file=483580#file483580line188>
> >
> >     These probably need to be done before we set leaderReplicaIdOpt. Otherwise, the OffsetFetchRequest may not realize that the cached offset is stale.

Yes it should be moved earlier. However, I don't think it will give a stale offset (since we clear out offsets on becoming a follower). It will not find an entry though which is wrong.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala, lines 297-298
> > <https://reviews.apache.org/r/18022/diff/1/?file=483587#file483587line297>
> >
> >     What about other types of errors? It seems that if the error code is not NoError, we should return false.

Yes this is incomplete. i.e., I have a TODO for this.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/KafkaApis.scala, lines 226-229
> > <https://reviews.apache.org/r/18022/diff/1/?file=483590#file483590line226>
> >
> >     Why can't the caller convert an OffsetCommitRequest to a ProduceRequest? Then we don't have to change the api for handleProducerRequest().

I need to pass in the original offset commit request since I need access to things such as the group-id (which is passed on to the OffsetManager). I also need the original offset commit request map since I may need to set an OffsetMetadataTooLarge error code on some partitions. (i.e., how to handle large metadata comes into play here - whether to reject the entire request or allow a partial commit, or just truncate the metadata to the max). My preference would actually be to fail the entire commit, and I think it would simplify some of this code.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/KafkaApis.scala, lines 723-727
> > <https://reviews.apache.org/r/18022/diff/1/?file=483590#file483590line723>
> >
> >     It would be better to use replicaManager.getLeaderReplicaIfLocal() to see if the leader is local, since it's the source of truth.

We could do that, although it is redundant (since that check already happens in the handleProducerOrCommitRequest). That said, I think I can in fact just delete this code and have all the work done by the handleProducerOrCommitRequest.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/KafkaApis.scala, lines 756-757
> > <https://reviews.apache.org/r/18022/diff/1/?file=483590#file483590line756>
> >
> >     Such check is already in offsetManager.fetchOffsets(). Do we need the check here?

We don't - and in fact I think the returned error code is inaccurate. It should be NotLeaderForPartitionCode. i.e., OffsetMetadataAndError.OffsetManagerNotLocal


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/KafkaConfig.scala, lines 254-261
> > <https://reviews.apache.org/r/18022/diff/1/?file=483591#file483591line254>
> >
> >     Could we add the comment for each new property?

Will do - actually I put the comments in OffsetManagerConfig.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, lines 172-174
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line172>
> >
> >     Is this correct? It could also be that the offset was never checkpointed.

If it was never checkpointed then I currently return this. It is odd, but that's a TODO for offset-management-specific error codes. Ideally it should be a new "NoOffset" error code.


> On Feb. 17, 2014, 7:16 p.m., Jun Rao wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, lines 401-402
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line401>
> >
> >     Perhaps we should throw an exception if version is not expected?

I'll need to think through this a little more. We definitely want to do some versioning of the messages - or we will be stuck with the current offset storage format indefinitely.


- Joel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review34549
-----------------------------------------------------------


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Jun Rao <ju...@gmail.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review34549
-----------------------------------------------------------


Thanks for the patch. Some comments.

1. KafkaApis: The way that we handle OffsetCommit request pollutes handleProducerRequest() and DelayedProduce. It would be great if we can isolate the logic better. I was thinking that we can do the following. First, only support ack=-1 when writing to the offset topic. This way, there is only one place where the response for a produce request to offset topic will be sent--DelayedProduce. Second, introduce an OffsetCommitDelayedProduce that extends DelayedProduce and overrides the respond() function. Then, when handling OffsetCommit request, we first create a ProduceRequest and pass it to handleProducerRequest() with a flag to indicate whether we should use DelayedProduce or OffsetCommitDelayedProduce.

2. ConsumerOffsetChecker: Do we need to provide an option to read the consumed offset from ZK so that for those consumers that haven't migrated off ZK, we can still check its lag?

3. OffsetMetadataAndError.scala should be renamed to OffsetAndMetadata.scala.

4. TestOffsetManager: Could you add some comments on what this test does? Also, the file is missing the license header.

5. offset_management_testsuite: The new suite seems to be just running a few consumers, bouncing them and validating the consumed data matching the produced. Instead of introducing a new test suite, could we just reuse the existing mirror maker test suite and maybe add a new test case there? If we do need this test suite, could we add some comments on what the test suite does?

6. OffsetFetchRequest: On the server side, if no partition is specified in the request, we fetch the offset for all partitions in this consumer group. It's not clear where we use this feature on the client side. If this feature is indeed needed, could we document it in the comment?


core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
<https://reviews.apache.org/r/18022/#comment64845>

    Do we really need two coordinators?



core/src/main/scala/kafka/client/ClientUtils.scala
<https://reviews.apache.org/r/18022/#comment64847>

    Stream.continually seems to be an un-intuitive way to implement a while loop.



core/src/main/scala/kafka/client/ClientUtils.scala
<https://reviews.apache.org/r/18022/#comment64846>

    The check on channel.isConnected in line 201 is already done inside BlockingChannel.disconnect().



core/src/main/scala/kafka/cluster/Partition.scala
<https://reviews.apache.org/r/18022/#comment64767>

    These probably need to be done before we set leaderReplicaIdOpt. Otherwise, the OffsetFetchRequest may not realize that the cached offset is stale.



core/src/main/scala/kafka/consumer/ConsoleConsumer.scala
<https://reviews.apache.org/r/18022/#comment64848>

    Unused imports.



core/src/main/scala/kafka/consumer/ConsumerConfig.scala
<https://reviews.apache.org/r/18022/#comment64849>

    We can probably use a larger backoff and socket timeout. The intuition is that if the server is somehow slow in processing the request, resending the requests too quickly is just going to make the server worse. How about 20secs timeout and 500ms backoff?



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment64861>

    Shouldn't we specify the offset commit timestamp here?



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment64716>

    Would it be clearer to just use a while loop?



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment64670>

    We can probably rename connectOffsetManager() to sth like ensureOffsetManagerConnected() and get rid of the if test here.



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment64717>

    It would be clearer to use "exists { case (topicAndPartition, errorCode) => ... }".



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment64718>

    What about other types of errors? It seems that if the error code is not NoError, we should return false.



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment64720>

    Same comment here. Would a while loop be easier to understand than Stream.continually?



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment64719>

    This can just be 
    offsetFetchResponse.requestInfo(topicAndPartition).offset



core/src/main/scala/kafka/server/KafkaApis.scala
<https://reviews.apache.org/r/18022/#comment64722>

    Why can't the caller convert an OffsetCommitRequest to a ProduceRequest? Then we don't have to change the api for handleProducerRequest().



core/src/main/scala/kafka/server/KafkaApis.scala
<https://reviews.apache.org/r/18022/#comment64843>

    It would be better to use replicaManager.getLeaderReplicaIfLocal() to see if the leader is local, since it's the source of truth.



core/src/main/scala/kafka/server/KafkaApis.scala
<https://reviews.apache.org/r/18022/#comment64844>

    Such check is already in offsetManager.fetchOffsets(). Do we need the check here?



core/src/main/scala/kafka/server/KafkaConfig.scala
<https://reviews.apache.org/r/18022/#comment64840>

    Could we add the comment for each new property?



core/src/main/scala/kafka/server/KafkaConfig.scala
<https://reviews.apache.org/r/18022/#comment64851>

    Could we add a comment to explain how offsetsBackupDisabledGroups will be used? Do we expect to upgrade all consumers in those consumer groups first and then add those groups to this list?



core/src/main/scala/kafka/server/KafkaConfig.scala
<https://reviews.apache.org/r/18022/#comment64841>

    Do we really need to make this configurable? It seems that we should always use ack=-1.



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64842>

    Do we need to have defaults here? This means that we have to maintain defaults both here and in KafkaConfig.



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64834>

    The registered mbean already includes the class name. Do we need to add the "OffsetManager" prefix in the sensor name?



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64835>

    It's easier to understand if we use filter { case(topicAndPartition, offsetAndMetadata) => ... }.



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64836>

    Could we change it to groupBy { case() => }?



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64768>

    Is this correct? It could also be that the offset was never checkpointed.



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64837>

    Would it be better to call this topic "__consumer_offsets"?



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64838>

    Perhaps we should throw an exception if version is not expected?



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment64839>

    We can write the string directly. Not sure why we need to convert it to bytes first.


- Jun Rao


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Joel Koshy <jj...@gmail.com>.

> On Feb. 28, 2014, 7:30 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/server/KafkaApis.scala, line 234
> > <https://reviews.apache.org/r/18022/diff/1/?file=483590#file483590line234>
> >
> >     unused

It is used further down.


> On Feb. 28, 2014, 7:30 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/server/KafkaApis.scala, line 235
> > <https://reviews.apache.org/r/18022/diff/1/?file=483590#file483590line235>
> >
> >     ditto

Also used further down.


> On Feb. 28, 2014, 7:30 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 149
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line149>
> >
> >     I wonder if it is safe to assume that the local broker's offsetsTopicNumPartitions matches the # of partitions for the offsets topic. 
> >     
> >     This assumes that every broker would have the same config value for offsetsTopicNumPartitions, something we don't check for today. 
> >     
> >     In any case, this brings up the question - is there any way that the user can safely increase the # of partitions for the offsets topic? i.e. If all consumers are stopped for some time, offsets topic is expanded and then consumers are started, would it cause offsets to be reported incorrectly?
> >     
> >     For some reason, if # of partitions need to change, can the user stop all consumers, delete the offsets topic and re-create it? If yes, then this still requires us to fetch the # of partitions for the offsets topic from zookeeper or through TopicMetadataRequest, no?
> >     
> >     If we decide to not let the # of partitions change through any of the above, then it will be good to explicitly prevent that from happening through the TopicCommand

Right - I put this under comments for discussion in the summary. My preference for now is to disallow changing number of partitions for the offsets topic and start with a large number for a deployment. If you want to change the number of partitions; at least with the current group->partition hashing scheme, we will need to first stop all consumers and then export offsets (that tool needs to be updated); change the partition count; and finally import offsets.

Another approach would be to maintain a history of changes to the offsets topic. (E.g., we currently have a config/changes path in zookeeper although we only store the topic in there.) So if we have a history of the number of partitions we can walk-backwards on that config-chain when looking up offsets: i.e., if we don't find the offset then try hashing to the previous number of partitions and so on. This "walking-back" would typically only be done once per consumer after increasing the partition count.


> On Feb. 28, 2014, 7:30 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, lines 172-174
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line172>
> >
> >     So this would happen in 2 cases -
> >     
> >     1. Leader -> Follower transition in the middle of an offset fetch
> >     2. Offset was never checkpointed
> >     
> >     Realize that there is a TODO for offset management specific error codes. I was wondering if NotLeaderForPartition is more appropriate than UnknownTopicPartition?

In this instance I think UnknownTopicPartition is correct. i.e., we attempt to look up a partition that the consumer probably never consumed. So it is "unknown" from the point of view of the offset manager. NotLeaderForPartition is returned when the consumer commits offsets to or fetches offsets from a broker that is not the leader for the offsets partition that the consumer hashes to.

This may be confusing to users which is why we may want to map these errors to offset management-specific error codes. OTOH, I don't think users need to know these details since they would just need to retry if there is any error.


> On Feb. 28, 2014, 7:30 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 190
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line190>
> >
> >     So are we explicitly disallowing rewinding consumption for group management? We don't have to address this in the current patch, but will be good to file a JIRA to fix this since the new consumer API actually wants to allow this.

This was temporary - for debugging. We should allow rewinding.


> On Feb. 28, 2014, 7:30 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 381
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line381>
> >
> >     There is a version for the key, but not value. Is this intentional?

You're right. The value should also have some versioning incorporated.


> On Feb. 28, 2014, 7:30 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala, line 121
> > <https://reviews.apache.org/r/18022/diff/1/?file=483595#file483595line121>
> >
> >     Is it worth exposing offsets.channel.backoff.ms and offsets.channel.socket.timeout.ms?
> >     
> >     Same for TestOffsetManager

That may be a good idea - or we can just set it to a very high value.


> On Feb. 28, 2014, 7:30 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/server/KafkaApis.scala, line 199
> > <https://reviews.apache.org/r/18022/diff/1/?file=483590#file483590line199>
> >
> >     I think it is easier to fail the request in this case instead of the partial commit. Might keep things simpler (like you said earlier).

I'm still not sure what the best approach for this would be. If we fail the request, then I suppose we would need to set the error code for the other partitions to "Unattempted" or equivalent.

In the updated patch (which I'm yet to submit) I'm sticking to the partial commit and we can discuss further on that review.


- Joel


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review35746
-----------------------------------------------------------


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Neha Narkhede <ne...@gmail.com>.

> On Feb. 28, 2014, 7:30 p.m., Neha Narkhede wrote:
> > core/src/main/scala/kafka/server/OffsetManager.scala, line 149
> > <https://reviews.apache.org/r/18022/diff/1/?file=483593#file483593line149>
> >
> >     I wonder if it is safe to assume that the local broker's offsetsTopicNumPartitions matches the # of partitions for the offsets topic. 
> >     
> >     This assumes that every broker would have the same config value for offsetsTopicNumPartitions, something we don't check for today. 
> >     
> >     In any case, this brings up the question - is there any way that the user can safely increase the # of partitions for the offsets topic? i.e. If all consumers are stopped for some time, offsets topic is expanded and then consumers are started, would it cause offsets to be reported incorrectly?
> >     
> >     For some reason, if # of partitions need to change, can the user stop all consumers, delete the offsets topic and re-create it? If yes, then this still requires us to fetch the # of partitions for the offsets topic from zookeeper or through TopicMetadataRequest, no?
> >     
> >     If we decide to not let the # of partitions change through any of the above, then it will be good to explicitly prevent that from happening through the TopicCommand
> 
> Joel Koshy wrote:
>     Right - I put this under comments for discussion in the summary. My preference for now is to disallow changing number of partitions for the offsets topic and start with a large number for a deployment. If you want to change the number of partitions; at least with the current group->partition hashing scheme, we will need to first stop all consumers and then export offsets (that tool needs to be updated); change the partition count; and finally import offsets.
>     
>     Another approach would be to maintain a history of changes to the offsets topic. (E.g., we currently have a config/changes path in zookeeper although we only store the topic in there.) So if we have a history of the number of partitions we can walk-backwards on that config-chain when looking up offsets: i.e., if we don't find the offset then try hashing to the previous number of partitions and so on. This "walking-back" would typically only be done once per consumer after increasing the partition count.

I think it is reasonable to assume that # of partitions will not change for the time being. However, going forward, it will be nice to have a JIRA to keep track of the ability to be able to change the # of partitions with minimum downtime. We can discuss various approaches there. The "walking back" idea you have is pretty interesting.


- Neha


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review35746
-----------------------------------------------------------


On March 5, 2014, 11:53 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated March 5, 2014, 11:53 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   config/server.properties c9e923a 
>   core/src/main/scala/kafka/admin/TopicCommand.scala dc9b092 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 882b6da 
>   core/src/main/scala/kafka/common/ConsumerCoordinatorNotAvailableException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/NotCoordinatorForConsumerException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/common/OffsetsLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/Topic.scala c1b9f65 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/ConsumerConnector.scala 13c3f77 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 098d6e4 
>   core/src/main/scala/kafka/consumer/TopicCount.scala e332633 
>   core/src/main/scala/kafka/consumer/TopicFilter.scala 4f20823 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/controller/KafkaController.scala 4deff9d 
>   core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java c45c803 
>   core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala 58e83f6 
>   core/src/main/scala/kafka/producer/ProducerConfig.scala 7947b18 
>   core/src/main/scala/kafka/server/KafkaApis.scala 215ac36 
>   core/src/main/scala/kafka/server/KafkaConfig.scala b871843 
>   core/src/main/scala/kafka/server/KafkaServer.scala feb2093 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala fb759d9 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/DumpLogSegments.scala 14f44d9 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 5e8c56d 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala eac9af2 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/main/scala/kafka/utils/VerifiableProperties.scala b070bb4 
>   core/src/main/scala/kafka/utils/ZkUtils.scala a198628 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala dbe078c 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala cf2724b 
>   core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala 8fe7259 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/utils/kafka_system_test_utils.py 5d2b7df 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>


Re: Review Request 18022: KAFKA-1012: In-built offset management in Kafka

Posted by Neha Narkhede <ne...@gmail.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/18022/#review35746
-----------------------------------------------------------


Patch needs to be rebased but I reviewed it as is anyways. Will take another look after the rebase. Also haven't reviewed the system test suite yet.


core/src/main/scala/kafka/consumer/ConsoleConsumer.scala
<https://reviews.apache.org/r/18022/#comment66476>

    This file can be reverted.



core/src/main/scala/kafka/consumer/ConsumerConfig.scala
<https://reviews.apache.org/r/18022/#comment66477>

    Can we also include a comment explaining offsets.channel.socket.timeout.ms?



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment66568>

    It will be nice to move this logic to a separate method. That makes it easier to understand the rebalance loop.



core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
<https://reviews.apache.org/r/18022/#comment66569>

    Shouldn't we also handle UnknownTopicOrPartition since you return that if the offset cache was cleared in the middle of an OffsetFetch request.



core/src/main/scala/kafka/server/KafkaApis.scala
<https://reviews.apache.org/r/18022/#comment66552>

    unused now.



core/src/main/scala/kafka/server/KafkaApis.scala
<https://reviews.apache.org/r/18022/#comment66553>

    I think it is easier to fail the request in this case instead of the partial commit. Might keep things simpler (like you said earlier).



core/src/main/scala/kafka/server/KafkaApis.scala
<https://reviews.apache.org/r/18022/#comment66496>

    unused



core/src/main/scala/kafka/server/KafkaApis.scala
<https://reviews.apache.org/r/18022/#comment66497>

    ditto



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment66555>

    I wonder if it is safe to assume that the local broker's offsetsTopicNumPartitions matches the # of partitions for the offsets topic. 
    
    This assumes that every broker would have the same config value for offsetsTopicNumPartitions, something we don't check for today. 
    
    In any case, this brings up the question - is there any way that the user can safely increase the # of partitions for the offsets topic? i.e. If all consumers are stopped for some time, offsets topic is expanded and then consumers are started, would it cause offsets to be reported incorrectly?
    
    For some reason, if # of partitions need to change, can the user stop all consumers, delete the offsets topic and re-create it? If yes, then this still requires us to fetch the # of partitions for the offsets topic from zookeeper or through TopicMetadataRequest, no?
    
    If we decide to not let the # of partitions change through any of the above, then it will be good to explicitly prevent that from happening through the TopicCommand



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment66565>

    So this would happen in 2 cases -
    
    1. Leader -> Follower transition in the middle of an offset fetch
    2. Offset was never checkpointed
    
    Realize that there is a TODO for offset management specific error codes. I was wondering if NotLeaderForPartition is more appropriate than UnknownTopicPartition?



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment66566>

    So are we explicitly disallowing rewinding consumption for group management? We don't have to address this in the current patch, but will be good to file a JIRA to fix this since the new consumer API actually wants to allow this.



core/src/main/scala/kafka/server/OffsetManager.scala
<https://reviews.apache.org/r/18022/#comment66567>

    There is a version for the key, but not value. Is this intentional?



core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala
<https://reviews.apache.org/r/18022/#comment66475>

    Is it worth exposing offsets.channel.backoff.ms and offsets.channel.socket.timeout.ms?
    
    Same for TestOffsetManager


- Neha Narkhede


On Feb. 12, 2014, 7:50 p.m., Joel Koshy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/18022/
> -----------------------------------------------------------
> 
> (Updated Feb. 12, 2014, 7:50 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1012
>     https://issues.apache.org/jira/browse/KAFKA-1012
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> I picked up most of Tejas' patch and made various edits for review here as I
> would like this to be completed and closed.
> 
> Here is a link to the original implementation wiki:
> https://cwiki.apache.org/confluence/display/KAFKA/Inbuilt+Consumer+Offset+Management
> 
> A lot of it is the same in this revision of the patch, but there is a bunch
> of refactoring. This patch does not use an "embedded producer" in the
> consumer. i.e., the consumer issues offset commit/fetch requests directly to
> the broker. Also, I decided against doing any kind of request forwarding and
> added a "ConsumerMetadataRequest" that will be used to determine the offset
> coordinator (and subsequently group coordinator that may be useful for the
> client rewrite - see
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design).
> Also, there were some questions on how to support multiple offset manager
> implementations cleanly. After thinking about it I think it makes the code
> simpler and clearer if we just have one good offset storage mechanism (i.e.,
> Kafka-based). Consumers that want to store offsets elsewhere can do so on
> their own. (However, if we do want to do this somewhat cleanly, see the
> discussion on separation of APIs below.)
> 
> Here is a quick recap of how offset management within Kafka works:
> - A special __offsets topic holds consumer offsets.
> - The consumer group serves as the partition key for offsets committed to
>   the __offsets topic. i.e., all offsets for all topics that a group
>   consumes will be in a single partition of the offsets topic.
> - The "group-topic-partition" is the actual (stored) key in each message of
>   the offsets topic.  This facilitates de-duplication (and thus removal) of
>   older offsets.
> - The offset manager also contains an in-memory cache of offsets so it can
>   serve offset fetch requests quickly.
> - Think of commits as a little more than a produce request. If and only if
>   the commit is appended to the __offsets log as a regular produce request
>   we update the offsets cache. So the semantics are identical to a produce
>   request.  Offset fetches return whatever is in the cache. If it is absent,
>   and offsets have not yet been loaded from the logs into the cache (on
>   becoming a leader), then we return an "OffsetsLoading" error code.
> 
> (Tejas' wiki has pretty good diagrams that describe the above.)
> 
> Some more details:
> 
> - Atomicity per-commit: One drawback of the Zookeeper-based offset commits
>   is that we when we commit multiple offsets (since we don't use
>   multi-write) we have to write offsets serially so it is not atomic.  In
>   this implementation I went with Jun's suggestion on using a compressed
>   message set. This ensures that we will disallow partial commits of a bulk
>   commit. I have hard-coded this to GZIP but maybe it is better to just
>   expose a config. Another option is to introduce an identity compression
>   codec.
> - The main corner cases to consider are when there is leader movement due to
>   broker failures and simultaneous offset commits/fetches. Offset fetches
>   would only occur if there are consumer-side rebalances or shutdowns. The
>   guarantees we want to provide are: (i) successfully acknowledged offset
>   commits should be returned on the next offset fetch - i.e., should not be
>   lost (ii) offset fetches should never return a stale offset.
>   - On becoming a follower of an offsets topic partition:
>     - Partition.makeFollower clears the offset cache of entries belonging to
>       this partition of __offsets.
>     - Any subsequent offset fetch request will find out that the partition
>       is no longer a leader and fail. There is one problem in the existing
>       patch which I will highlight in the RB along with a suggested fix.
>     - Likewise, any subsequent offset commit request will fail (since the
>       underlying producer request will fail). It is okay if the underlying
>       producer request succeeds and the broker becomes a follower for that
>       partition just before the offset cache is updated (since the broker
>       will not serve any OffsetFetchRequests for that partition until it
>       becomes a leader again).
>   - On becoming a leader of an offsets topic partition:
>     - Partition.makeLeader: will load the offsets from the log
>       (asynchronously). While this is in progress, the broker rejects offset
>       fetches to this partition. Offset commits may continue to arrive -
>       i.e., will be appended to the log and then written to the cache. The
>       load loop might actually overwrite it with an earlier offset from the
>       log but that is okay - since it will eventually reach the more recent
>       update in the log and load that into the cache.
> 
> Migrating from ZooKeeper-based offset storage to Kafka-based offset storage:
> - The broker config should set offsets.backup.enabled=true
> - Upgrade the brokers to the latest jar. (Consumers still commit
>   directly to ZooKeeper).
> - Start migrating the consumers over.
> - Consumers will now start sending offset commits to the broker. Since the
>   backup setting is enabled, offsets will also be committed to ZooKeeper.
>   This is necessary when migrating consumers.
> - After _all_ consumers have moved over you can turn off the backup.
> 
> I have made a number of preliminary comments as TODOs in the RB myself (i.e.,
> as a note to myself and others reviewing).
> 
> Questions/comments for discussion
> - Should we explicitly disallow changes to the number of offset topic partitions?
>   This is necessary (or at least prompt with a warning) since changing the number
>   of partitions would affect the partitioning strategy.
> - Should we remove per-partition error codes for offset commits and use just
>   a global error code for the entire request? I'm using compressed message
>   sets for commits.  i.e., the log append for a given commit will either
>   fail entirely or succeed entirely. The OffsetCommitResponse contains
>   per-partition error codes. So if the log append fails for any reason the
>   same error code would apply for all partitions. i.e., it is sufficient to
>   have a global error code. I think we currently have per-partition error
>   codes due to the fact that offset commit requests can include metadata for
>   each offset. The per-partition error code is set to MetadataTooLarge if
>   the metadata entry exceeds the MaxMetadataLength. However, in this case I
>   would prefer to just fail the entire request as opposed to doing partial
>   commits (as I am in the current patch). Anyone have thoughts on this?
> - Error codes: right now I'm using existing error codes (with the exception
>   of OffsetsLoading). It may be better to return more specific error codes
>   but I'm not sure if it matters - since the client-side implementation
>   needs to check for _any_ error and if any error exists (other than
>   MetadataTooLarge) just retry the offset commit/fetch until it succeeds.
>   i.e., the client should not really care about the actual error. If people
>   have any strong preference on this let me know.
> - Separation of APIs: Right now, the offset manager, replica manager are
>   intertwined which is less than ideal. It is okay if offset manager depends
>   on replica manager but not the other way around. Ideally, I would like to
>   have KafkaApis hand off offset commit/fetch requests to the offset manager
>   which then handles it. However, the inter-dependence comes about due to
>   the need to clear out the offset cache on becoming a follower and the need
>   to load offsets on becoming a leader. I think we can improve the
>   separation as follows:
>   - Don't optimistically load offsets/clear offsets on a leader/follower
>     transition. Instead, load offsets only when an offset fetch request
>     arrives for a partition that had not been loaded yet.
>   - The OffsetManager will need to maintain a Map[partition ->
>     lastKnownLeaderEpoch] to determine whether to load offsets or not.
>   - The above will eliminate the reference to OffsetManager from
>     ReplicaManager. KafkaApis still needs to reference the OffsetManager and
>     will need to create the offset commit message to append to the __offsets
>     log.
>   - We can actually avoid the need for KafkaApis to know about offset commit
>     messsages as well: in order to do that, we will need to create a
>     "DurableLog" layer on top of LogManager and move all the purgatory stuff
>     in there. The LogManager supports appends/reads from the local log, but
>     does not know anything about the replicas. Instead, we can have a
>     DurableLog layer that depends on ReplicaManager and LogManager and
>     contains the Producer/Fetch-Request purgatories. So OffsetManager will
>     need to depend on this DurableLog component. So KafkaApis can just hand
>     off ProducerRequests, FetchRequests to the DurableLog layer directly. It
>     will hand off OffsetCommit/OffsetFetch requests to the OffsetManager
>     which will then hand it off to the DurableLog layer.
>   - Is the above worth it? I'm not sure it is, especially if we are sticking
>     to only one offset management implementation.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala PRE-CREATION 
>   core/src/main/scala/kafka/api/OffsetCommitRequest.scala 4d1fa5c 
>   core/src/main/scala/kafka/api/OffsetCommitResponse.scala 9e1795f 
>   core/src/main/scala/kafka/api/OffsetFetchRequest.scala 7036532 
>   core/src/main/scala/kafka/api/RequestKeys.scala c81214f 
>   core/src/main/scala/kafka/client/ClientUtils.scala 1d2f81b 
>   core/src/main/scala/kafka/cluster/Partition.scala 1087a2e 
>   core/src/main/scala/kafka/common/ErrorMapping.scala b0b5dce 
>   core/src/main/scala/kafka/common/OffsetLoadInProgressException.scala PRE-CREATION 
>   core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 59608a3 
>   core/src/main/scala/kafka/consumer/ConsoleConsumer.scala dc066c2 
>   core/src/main/scala/kafka/consumer/ConsumerConfig.scala e6875d6 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 6dae149 
>   core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala 703b2e2 
>   core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala 57b9d2a 
>   core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala 570bf31 
>   core/src/main/scala/kafka/server/KafkaApis.scala ae2df20 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 3c3aafc 
>   core/src/main/scala/kafka/server/KafkaServer.scala 5e34f95 
>   core/src/main/scala/kafka/server/OffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 21bba48 
>   core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 33d7c2c 
>   core/src/main/scala/kafka/tools/MirrorMaker.scala f0f871c 
>   core/src/main/scala/kafka/tools/TestOffsetManager.scala PRE-CREATION 
>   core/src/main/scala/kafka/utils/Utils.scala a89b046 
>   core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala 31534ca 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala eb274d1 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala 6a96d80 
>   core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala 1317b4c 
>   sbt 944ebf8 
>   system_test/mirror_maker/README da53c14 
>   system_test/mirror_maker/bin/expected.out 0a1bbaf 
>   system_test/mirror_maker/bin/run-test.sh e5e6c08 
>   system_test/mirror_maker/config/blacklisttest.consumer.properties ff12015 
>   system_test/mirror_maker/config/mirror_producer.properties aa8be65 
>   system_test/mirror_maker/config/server_source_1_1.properties 2f070a7 
>   system_test/mirror_maker/config/server_source_1_2.properties f9353e8 
>   system_test/mirror_maker/config/server_source_2_1.properties daa01ad 
>   system_test/mirror_maker/config/server_source_2_2.properties be6fdfc 
>   system_test/mirror_maker/config/server_target_1_1.properties d37955a 
>   system_test/mirror_maker/config/server_target_1_2.properties aa7546c 
>   system_test/mirror_maker/config/whitelisttest_1.consumer.properties ff12015 
>   system_test/mirror_maker/config/whitelisttest_2.consumer.properties f1a902b 
>   system_test/mirror_maker/config/zookeeper_source_1.properties f851796 
>   system_test/mirror_maker/config/zookeeper_source_2.properties d534d18 
>   system_test/mirror_maker/config/zookeeper_target.properties 55a7eb1 
>   system_test/offset_management_testsuite/cluster_config.json PRE-CREATION 
>   system_test/offset_management_testsuite/config/console_consumer.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/server.properties PRE-CREATION 
>   system_test/offset_management_testsuite/config/zookeeper.properties PRE-CREATION 
>   system_test/offset_management_testsuite/offset_management_test.py PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_1.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_2.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_3.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/kafka_server_4.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/config/zookeeper_0.properties PRE-CREATION 
>   system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json PRE-CREATION 
>   system_test/testcase_to_run.json 8252860 
>   system_test/utils/kafka_system_test_utils.py fb4a9c0 
>   system_test/utils/testcase_env.py bee8716 
> 
> Diff: https://reviews.apache.org/r/18022/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Joel Koshy
> 
>