[ 
https://issues.apache.org/jira/browse/KAFKA-343?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Yang Ye updated KAFKA-343:
--------------------------

    Attachment: kafka_controller_v5.patch

50. RequestSendThread.run(): Not sure if we need to handle 
AsynchronousCloseException specially. If we hit any exception while sending the 
request, we should probably just log the exception at the info level and let it 
go. If the exception is due to a failed broker, the controller will eventually 
shutdown this thread.

Without catching this exception, there will a lot of erro messages in the log, 
no other difference



51. KafkaController:
51.1 getLeaderAndISRFromZookeeper():
51.1.1 Can we rename it to deliverLeaderAndISRFromZookeeper()?
OK

51.1.2 brokerToLeaderAndISRInfosMap.get(b).get can be 
brokerToLeaderAndISRInfosMap.(b)
changed all occurrences like this in the code


51.2 tryToBecomeController(): doesn't need to be synchronized since all callers 
are already synchronized.
OK

52. KafkaApis:
52.1 handleStopReplicaRequest: we should put in a TODO saying that this will be 
handled in kafka-330
I added *TODO* in the deleteReplica function of ReplicaManager


52.2 The check of replicaManager.hwCheckPointThreadStarted should be done in 
replicaManager.startHighWaterMarksCheckPointThread, not in KafkaApis.

OK


52.3 handleLeaderAndISRRequest(): Could you fix the indentation of the 
following line?
              (!replica.partition.leaderId().isDefined || 
replica.partition.leaderId().get != brokerId)){
52.4 handleDeletedTopics: During controller failover, we need to remove 
unneeded leaderAndISRPath that the previous controller didn't get a chance to 
remove. This can be done as part of kafka-330. Please add this note to 
kafka-330.

ok

53. ReplicaManager:
53.1 addLocalReplica(): The local replica should only be created if it doesn't 
exist.
ok

53.2 fix the indentation of the following line in makeFollower()
      ErrorMapping.UnknownCode
fixed

53.3 partitionExists(): Can we just use getPartition() instead?
sure


54.AbstractFetcherThread: It's useful to see the exception in the log message. 
This is ok since it's only in debug mode. 
reverted

So, just revert the change.

55. ConsumerFetcherManager, 
DefaultEventHandler,ProducerRequest,ProducerSendThread: It's actually better to 
put the long statement in 2 lines. So, revert this change.
OK
I've done that

56. LogManager.flushAllLogs(): It's actually better to put the long statement 
in 2 lines.
OK



57. Partition.updateISR(): Updating ISR in ZK should be done conditionally 
using zk version. Also, we should update ISR in ZK first before updating in 
memory. This can be done in a separate jira. Please file a new one.

Already fixed in latest patch



58. RequestPurgatory: name in constructor. Change it logPrefix. Ditto for 
ExpiredRequestReaper. Also there were lots of changes that seem to due to white 
spaces.''

old  changed 


59. LeaderExistsOrChangedListener: The 1st is too long. Break it into 2 lines.
OK


60. TestUtils: There are changes that seem to due to white spaces. 
It's indention problem with existing code
                
> revisit the become leader and become follower state change operations using 
> V3 design
> -------------------------------------------------------------------------------------
>
>                 Key: KAFKA-343
>                 URL: https://issues.apache.org/jira/browse/KAFKA-343
>             Project: Kafka
>          Issue Type: Sub-task
>          Components: core
>    Affects Versions: 0.8
>            Reporter: Jun Rao
>            Assignee: Yang Ye
>             Fix For: 0.8
>
>         Attachments: kafka_343.diff.2, kafka_343.diff.3, kafka_343.diff.4, 
> kafka_343.patch, kafka_controller_v5.patch
>
>
> We need to reimplement become leader/follower using the controller model 
> described in 
> https://cwiki.apache.org/confluence/display/KAFKA/kafka+Detailed+Replication+Design+V3

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to