hachikuji commented on a change in pull request #9986: URL: https://github.com/apache/kafka/pull/9986#discussion_r567073840
########## File path: core/src/test/java/kafka/test/annotation/ClusterProperty.java ########## @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.annotation; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Documented +@Target({ElementType.ANNOTATION_TYPE}) +@Retention(RetentionPolicy.RUNTIME) +public @interface ClusterProperty { Review comment: Maybe `ClusterConfig`? Would be nice to clarify the scope? I assume this is only for `KafkaConfig`? ########## File path: core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java ########## @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.junit; + +import integration.kafka.server.IntegrationTestHelper; +import kafka.api.IntegrationTestHarness; +import kafka.network.SocketServer; +import kafka.server.KafkaServer; +import kafka.utils.TestUtils; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import kafka.test.ClusterConfig; +import kafka.test.ClusterInstance; +import org.junit.jupiter.api.extension.AfterTestExecutionCallback; +import org.junit.jupiter.api.extension.BeforeTestExecutionCallback; +import org.junit.jupiter.api.extension.Extension; +import org.junit.jupiter.api.extension.TestTemplateInvocationContext; +import scala.Option; +import scala.collection.JavaConverters; +import scala.compat.java8.OptionConverters; + +import java.io.File; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +/** + * Wraps a {@link IntegrationTestHarness} inside lifecycle methods for a test invocation. Each instance of this + * class is provided with a configuration for the cluster. + * + * This context also provides parameter resolvers for: + * + * <ul> + * <li>ClusterConfig (the same instance passed to the constructor)</li> + * <li>ClusterInstance (includes methods to expose underlying SocketServer-s)</li> + * <li>IntegrationTestHelper (helper methods)</li> + * </ul> + */ +public class ZkClusterInvocationContext implements TestTemplateInvocationContext { + + private final ClusterConfig clusterConfig; + private final AtomicReference<IntegrationTestHarness> clusterReference; + + public ZkClusterInvocationContext(ClusterConfig clusterConfig) { + this.clusterConfig = clusterConfig; + this.clusterReference = new AtomicReference<>(); + } + + @Override + public String getDisplayName(int invocationIndex) { + String clusterDesc = clusterConfig.nameTags().entrySet().stream() + .map(Object::toString) + .collect(Collectors.joining(", ")); + return String.format("[Zk %d] %s", invocationIndex, clusterDesc); + } + + @Override + public List<Extension> getAdditionalExtensions() { + ClusterInstance clusterShim = new ZkClusterInstance(clusterConfig, clusterReference); + return Arrays.asList( + (BeforeTestExecutionCallback) context -> { + // We have to wait to actually create the underlying cluster until after our @BeforeEach methods + // have run. This allows tests to set up external dependencies like ZK, MiniKDC, etc. + // However, since we cannot create this instance until we are inside the test invocation, we have + // to use a container class (AtomicReference) to provide this cluster object to the test itself + + // This is what tests normally extend from to start a cluster, here we create it anonymously and + // configure the cluster using values from ClusterConfig + IntegrationTestHarness cluster = new IntegrationTestHarness() { + @Override + public Properties serverConfig() { + return clusterConfig.serverProperties(); + } + + @Override + public Properties adminClientConfig() { + return clusterConfig.adminClientProperties(); + } + + @Override + public Properties consumerConfig() { + return clusterConfig.consumerProperties(); + } + + @Override + public Properties producerConfig() { + return clusterConfig.producerProperties(); + } + + @Override + public SecurityProtocol securityProtocol() { + return SecurityProtocol.forName(clusterConfig.securityProtocol()); + } + + @Override + public ListenerName listenerName() { + return clusterConfig.listenerName().map(ListenerName::normalised) + .orElseGet(() -> ListenerName.forSecurityProtocol(securityProtocol())); + } + + @Override + public Option<Properties> serverSaslProperties() { + if (clusterConfig.saslServerProperties().isEmpty()) { + return Option.empty(); + } else { + return Option.apply(clusterConfig.saslServerProperties()); + } + } + + @Override + public Option<Properties> clientSaslProperties() { + if (clusterConfig.saslClientProperties().isEmpty()) { + return Option.empty(); + } else { + return Option.apply(clusterConfig.saslClientProperties()); + } + } + + @Override + public int brokerCount() { + // Brokers and controllers are the same in zk mode, so just use the max + return Math.max(clusterConfig.brokers(), clusterConfig.controllers()); + } + + @Override + public Option<File> trustStoreFile() { + return OptionConverters.toScala(clusterConfig.trustStoreFile()); + } + }; + + // TODO consumer and producer configs Review comment: File a jira instead? ########## File path: core/src/test/java/kafka/test/ClusterForEachTest.java ########## @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test; + +import kafka.test.annotation.AutoStart; +import kafka.test.annotation.ClusterProperty; +import kafka.test.annotation.ClusterTemplate; +import kafka.test.annotation.ClusterTest; +import kafka.test.annotation.ClusterTestDefaults; +import kafka.test.annotation.ClusterTests; +import kafka.test.annotation.Type; +import kafka.test.junit.ClusterForEach; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + + +@ClusterTestDefaults(clusterType = Type.Zk) // Set defaults for a few params in @ClusterTest(s) +@ExtendWith(ClusterForEach.class) Review comment: Wow, so meta. Is it possible to have any validation that this gets executed? ########## File path: core/src/test/java/kafka/test/annotation/ClusterTest.java ########## @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.annotation; + +import org.junit.jupiter.api.TestTemplate; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Documented +@Target({METHOD}) +@Retention(RUNTIME) +@TestTemplate +public @interface ClusterTest { + Type clusterType() default Type.Default; + int brokers() default 0; + int controllers() default 0; + AutoStart autoStart() default AutoStart.Default; + + String name() default ""; + String securityProtocol() default "PLAINTEXT"; + String listener() default ""; + ClusterProperty[] serverProperties() default {}; +} + Review comment: nit: unneeded newlines ########## File path: core/src/test/java/kafka/test/ClusterInstance.java ########## @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test; + +import kafka.network.SocketServer; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.common.network.ListenerName; + +import java.util.Collection; +import java.util.Optional; +import java.util.Properties; + +public interface ClusterInstance { + + enum ClusterType { + Zk, + // Raft + } + + ClusterType clusterType(); + + ClusterConfig config(); + + ListenerName listener(); + + String brokerList(); + + Collection<SocketServer> brokers(); + + Collection<SocketServer> controllers(); Review comment: Might be useful to have some documentation for these methods. I think `controllers` will always return empty list if this is a zk cluster? ########## File path: build.gradle ########## @@ -1011,7 +1011,10 @@ project(':core') { } test { java { - srcDirs = ["src/generated/java", "src/test/java"] + srcDirs = [] + } + scala { + srcDirs = ["src/generated/java", "src/test/java", "src/test/scala"] Review comment: Do we need `src/generated/java` to be built with scala? I think it shouldn't have any dependence on anything in core. ########## File path: core/src/test/java/kafka/test/junit/ClusterForEach.java ########## @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.test.junit; + +import kafka.test.ClusterConfig; +import kafka.test.ClusterGenerator; +import kafka.test.annotation.ClusterTestDefaults; +import kafka.test.annotation.ClusterProperty; +import kafka.test.annotation.ClusterTemplate; +import kafka.test.annotation.ClusterTest; +import kafka.test.annotation.ClusterTests; +import kafka.test.annotation.Type; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.api.extension.TestTemplateInvocationContext; +import org.junit.jupiter.api.extension.TestTemplateInvocationContextProvider; +import org.junit.platform.commons.util.ReflectionUtils; + +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.function.Consumer; +import java.util.stream.Stream; + +/** + * ClusterForEach is a custom JUnit extension that will generate some number of test invocations depending on a few + * custom annotations. These annotations are placed on so-called test template methods. Template methods look like + * normal JUnit test methods, but instead of being invoked directly, they are used as templates for generating + * multiple test invocations. + * + * Test class that use this extension should use one of the following annotations on each template method: + * + * <ul> + * <li>{@link ClusterTest}, define a single cluster configuration</li> + * <li>{@link ClusterTests}, provide multiple instances of @ClusterTest</li> + * <li>{@link ClusterTemplate}, define a static method that generates cluster configurations</li> + * </ul> + * + * Any combination of these annotations may be used on a given test template method. If no test invocations are + * generated after processing the annotations, an error is thrown. + * + * Depending on which annotations are used, and what values are given, different {@link ClusterConfig} will be + * generated. Each ClusterConfig is used to create an underlying Kafka cluster that is used for the actual test + * invocation. + * + * For example: + * + * <pre> + * @ExtendWith(value = Array(classOf[ClusterForEach])) + * class SomeIntegrationTest { + * @ClusterTest(brokers = 1, controllers = 1, clusterType = ClusterType.Both) + * def someTest(): Unit = { + * assertTrue(condition) + * } + * } + * </pre> + * + * will generate two invocations of "someTest" (since ClusterType.Both was given). For each invocation, the test class + * SomeIntegrationTest will be instantiated, lifecycle methods (before/after) will be run, and "someTest" will be invoked. + * + **/ +public class ClusterForEach implements TestTemplateInvocationContextProvider { Review comment: Wonder if the name could be improved. It's boring, but would about `ClusterTestExtensions`? ```scala @ExtendWith(Array(classOf[ClusterTestExtensions])) class SomeIntegrationTest ``` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org