|
|
|
@ -30,6 +30,7 @@ import org.apache.kafka.common.acl.AclBindingFilter;
@@ -30,6 +30,7 @@ import org.apache.kafka.common.acl.AclBindingFilter;
|
|
|
|
|
import org.apache.kafka.common.acl.AclOperation; |
|
|
|
|
import org.apache.kafka.common.acl.AclPermissionType; |
|
|
|
|
import org.apache.kafka.common.config.ConfigResource; |
|
|
|
|
import org.apache.kafka.common.errors.AuthenticationException; |
|
|
|
|
import org.apache.kafka.common.errors.InvalidTopicException; |
|
|
|
|
import org.apache.kafka.common.errors.LeaderNotAvailableException; |
|
|
|
|
import org.apache.kafka.common.errors.NotLeaderForPartitionException; |
|
|
|
@ -248,6 +249,75 @@ public class KafkaAdminClientTest {
@@ -248,6 +249,75 @@ public class KafkaAdminClientTest {
|
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@Test |
|
|
|
|
public void testAdminClientApisWithinBlackoutPeriodAfterAuthenticationFailure() throws Exception { |
|
|
|
|
AdminClientUnitTestEnv env = mockClientEnv(); |
|
|
|
|
Node node = env.cluster().controller(); |
|
|
|
|
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); |
|
|
|
|
env.kafkaClient().setNode(node); |
|
|
|
|
env.kafkaClient().authenticationFailed(node, 300); |
|
|
|
|
|
|
|
|
|
callAdminClientApisAndExpectAnAuthenticationError(env); |
|
|
|
|
|
|
|
|
|
// wait less than the blackout period, the connection should fail and the authentication error should remain
|
|
|
|
|
env.time().sleep(30); |
|
|
|
|
assertTrue(env.kafkaClient().connectionFailed(node)); |
|
|
|
|
callAdminClientApisAndExpectAnAuthenticationError(env); |
|
|
|
|
|
|
|
|
|
env.close(); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
private void callAdminClientApisAndExpectAnAuthenticationError(AdminClientUnitTestEnv env) throws InterruptedException { |
|
|
|
|
env.kafkaClient().prepareMetadataUpdate(env.cluster(), Collections.<String>emptySet()); |
|
|
|
|
|
|
|
|
|
try { |
|
|
|
|
env.adminClient().createTopics( |
|
|
|
|
Collections.singleton(new NewTopic("myTopic", Collections.singletonMap(Integer.valueOf(0), asList(new Integer[]{0, 1, 2})))), |
|
|
|
|
new CreateTopicsOptions().timeoutMs(10000)).all().get(); |
|
|
|
|
fail("Expected an authentication error."); |
|
|
|
|
} catch (ExecutionException e) { |
|
|
|
|
assertTrue("Expected only an authentication error.", e.getCause() instanceof AuthenticationException); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
try { |
|
|
|
|
Map<String, NewPartitions> counts = new HashMap<>(); |
|
|
|
|
counts.put("my_topic", NewPartitions.increaseTo(3)); |
|
|
|
|
counts.put("other_topic", NewPartitions.increaseTo(3, asList(asList(2), asList(3)))); |
|
|
|
|
env.adminClient().createPartitions(counts).all().get(); |
|
|
|
|
fail("Expected an authentication error."); |
|
|
|
|
} catch (ExecutionException e) { |
|
|
|
|
assertTrue("Expected only an authentication error.", e.getCause() instanceof AuthenticationException); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
try { |
|
|
|
|
env.adminClient().createAcls(asList(ACL1, ACL2)).all().get(); |
|
|
|
|
fail("Expected an authentication error."); |
|
|
|
|
} catch (ExecutionException e) { |
|
|
|
|
assertTrue("Expected only an authentication error.", e.getCause() instanceof AuthenticationException); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
try { |
|
|
|
|
env.adminClient().describeAcls(FILTER1).values().get(); |
|
|
|
|
fail("Expected an authentication error."); |
|
|
|
|
} catch (ExecutionException e) { |
|
|
|
|
assertTrue("Expected only an authentication error.", e.getCause() instanceof AuthenticationException); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
try { |
|
|
|
|
env.adminClient().deleteAcls(asList(FILTER1, FILTER2)).all().get(); |
|
|
|
|
fail("Expected an authentication error."); |
|
|
|
|
} catch (ExecutionException e) { |
|
|
|
|
assertTrue("Expected only an authentication error.", e.getCause() instanceof AuthenticationException); |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
try { |
|
|
|
|
env.adminClient().describeConfigs(Collections.singleton(new ConfigResource(ConfigResource.Type.BROKER, "0"))).all().get(); |
|
|
|
|
fail("Expected an authentication error."); |
|
|
|
|
} catch (ExecutionException e) { |
|
|
|
|
assertTrue("Expected only an authentication error.", e.getCause() instanceof AuthenticationException); |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
private static final AclBinding ACL1 = new AclBinding(new Resource(ResourceType.TOPIC, "mytopic3"), |
|
|
|
|
new AccessControlEntry("User:ANONYMOUS", "*", AclOperation.DESCRIBE, AclPermissionType.ALLOW)); |
|
|
|
|
private static final AclBinding ACL2 = new AclBinding(new Resource(ResourceType.TOPIC, "mytopic4"), |
|
|
|
@ -579,7 +649,7 @@ public class KafkaAdminClientTest {
@@ -579,7 +649,7 @@ public class KafkaAdminClientTest {
|
|
|
|
|
private int numTries = 0; |
|
|
|
|
|
|
|
|
|
private int failuresInjected = 0; |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
@Override |
|
|
|
|
public KafkaAdminClient.TimeoutProcessor create(long now) { |
|
|
|
|
return new FailureInjectingTimeoutProcessor(now); |
|
|
|
|