@ -20,6 +20,7 @@ package kafka.server
@@ -20,6 +20,7 @@ package kafka.server
import java.lang. { Long => JLong }
import java.net.InetAddress
import java.util
import java.util.Collections
import kafka.api. { ApiVersion , KAFKA_0_10_2_IV0 }
import kafka.cluster.Replica
@ -40,6 +41,7 @@ import org.apache.kafka.common.network.ListenerName
@@ -40,6 +41,7 @@ import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol. { ApiKeys , Errors }
import org.apache.kafka.common.record.RecordBatch
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.requests.UpdateMetadataRequest. { Broker , EndPoint }
import org.apache.kafka.common.requests.WriteTxnMarkersRequest.TxnMarkerEntry
import org.apache.kafka.common.requests._
import org.apache.kafka.common.security.auth. { KafkaPrincipal , SecurityProtocol }
@ -106,6 +108,84 @@ class KafkaApisTest {
@@ -106,6 +108,84 @@ class KafkaApisTest {
)
}
@Test
def testOffsetCommitWithInvalidPartition ( ) : Unit = {
val topic = "topic"
setupBasicMetadataCache ( topic , numPartitions = 1 )
def checkInvalidPartition ( invalidPartitionId : Int ) : Unit = {
EasyMock . reset ( replicaManager , clientRequestQuotaManager , requestChannel )
val invalidTopicPartition = new TopicPartition ( topic , invalidPartitionId )
val partitionOffsetCommitData = new OffsetCommitRequest . PartitionData ( 15L , "" )
val ( offsetCommitRequest , request ) = buildRequest ( new OffsetCommitRequest . Builder ( "groupId" ,
Map ( invalidTopicPartition -> partitionOffsetCommitData ) . asJava ) )
val capturedResponse = expectThrottleCallbackAndInvoke ( )
EasyMock . replay ( replicaManager , clientRequestQuotaManager , requestChannel )
createKafkaApis ( ) . handleOffsetCommitRequest ( request )
val response = readResponse ( ApiKeys . OFFSET_COMMIT , offsetCommitRequest , capturedResponse )
. asInstanceOf [ OffsetCommitResponse ]
assertEquals ( Errors . UNKNOWN_TOPIC_OR_PARTITION , response . responseData ( ) . get ( invalidTopicPartition ) )
}
checkInvalidPartition ( - 1 )
checkInvalidPartition ( 1 ) // topic has only one partition
}
@Test
def testTxnOffsetCommitWithInvalidPartition ( ) : Unit = {
val topic = "topic"
setupBasicMetadataCache ( topic , numPartitions = 1 )
def checkInvalidPartition ( invalidPartitionId : Int ) : Unit = {
EasyMock . reset ( replicaManager , clientRequestQuotaManager , requestChannel )
val invalidTopicPartition = new TopicPartition ( topic , invalidPartitionId )
val partitionOffsetCommitData = new TxnOffsetCommitRequest . CommittedOffset ( 15L , "" )
val ( offsetCommitRequest , request ) = buildRequest ( new TxnOffsetCommitRequest . Builder ( "txnlId" , "groupId" ,
15L , 0. toShort , Map ( invalidTopicPartition -> partitionOffsetCommitData ) . asJava ) )
val capturedResponse = expectThrottleCallbackAndInvoke ( )
EasyMock . replay ( replicaManager , clientRequestQuotaManager , requestChannel )
createKafkaApis ( ) . handleTxnOffsetCommitRequest ( request )
val response = readResponse ( ApiKeys . TXN_OFFSET_COMMIT , offsetCommitRequest , capturedResponse )
. asInstanceOf [ TxnOffsetCommitResponse ]
assertEquals ( Errors . UNKNOWN_TOPIC_OR_PARTITION , response . errors ( ) . get ( invalidTopicPartition ) )
}
checkInvalidPartition ( - 1 )
checkInvalidPartition ( 1 ) // topic has only one partition
}
@Test
def testAddPartitionsToTxnWithInvalidPartition ( ) : Unit = {
val topic = "topic"
setupBasicMetadataCache ( topic , numPartitions = 1 )
def checkInvalidPartition ( invalidPartitionId : Int ) : Unit = {
EasyMock . reset ( replicaManager , clientRequestQuotaManager , requestChannel )
val invalidTopicPartition = new TopicPartition ( topic , invalidPartitionId )
val ( addPartitionsToTxnRequest , request ) = buildRequest ( new AddPartitionsToTxnRequest . Builder (
"txnlId" , 15L , 0. toShort , List ( invalidTopicPartition ) . asJava ) )
val capturedResponse = expectThrottleCallbackAndInvoke ( )
EasyMock . replay ( replicaManager , clientRequestQuotaManager , requestChannel )
createKafkaApis ( ) . handleAddPartitionToTxnRequest ( request )
val response = readResponse ( ApiKeys . ADD_PARTITIONS_TO_TXN , addPartitionsToTxnRequest , capturedResponse )
. asInstanceOf [ AddPartitionsToTxnResponse ]
assertEquals ( Errors . UNKNOWN_TOPIC_OR_PARTITION , response . errors ( ) . get ( invalidTopicPartition ) )
}
checkInvalidPartition ( - 1 )
checkInvalidPartition ( 1 ) // topic has only one partition
}
@Test ( expected = classOf [ UnsupportedVersionException ] )
def shouldThrowUnsupportedVersionExceptionOnHandleAddOffsetToTxnRequestWhenInterBrokerProtocolNotSupported ( ) : Unit = {
createKafkaApis ( KAFKA_0_10_2_IV0 ) . handleAddOffsetsToTxnRequest ( null )
@ -284,8 +364,6 @@ class KafkaApisTest {
@@ -284,8 +364,6 @@ class KafkaApisTest {
val timestamp : JLong = time . milliseconds ( )
val limitOffset = 15L
val capturedResponse = EasyMock . newCapture [ RequestChannel . Response ] ( )
val capturedThrottleCallback = EasyMock . newCapture [ Int => Unit ] ( )
val replica = EasyMock . mock ( classOf [ Replica ] )
val log = EasyMock . mock ( classOf [ Log ] )
EasyMock . expect ( replicaManager . getLeaderReplicaIfLocal ( tp ) ) . andReturn ( replica )
@ -295,8 +373,7 @@ class KafkaApisTest {
@@ -295,8 +373,7 @@ class KafkaApisTest {
EasyMock . expect ( replica . lastStableOffset ) . andReturn ( LogOffsetMetadata ( messageOffset = limitOffset ) )
EasyMock . expect ( replicaManager . getLog ( tp ) ) . andReturn ( Some ( log ) )
EasyMock . expect ( log . fetchOffsetsByTimestamp ( timestamp ) ) . andReturn ( Some ( TimestampOffset ( timestamp = timestamp , offset = limitOffset ) ) )
expectThrottleCallbackAndInvoke ( capturedThrottleCallback )
EasyMock . expect ( requestChannel . sendResponse ( EasyMock . capture ( capturedResponse ) ) )
val capturedResponse = expectThrottleCallbackAndInvoke ( )
EasyMock . replay ( replicaManager , clientRequestQuotaManager , requestChannel , replica , log )
val builder = ListOffsetRequest . Builder . forConsumer ( true , isolationLevel )
@ -327,8 +404,6 @@ class KafkaApisTest {
@@ -327,8 +404,6 @@ class KafkaApisTest {
val tp = new TopicPartition ( "foo" , 0 )
val limitOffset = 15L
val capturedResponse = EasyMock . newCapture [ RequestChannel . Response ] ( )
val capturedThrottleCallback = EasyMock . newCapture [ Int => Unit ] ( )
val replica = EasyMock . mock ( classOf [ Replica ] )
val log = EasyMock . mock ( classOf [ Log ] )
EasyMock . expect ( replicaManager . getLeaderReplicaIfLocal ( tp ) ) . andReturn ( replica )
@ -339,8 +414,7 @@ class KafkaApisTest {
@@ -339,8 +414,7 @@ class KafkaApisTest {
EasyMock . expect ( replicaManager . getLog ( tp ) ) . andReturn ( Some ( log ) )
EasyMock . expect ( log . fetchOffsetsByTimestamp ( ListOffsetRequest . EARLIEST_TIMESTAMP ) )
. andReturn ( Some ( TimestampOffset ( timestamp = ListOffsetResponse . UNKNOWN_TIMESTAMP , offset = limitOffset ) ) )
expectThrottleCallbackAndInvoke ( capturedThrottleCallback )
EasyMock . expect ( requestChannel . sendResponse ( EasyMock . capture ( capturedResponse ) ) )
val capturedResponse = expectThrottleCallbackAndInvoke ( )
EasyMock . replay ( replicaManager , clientRequestQuotaManager , requestChannel , replica , log )
val builder = ListOffsetRequest . Builder . forConsumer ( true , isolationLevel )
@ -393,14 +467,12 @@ class KafkaApisTest {
@@ -393,14 +467,12 @@ class KafkaApisTest {
* Return pair of listener names in the metadataCache : PLAINTEXT and LISTENER2 respectively .
*/
private def updateMetadataCacheWithInconsistentListeners ( ) : ( ListenerName , ListenerName ) = {
import UpdateMetadataRequest. { Broker => UBroker }
import UpdateMetadataRequest. { EndPoint => UEndPoint }
val plaintextListener = ListenerName . forSecurityProtocol ( SecurityProtocol . PLAINTEXT )
val anotherListener = new ListenerName ( "LISTENER2" )
val brokers = Set (
new U Broker( 0 , Seq ( new U EndPoint( "broker0" , 9092 , SecurityProtocol . PLAINTEXT , plaintextListener ) ,
new U EndPoint( "broker0" , 9093 , SecurityProtocol . PLAINTEXT , anotherListener ) ) . asJava , "rack" ) ,
new U Broker( 1 , Seq ( new U EndPoint( "broker1" , 9092 , SecurityProtocol . PLAINTEXT , plaintextListener ) ) . asJava ,
new Broker ( 0 , Seq ( new EndPoint ( "broker0" , 9092 , SecurityProtocol . PLAINTEXT , plaintextListener ) ,
new EndPoint ( "broker0" , 9093 , SecurityProtocol . PLAINTEXT , anotherListener ) ) . asJava , "rack" ) ,
new Broker ( 1 , Seq ( new EndPoint ( "broker1" , 9092 , SecurityProtocol . PLAINTEXT , plaintextListener ) ) . asJava ,
"rack" )
)
val updateMetadataRequest = new UpdateMetadataRequest . Builder ( ApiKeys . UPDATE_METADATA . latestVersion , 0 ,
@ -410,10 +482,7 @@ class KafkaApisTest {
@@ -410,10 +482,7 @@ class KafkaApisTest {
}
private def sendMetadataRequestWithInconsistentListeners ( requestListener : ListenerName ) : MetadataResponse = {
val capturedResponse = EasyMock . newCapture [ RequestChannel . Response ] ( )
val capturedThrottleCallback = EasyMock . newCapture [ Int => Unit ] ( )
expectThrottleCallbackAndInvoke ( capturedThrottleCallback )
EasyMock . expect ( requestChannel . sendResponse ( EasyMock . capture ( capturedResponse ) ) )
val capturedResponse = expectThrottleCallbackAndInvoke ( )
EasyMock . replay ( clientRequestQuotaManager , requestChannel )
val ( metadataRequest , requestChannelRequest ) = buildRequest ( MetadataRequest . Builder . allTopics , requestListener )
@ -426,8 +495,6 @@ class KafkaApisTest {
@@ -426,8 +495,6 @@ class KafkaApisTest {
val tp = new TopicPartition ( "foo" , 0 )
val latestOffset = 15L
val capturedResponse = EasyMock . newCapture [ RequestChannel . Response ] ( )
val capturedThrottleCallback = EasyMock . newCapture [ Int => Unit ] ( )
val replica = EasyMock . mock ( classOf [ Replica ] )
val log = EasyMock . mock ( classOf [ Log ] )
EasyMock . expect ( replicaManager . getLeaderReplicaIfLocal ( tp ) ) . andReturn ( replica )
@ -435,8 +502,8 @@ class KafkaApisTest {
@@ -435,8 +502,8 @@ class KafkaApisTest {
EasyMock . expect ( replica . highWatermark ) . andReturn ( LogOffsetMetadata ( messageOffset = latestOffset ) )
else
EasyMock . expect ( replica . lastStableOffset ) . andReturn ( LogOffsetMetadata ( messageOffset = latestOffset ) )
expectThrottleCallbackAndInvoke ( capturedThrottleCallback )
EasyMock . expect ( requestChannel . sendResponse ( EasyMock . capture ( capturedResponse ) ) )
val capturedResponse = expectThrottleCallbackAndInvoke ( )
EasyMock . replay ( replicaManager , clientRequestQuotaManager , requestChannel , replica , log )
val builder = ListOffsetRequest . Builder . forConsumer ( true , isolationLevel )
@ -484,7 +551,8 @@ class KafkaApisTest {
@@ -484,7 +551,8 @@ class KafkaApisTest {
AbstractResponse . parseResponse ( api , struct )
}
private def expectThrottleCallbackAndInvoke ( capturedThrottleCallback : Capture [ Int => Unit ] ) : Unit = {
private def expectThrottleCallbackAndInvoke ( ) : Capture [ RequestChannel . Response ] = {
val capturedThrottleCallback = EasyMock . newCapture [ Int => Unit ] ( )
EasyMock . expect ( clientRequestQuotaManager . maybeRecordAndThrottle (
EasyMock . anyObject [ RequestChannel . Request ] ( ) ,
EasyMock . capture ( capturedThrottleCallback ) ) )
@ -494,6 +562,21 @@ class KafkaApisTest {
@@ -494,6 +562,21 @@ class KafkaApisTest {
callback ( 0 )
}
} )
val capturedResponse = EasyMock . newCapture [ RequestChannel . Response ] ( )
EasyMock . expect ( requestChannel . sendResponse ( EasyMock . capture ( capturedResponse ) ) )
capturedResponse
}
private def setupBasicMetadataCache ( topic : String , numPartitions : Int = 1 ) : Unit = {
val replicas = List ( 0. asInstanceOf [ Integer ] ) . asJava
val partitionState = new UpdateMetadataRequest . PartitionState ( 1 , 0 , 1 , replicas , 0 , replicas , Collections . emptyList ( ) )
val plaintextListener = ListenerName . forSecurityProtocol ( SecurityProtocol . PLAINTEXT )
val broker = new Broker ( 0 , Seq ( new EndPoint ( "broker0" , 9092 , SecurityProtocol . PLAINTEXT , plaintextListener ) ) . asJava , "rack" )
val partitions = ( 0 until numPartitions ) . map ( new TopicPartition ( topic , _ ) -> partitionState ) . toMap
val updateMetadataRequest = new UpdateMetadataRequest . Builder ( ApiKeys . UPDATE_METADATA . latestVersion , 0 ,
0 , partitions . asJava , Set ( broker ) . asJava ) . build ( )
metadataCache . updateCache ( correlationId = 0 , updateMetadataRequest )
}
}