@ -17,20 +17,23 @@
@@ -17,20 +17,23 @@
package kafka.coordinator
import java.nio.ByteBuffer
import kafka.api.ApiVersion
import kafka.cluster.Partition
import kafka.common. { OffsetAndMetadata , Topic }
import kafka.log.LogAppendInfo
import kafka.server. { KafkaConfig , ReplicaManager }
import kafka.log. { Log , LogAppendInfo }
import kafka.server. { FetchDataInfo , KafkaConfig , LogOffsetMetadata , ReplicaManager }
import kafka.utils. { KafkaScheduler , MockTime , TestUtils , ZkUtils }
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record. { MemoryRecords , Record , TimestampType }
import org.apache.kafka.common.record. { FileRecords , MemoryRecords , Record , TimestampType }
import org.apache.kafka.common.requests.OffsetFetchResponse
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.easymock. { Capture , EasyMock , IAnswer }
import org.junit. { After , Before , Test }
import org.junit.Assert._
import org.junit. { Before , Test }
import org.junit.Assert. { assertEquals , assertFalse , assertTrue }
import kafka.utils.TestUtils.fail
import scala.collection._
import JavaConverters._
@ -79,10 +82,153 @@ class GroupMetadataManagerTest {
@@ -79,10 +82,153 @@ class GroupMetadataManagerTest {
partition = EasyMock . niceMock ( classOf [ Partition ] )
}
@After
def tearDown ( ) {
EasyMock . reset ( replicaManager )
EasyMock . reset ( partition )
@Test
def testLoadOffsetsWithoutGroup ( ) {
val groupMetadataTopicPartition = new TopicPartition ( Topic . GroupMetadataTopicName , groupPartitionId )
val startOffset = 15L
val committedOffsets = Map (
new TopicPartition ( "foo" , 0 ) -> 23L ,
new TopicPartition ( "foo" , 1 ) -> 455L ,
new TopicPartition ( "bar" , 0 ) -> 8992L
)
val offsetCommitRecords = createCommittedOffsetRecords ( committedOffsets )
val records = MemoryRecords . withRecords ( startOffset , offsetCommitRecords : _ * )
expectGroupMetadataLoad ( groupMetadataTopicPartition , startOffset , records )
EasyMock . replay ( replicaManager )
groupMetadataManager . loadGroupsAndOffsets ( groupMetadataTopicPartition , _ => ( ) )
val group = groupMetadataManager . getGroup ( groupId ) . getOrElse ( fail ( "Group was not loaded into the cache" ) )
assertEquals ( groupId , group . groupId )
assertEquals ( Empty , group . currentState )
assertEquals ( committedOffsets . size , group . allOffsets . size )
committedOffsets . foreach { case ( topicPartition , offset ) =>
assertEquals ( Some ( offset ) , group . offset ( topicPartition ) . map ( _ . offset ) )
}
}
@Test
def testLoadOffsetsWithTombstones ( ) {
val groupMetadataTopicPartition = new TopicPartition ( Topic . GroupMetadataTopicName , groupPartitionId )
val startOffset = 15L
val tombstonePartition = new TopicPartition ( "foo" , 1 )
val committedOffsets = Map (
new TopicPartition ( "foo" , 0 ) -> 23L ,
tombstonePartition -> 455L ,
new TopicPartition ( "bar" , 0 ) -> 8992L
)
val offsetCommitRecords = createCommittedOffsetRecords ( committedOffsets )
val tombstone = Record . create ( GroupMetadataManager . offsetCommitKey ( groupId , tombstonePartition ) , null )
val records = MemoryRecords . withRecords ( startOffset , offsetCommitRecords ++ Seq ( tombstone ) : _ * )
expectGroupMetadataLoad ( groupMetadataTopicPartition , startOffset , records )
EasyMock . replay ( replicaManager )
groupMetadataManager . loadGroupsAndOffsets ( groupMetadataTopicPartition , _ => ( ) )
val group = groupMetadataManager . getGroup ( groupId ) . getOrElse ( fail ( "Group was not loaded into the cache" ) )
assertEquals ( groupId , group . groupId )
assertEquals ( Empty , group . currentState )
assertEquals ( committedOffsets . size - 1 , group . allOffsets . size )
committedOffsets . foreach { case ( topicPartition , offset ) =>
if ( topicPartition == tombstonePartition )
assertEquals ( None , group . offset ( topicPartition ) )
else
assertEquals ( Some ( offset ) , group . offset ( topicPartition ) . map ( _ . offset ) )
}
}
@Test
def testLoadOffsetsAndGroup ( ) {
val groupMetadataTopicPartition = new TopicPartition ( Topic . GroupMetadataTopicName , groupPartitionId )
val startOffset = 15L
val committedOffsets = Map (
new TopicPartition ( "foo" , 0 ) -> 23L ,
new TopicPartition ( "foo" , 1 ) -> 455L ,
new TopicPartition ( "bar" , 0 ) -> 8992L
)
val offsetCommitRecords = createCommittedOffsetRecords ( committedOffsets )
val memberId = "98098230493"
val groupMetadataRecord = buildStableGroupRecordWithMember ( memberId )
val records = MemoryRecords . withRecords ( startOffset , offsetCommitRecords ++ Seq ( groupMetadataRecord ) : _ * )
expectGroupMetadataLoad ( groupMetadataTopicPartition , startOffset , records )
EasyMock . replay ( replicaManager )
groupMetadataManager . loadGroupsAndOffsets ( groupMetadataTopicPartition , _ => ( ) )
val group = groupMetadataManager . getGroup ( groupId ) . getOrElse ( fail ( "Group was not loaded into the cache" ) )
assertEquals ( groupId , group . groupId )
assertEquals ( Stable , group . currentState )
assertEquals ( memberId , group . leaderId )
assertEquals ( Set ( memberId ) , group . allMembers )
assertEquals ( committedOffsets . size , group . allOffsets . size )
committedOffsets . foreach { case ( topicPartition , offset ) =>
assertEquals ( Some ( offset ) , group . offset ( topicPartition ) . map ( _ . offset ) )
}
}
@Test
def testLoadGroupWithTombstone ( ) {
val groupMetadataTopicPartition = new TopicPartition ( Topic . GroupMetadataTopicName , groupPartitionId )
val startOffset = 15L
val memberId = "98098230493"
val groupMetadataRecord = buildStableGroupRecordWithMember ( memberId )
val groupMetadataTombstone = Record . create ( GroupMetadataManager . groupMetadataKey ( groupId ) , null )
val records = MemoryRecords . withRecords ( startOffset , Seq ( groupMetadataRecord , groupMetadataTombstone ) : _ * )
expectGroupMetadataLoad ( groupMetadataTopicPartition , startOffset , records )
EasyMock . replay ( replicaManager )
groupMetadataManager . loadGroupsAndOffsets ( groupMetadataTopicPartition , _ => ( ) )
assertEquals ( None , groupMetadataManager . getGroup ( groupId ) )
}
@Test
def testOffsetWriteAfterGroupRemoved ( ) : Unit = {
// this test case checks the following scenario :
// 1. the group exists at some point in time , but is later removed ( because all members left )
// 2. a "simple" consumer ( i . e . not a consumer group ) then uses the same groupId to commit some offsets
val groupMetadataTopicPartition = new TopicPartition ( Topic . GroupMetadataTopicName , groupPartitionId )
val startOffset = 15L
val committedOffsets = Map (
new TopicPartition ( "foo" , 0 ) -> 23L ,
new TopicPartition ( "foo" , 1 ) -> 455L ,
new TopicPartition ( "bar" , 0 ) -> 8992L
)
val offsetCommitRecords = createCommittedOffsetRecords ( committedOffsets )
val memberId = "98098230493"
val groupMetadataRecord = buildStableGroupRecordWithMember ( memberId )
val groupMetadataTombstone = Record . create ( GroupMetadataManager . groupMetadataKey ( groupId ) , null )
val records = MemoryRecords . withRecords ( startOffset ,
Seq ( groupMetadataRecord , groupMetadataTombstone ) ++ offsetCommitRecords : _ * )
expectGroupMetadataLoad ( groupMetadataTopicPartition , startOffset , records )
EasyMock . replay ( replicaManager )
groupMetadataManager . loadGroupsAndOffsets ( groupMetadataTopicPartition , _ => ( ) )
val group = groupMetadataManager . getGroup ( groupId ) . getOrElse ( TestUtils . fail ( "Group was not loaded into the cache" ) )
assertEquals ( groupId , group . groupId )
assertEquals ( Empty , group . currentState )
assertEquals ( committedOffsets . size , group . allOffsets . size )
committedOffsets . foreach { case ( topicPartition , offset ) =>
assertEquals ( Some ( offset ) , group . offset ( topicPartition ) . map ( _ . offset ) )
}
}
@Test
@ -156,7 +302,7 @@ class GroupMetadataManagerTest {
@@ -156,7 +302,7 @@ class GroupMetadataManagerTest {
val member = new MemberMetadata ( memberId , groupId , clientId , clientHost , rebalanceTimeout , sessionTimeout ,
protocolType , List ( ( "protocol" , Array [ Byte ] ( ) ) ) )
member . awaitingJoinCallback = _ => ( )
group . add ( memberId , member )
group . add ( member )
group . transitionTo ( PreparingRebalance )
group . initNextGeneration ( )
@ -185,7 +331,7 @@ class GroupMetadataManagerTest {
@@ -185,7 +331,7 @@ class GroupMetadataManagerTest {
val member = new MemberMetadata ( memberId , groupId , clientId , clientHost , rebalanceTimeout , sessionTimeout ,
protocolType , List ( ( "protocol" , Array [ Byte ] ( ) ) ) )
member . awaitingJoinCallback = _ => ( )
group . add ( memberId , member )
group . add ( member )
group . transitionTo ( PreparingRebalance )
group . initNextGeneration ( )
@ -264,7 +410,7 @@ class GroupMetadataManagerTest {
@@ -264,7 +410,7 @@ class GroupMetadataManagerTest {
commitErrors = Some ( errors )
}
val delayedStoreOpt = groupMetadataManager . prepareStoreOffsets ( group , memberId , generationId , offsets , callback )
groupMetadataManager . prepareStoreOffsets ( group , memberId , generationId , offsets , callback )
assertFalse ( commitErrors . isEmpty )
val maybeError = commitErrors . get . get ( topicPartition )
@ -557,7 +703,7 @@ class GroupMetadataManagerTest {
@@ -557,7 +703,7 @@ class GroupMetadataManagerTest {
val member = new MemberMetadata ( memberId , groupId , clientId , clientHost , rebalanceTimeout , sessionTimeout ,
protocolType , List ( ( "protocol" , Array [ Byte ] ( ) ) ) )
member . awaitingJoinCallback = _ => ( )
group . add ( memberId , member )
group . add ( member )
group . transitionTo ( PreparingRebalance )
group . initNextGeneration ( )
@ -620,4 +766,47 @@ class GroupMetadataManagerTest {
@@ -620,4 +766,47 @@ class GroupMetadataManagerTest {
. andStubReturn ( Some ( Record . MAGIC_VALUE_V1 , TimestampType . CREATE_TIME ) )
}
private def buildStableGroupRecordWithMember ( memberId : String ) : Record = {
val group = new GroupMetadata ( groupId )
group . transitionTo ( PreparingRebalance )
val memberProtocols = List ( ( "roundrobin" , Array . emptyByteArray ) )
val member = new MemberMetadata ( memberId , groupId , "clientId" , "clientHost" , 30000 , 10000 , "consumer" , memberProtocols )
group . add ( member )
member . awaitingJoinCallback = _ => { }
group . initNextGeneration ( )
group . transitionTo ( Stable )
val groupMetadataKey = GroupMetadataManager . groupMetadataKey ( groupId )
val groupMetadataValue = GroupMetadataManager . groupMetadataValue ( group , Map ( memberId -> Array . empty [ Byte ] ) )
Record . create ( groupMetadataKey , groupMetadataValue )
}
private def expectGroupMetadataLoad ( groupMetadataTopicPartition : TopicPartition ,
startOffset : Long ,
records : MemoryRecords ) : Unit = {
val endOffset = startOffset + records . deepEntries . asScala . size
val logMock = EasyMock . mock ( classOf [ Log ] )
val fileRecordsMock = EasyMock . mock ( classOf [ FileRecords ] )
EasyMock . expect ( replicaManager . getLog ( groupMetadataTopicPartition ) ) . andStubReturn ( Some ( logMock ) )
EasyMock . expect ( logMock . logStartOffset ) . andStubReturn ( startOffset )
EasyMock . expect ( replicaManager . getHighWatermark ( groupMetadataTopicPartition ) ) . andStubReturn ( Some ( endOffset ) )
EasyMock . expect ( logMock . read ( EasyMock . eq ( startOffset ) , EasyMock . anyInt ( ) , EasyMock . eq ( None ) , EasyMock . eq ( true ) ) )
. andReturn ( FetchDataInfo ( LogOffsetMetadata ( startOffset ) , fileRecordsMock ) )
EasyMock . expect ( fileRecordsMock . readInto ( EasyMock . anyObject ( classOf [ ByteBuffer ] ) , EasyMock . anyInt ( ) ) )
. andReturn ( records . buffer )
EasyMock . replay ( logMock , fileRecordsMock )
}
private def createCommittedOffsetRecords ( committedOffsets : Map [ TopicPartition , Long ] ,
groupId : String = groupId ) : Seq [ Record ] = {
committedOffsets . map { case ( topicPartition , offset ) =>
val offsetAndMetadata = OffsetAndMetadata ( offset )
val offsetCommitKey = GroupMetadataManager . offsetCommitKey ( groupId , topicPartition )
val offsetCommitValue = GroupMetadataManager . offsetCommitValue ( offsetAndMetadata )
Record . create ( offsetCommitKey , offsetCommitValue )
} . toSeq
}
}