diff --git a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListTopicsExecutor.java b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListTopicsExecutor.java index c7d34ed66cdc..96da8ecec210 100644 --- a/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListTopicsExecutor.java +++ b/ksql-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListTopicsExecutor.java @@ -125,7 +125,7 @@ private static Map filterKsqlInternalTopics( for (final Map.Entry entry : kafkaTopicDescriptions.entrySet()) { if (!entry.getKey().startsWith(serviceId + persistentQueryPrefix) && !entry.getKey().startsWith(serviceId + transientQueryPrefix)) { - filteredKafkaTopics.put(entry.getKey().toLowerCase(), entry.getValue()); + filteredKafkaTopics.put(entry.getKey(), entry.getValue()); } } return filteredKafkaTopics; diff --git a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListTopicsExecutorTest.java b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListTopicsExecutorTest.java index 0f0d2c3b99f8..008b7faedc07 100644 --- a/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListTopicsExecutorTest.java +++ b/ksql-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListTopicsExecutorTest.java @@ -34,15 +34,32 @@ import org.apache.kafka.clients.admin.ConsumerGroupListing; import org.apache.kafka.clients.admin.ListConsumerGroupsResult; import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; +import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; @RunWith(MockitoJUnitRunner.class) public class ListTopicsExecutorTest { @Rule public final TemporaryEngine engine = new TemporaryEngine(); + @Mock + private AdminClient adminClient; + + private ServiceContext serviceContext; + + @Before + public void setUp() { + serviceContext = TestServiceContext.create( + engine.getServiceContext().getKafkaClientSupplier(), + adminClient, + engine.getServiceContext().getTopicClient(), + engine.getServiceContext().getSchemaRegistryClientFactory(), + engine.getServiceContext().getConnectClient() + ); + } @Test public void shouldListKafkaTopics() { @@ -50,15 +67,27 @@ public void shouldListKafkaTopics() { engine.givenKafkaTopic("topic1"); engine.givenKafkaTopic("topic2"); - final AdminClient mockAdminClient = mock(AdminClient.class); + // When: + final KafkaTopicsList topicsList = + (KafkaTopicsList) CustomExecutors.LIST_TOPICS.execute( + engine.configure("LIST TOPICS;"), + ImmutableMap.of(), + engine.getEngine(), + serviceContext + ).orElseThrow(IllegalStateException::new); - final ServiceContext serviceContext = TestServiceContext.create( - engine.getServiceContext().getKafkaClientSupplier(), - mockAdminClient, - engine.getServiceContext().getTopicClient(), - engine.getServiceContext().getSchemaRegistryClientFactory(), - engine.getServiceContext().getConnectClient() - ); + // Then: + assertThat(topicsList.getTopics(), containsInAnyOrder( + new KafkaTopicInfo("topic1", ImmutableList.of(1)), + new KafkaTopicInfo("topic2", ImmutableList.of(1)) + )); + } + + @Test + public void shouldListKafkaTopicsThatDifferByCase() { + // Given: + engine.givenKafkaTopic("topic1"); + engine.givenKafkaTopic("toPIc1"); // When: final KafkaTopicsList topicsList = @@ -72,7 +101,7 @@ public void shouldListKafkaTopics() { // Then: assertThat(topicsList.getTopics(), containsInAnyOrder( new KafkaTopicInfo("topic1", ImmutableList.of(1)), - new KafkaTopicInfo("topic2", ImmutableList.of(1)) + new KafkaTopicInfo("toPIc1", ImmutableList.of(1)) )); } @@ -82,22 +111,13 @@ public void shouldListKafkaTopicsExtended() { engine.givenKafkaTopic("topic1"); engine.givenKafkaTopic("topic2"); - final AdminClient mockAdminClient = mock(AdminClient.class); final ListConsumerGroupsResult result = mock(ListConsumerGroupsResult.class); final KafkaFutureImpl> groups = new KafkaFutureImpl<>(); when(result.all()).thenReturn(groups); - when(mockAdminClient.listConsumerGroups()).thenReturn(result); + when(adminClient.listConsumerGroups()).thenReturn(result); groups.complete(ImmutableList.of()); - final ServiceContext serviceContext = TestServiceContext.create( - engine.getServiceContext().getKafkaClientSupplier(), - mockAdminClient, - engine.getServiceContext().getTopicClient(), - engine.getServiceContext().getSchemaRegistryClientFactory(), - engine.getServiceContext().getConnectClient() - ); - // When: final KafkaTopicsListExtended topicsList = (KafkaTopicsListExtended) CustomExecutors.LIST_TOPICS.execute( @@ -113,5 +133,4 @@ public void shouldListKafkaTopicsExtended() { new KafkaTopicInfoExtended("topic2", ImmutableList.of(1), 0, 0) )); } - }