@Test public void testAdminClientApisAuthenticationFailure() throws Exception { Cluster cluster = mockBootstrapCluster(); try (final AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(Time.SYSTEM, cluster, newStrMap(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "1000"))) { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().createPendingAuthenticationError(cluster.nodes().get(0), TimeUnit.DAYS.toMillis(1)); callAdminClientApisAndExpectAnAuthenticationError(env); } }
/** * Test that the client properly times out when we don't receive any metadata. */ @Test public void testTimeoutWithoutMetadata() throws Exception { try (final AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(Time.SYSTEM, mockBootstrapCluster(), newStrMap(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "10"))) { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().prepareResponse(new CreateTopicsResponse(Collections.singletonMap("myTopic", new ApiError(Errors.NONE, "")))); KafkaFuture<Void> future = env.adminClient().createTopics( Collections.singleton(new NewTopic("myTopic", Collections.singletonMap(0, asList(0, 1, 2)))), new CreateTopicsOptions().timeoutMs(1000)).all(); TestUtils.assertFutureError(future, TimeoutException.class); } }
@Test public void testConnectionFailureOnMetadataUpdate() throws Exception { // This tests the scenario in which we successfully connect to the bootstrap server, but // the server disconnects before sending the full response Cluster cluster = mockBootstrapCluster(); try (final AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(Time.SYSTEM, cluster)) { Cluster discoveredCluster = mockCluster(0); env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().prepareResponse(request -> request instanceof MetadataRequest, null, true); env.kafkaClient().prepareResponse(request -> request instanceof MetadataRequest, new MetadataResponse(discoveredCluster.nodes(), discoveredCluster.clusterResource().clusterId(), 1, Collections.emptyList())); env.kafkaClient().prepareResponse(body -> body instanceof CreateTopicsRequest, new CreateTopicsResponse(Collections.singletonMap("myTopic", new ApiError(Errors.NONE, "")))); KafkaFuture<Void> future = env.adminClient().createTopics( Collections.singleton(new NewTopic("myTopic", Collections.singletonMap(0, asList(0, 1, 2)))), new CreateTopicsOptions().timeoutMs(10000)).all(); future.get(); } }