yashmayya commented on code in PR #14538:
URL: https://github.com/apache/kafka/pull/14538#discussion_r1362109618


##########
connect/runtime/src/test/java/org/apache/kafka/connect/integration/StandaloneWorkerTest.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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 org.apache.kafka.connect.integration;
+
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel;
+import org.apache.kafka.connect.util.clusters.EmbeddedConnectStandalone;
+import org.apache.kafka.test.IntegrationTest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+@Category(IntegrationTest.class)
+public class StandaloneWorkerTest {

Review Comment:
   The existing integration test classes use the suffix `IntegrationTest` in 
their class names to distinguish them more easily from unit test classes that 
only use the `Test` suffix; any particular reason for not doing so here?



##########
connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectStandalone.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.kafka.connect.util.clusters;
+
+import org.apache.kafka.connect.cli.ConnectStandalone;
+import org.apache.kafka.connect.runtime.Connect;
+import org.apache.kafka.test.TestUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import static 
org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.WorkerConfig.BOOTSTRAP_SERVERS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.WorkerConfig.PLUGIN_DISCOVERY_CONFIG;
+import static 
org.apache.kafka.connect.runtime.rest.RestServerConfig.LISTENERS_CONFIG;
+import static 
org.apache.kafka.connect.runtime.standalone.StandaloneConfig.OFFSET_STORAGE_FILE_FILENAME_CONFIG;
+
+/**
+ * Start a standalone embedded connect worker. Internally, this class will 
spin up a Kafka and Zk cluster,
+ * setup any tmp directories and clean up them on them. Methods on the same

Review Comment:
   > clean up them on them
   
   Was this supposed to be something like `clean them up on exit`?



##########
connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectBuilder.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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 org.apache.kafka.connect.util.clusters;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+abstract class EmbeddedConnectBuilder<C extends EmbeddedConnect, B extends 
EmbeddedConnectBuilder<C, B>> {

Review Comment:
   Interesting pattern! Could we add a small helper method that just returns 
`(B) this` so we can centralize the "unchecked" warning suppression to a single 
method instead of adding it to every method in the builder?



##########
connect/runtime/src/test/java/org/apache/kafka/connect/integration/StandaloneWorkerTest.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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 org.apache.kafka.connect.integration;
+
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel;
+import org.apache.kafka.connect.util.clusters.EmbeddedConnectStandalone;
+import org.apache.kafka.test.IntegrationTest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+@Category(IntegrationTest.class)
+public class StandaloneWorkerTest {
+
+    private EmbeddedConnectStandalone connect;
+
+    @Before
+    public void setup() {
+        connect = new EmbeddedConnectStandalone.Builder()
+                .build();
+        connect.start();
+    }
+
+    @After
+    public void cleanup() {
+        connect.stop();
+    }
+
+    @Test
+    public void testDynamicLogging() {
+        Map<String, LoggerLevel> initialLevels = connect.allLogLevels();
+        assertFalse("Connect REST API did not list any known loggers", 
initialLevels.isEmpty());
+        Map<String, LoggerLevel> invalidModifiedLoggers = Utils.filterMap(
+                initialLevels,
+                StandaloneWorkerTest::isModified
+        );
+        assertEquals(
+                "No loggers should have a non-null last-modified timestamp",
+                Collections.emptyMap(),
+                invalidModifiedLoggers
+        );
+
+        // Tests with no scope
+        // The current level may match the first level we set the namespace to,
+        // so we issue a preliminary request with a different level to 
guarantee that a
+        // change takes place and that the last modified timestamp should be 
non-null
+        final String namespace1 = "org.apache.kafka.connect";
+        final String level1 = "DEBUG";
+        connect.setLogLevel(namespace1, "ERROR", null);
+        Map<String, LoggerLevel> currentLevels = 
testSetLoggingLevel(namespace1, level1, null, initialLevels);
+
+        // Tests with scope=worker
+        final String namespace2 = "org.apache.kafka.clients";
+        final String level2 = "INFO";
+        connect.setLogLevel(namespace2, "WARN", "worker");
+        currentLevels = testSetLoggingLevel(namespace2, level2, "worker", 
currentLevels);
+
+        LoggerLevel priorLoggerLevel = connect.getLogLevel(namespace2);
+        connect.setLogLevel(namespace2, level2, "worker");
+        LoggerLevel currentLoggerLevel = connect.getLogLevel(namespace2);
+        assertEquals(
+                "Log level and last-modified timestamp should not be affected 
by consecutive identical requests",
+                priorLoggerLevel,
+                currentLoggerLevel
+        );
+
+        // Tests with scope=cluster
+        final String namespace3 = "org.apache.kafka.streams";
+        final String level3 = "TRACE";
+        connect.setLogLevel(namespace3, "DEBUG", "cluster");
+        testSetLoggingLevel(namespace3, level3, "cluster", currentLevels);
+    }
+
+    private Map<String, LoggerLevel> testSetLoggingLevel(

Review Comment:
   This is impressively comprehensive, I really like it!



-- 
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.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to