[ 
https://issues.apache.org/jira/browse/KAFKA-554?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13591984#comment-13591984
 ] 

Jun Rao commented on KAFKA-554:
-------------------------------

Thanks for patch v1. Looks good overall. Some comments:

1. TopicCommand: Some options are only available for certain actions. Should we 
explain that in the description of those options?

2. AdminUtils:
2.1 comment: "is there is any" => "if there is any"
2.2 We have standardized non-singleton values in ZK to JSON. Should the values 
stored in the topic config path be JSON too?

3. TopicConfigManager:
3.1 add the missing > in the following comment.
 *   /brokers/topics/<topic_name/config
3.2 startup(): It seems there is no need to make sure TopicConfigChangesPath 
exists here since that's covered in initZk() in KafkaServer.startup().
3.3 processConfigChanges():
3.3.1 How about using "processing config change notifications" in the following 
logging to make it more specific.
      info("Processing %d change notifications...".format(notifications.size))
3.3.2 Reading the config from ZK can be done only if changeId > 
lastExecutedChange
3.3.3 Not sure why we don't delete the sequential node corresponding to 
lastChangeId from ZK.
3.3.4 It seems that sequential nodes under /brokers/config_changes are only 
deleted when there is a new config change. So, they are not always deleted 
after the configured expiration time.
3.4. ConfigChangeListener.handleChildChange(): chillins are obtained from 
zk.getChildren(). There is no guarantee that the list is sorted. So, you will 
need to sort it yourself since ordering is important here.

4. KafkaApis.handleOffsetCommitRequest(): The following statements 
    val responseInfo = offsetCommitRequest.requestInfo.map( t => {
      val (topicAndPartition, metadataAndError) = t
can be simplified to 
    val responseInfo = offsetCommitRequest.requestInfo.map( case 
(topicAndPartition, metadataAndError) => {

5. LogConfig: Can we define the two retentionPolicies "delete" and "dedupe" as 
contants and reuse them in LogConfig and KafkaConfig?

6. RequestKeys: ModifyTopicKey is not used.

7. ZkUtils: remove the following unused imports
import java.util.Properties
import java.io.{StringReader, StringWriter}

8. PrimitiveApiTest: Instead of commenting out the following lines, should we 
just remove them?
    // temporarily set request handler logger to a higher level
    //requestHandlerLogger.setLevel(Level.FATAL)

9. ReplicaFetchTest.logsMatch(): tandp is a bit confusing. Could we rename it 
to topicAndPart?

The patch needs to be rebased. Some of the changes are no longer necessary 
after the patch that standardizes the ZK paths/values.

                
> Move all per-topic configuration into ZK and add to the CreateTopicCommand
> --------------------------------------------------------------------------
>
>                 Key: KAFKA-554
>                 URL: https://issues.apache.org/jira/browse/KAFKA-554
>             Project: Kafka
>          Issue Type: New Feature
>            Reporter: Jay Kreps
>              Labels: project
>             Fix For: 0.8.1
>
>         Attachments: KAFKA-554-v1.patch
>
>
> We have a number of per-topic configurations that control message retention 
> and flush interval. Here is the list of properties I find in KafkaConfig that 
> appear to be per-topic:
>   topic.log.file.size
>   topic.log.roll.hours
>   topic.log.retention.hours
>   topic.log.retention.size
>   topic.flush.intervals.ms
> Currently we specify these in server.properties. This is not a good solution 
> as it requires a rolling bounce of the cluster to make a change, which just 
> doesn't scale to having hundreds of topics. Also the map encoded in a CSV 
> string is kind of hacky.
> We should move these into ZK in some kind of JSON blob that allows easily 
> adding new per-topic configs and we should remove these from 
> server.properties.
> It would be good to start with a wiki design and get consensus on that first.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to