From fd20bbd30ecee969fd3f38cbe1d45de7d763f531 Mon Sep 17 00:00:00 2001 From: imply-cheddar <86940447+imply-cheddar@users.noreply.github.com> Date: Wed, 28 Jun 2023 13:49:27 +0900 Subject: [PATCH] Fix another infinite loop and remove Mockito usage (#14493) * Fix another infinite loop and remove Mockito usage The ConfigManager objects were `started()` without ever being stopped. This scheduled a poll call that never-ended, to make matters worse, the poll interval was set to 0 ms, making an infinite poll with 0 sleep, i.e. an infinite loop. Also introduce test classes and remove usage of mocks * Checkstyle --- .../apache/druid/audit/TestAuditManager.java | 71 +++++++ .../common/config/ConfigManagerTest.java | 182 +++++++++++------- .../config/TestConfigManagerConfig.java | 40 ++++ .../TestMetadataStorageConnector.java | 92 +++++++++ .../TestMetadataStorageTablesConfig.java | 43 +++++ 5 files changed, 361 insertions(+), 67 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/audit/TestAuditManager.java create mode 100644 processing/src/test/java/org/apache/druid/common/config/TestConfigManagerConfig.java create mode 100644 processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java create mode 100644 processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java diff --git a/processing/src/test/java/org/apache/druid/audit/TestAuditManager.java b/processing/src/test/java/org/apache/druid/audit/TestAuditManager.java new file mode 100644 index 00000000000..fd34330a68b --- /dev/null +++ b/processing/src/test/java/org/apache/druid/audit/TestAuditManager.java @@ -0,0 +1,71 @@ +/* + * 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.druid.audit; + +import org.apache.druid.common.config.ConfigSerde; +import org.joda.time.Interval; +import org.skife.jdbi.v2.Handle; + +import java.util.List; + +public class TestAuditManager implements AuditManager +{ + @Override + public void doAudit(String key, String type, AuditInfo auditInfo, T payload, ConfigSerde configSerde) + { + throw new UnsupportedOperationException(); + } + + @Override + public void doAudit(AuditEntry auditEntry, Handle handler) + { + throw new UnsupportedOperationException(); + } + + @Override + public List fetchAuditHistory(String key, String type, Interval interval) + { + throw new UnsupportedOperationException(); + } + + @Override + public List fetchAuditHistory(String type, Interval interval) + { + throw new UnsupportedOperationException(); + } + + @Override + public List fetchAuditHistory(String key, String type, int limit) + { + throw new UnsupportedOperationException(); + } + + @Override + public List fetchAuditHistory(String type, int limit) + { + throw new UnsupportedOperationException(); + } + + @Override + public int removeAuditLogsOlderThan(long timestamp) + { + throw new UnsupportedOperationException(); + } +} diff --git a/processing/src/test/java/org/apache/druid/common/config/ConfigManagerTest.java b/processing/src/test/java/org/apache/druid/common/config/ConfigManagerTest.java index 9705939991f..2f58147e77c 100644 --- a/processing/src/test/java/org/apache/druid/common/config/ConfigManagerTest.java +++ b/processing/src/test/java/org/apache/druid/common/config/ConfigManagerTest.java @@ -25,26 +25,19 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Suppliers; import org.apache.druid.audit.AuditManager; +import org.apache.druid.audit.TestAuditManager; import org.apache.druid.metadata.MetadataCASUpdate; import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageTablesConfig; -import org.joda.time.Period; +import org.apache.druid.metadata.TestMetadataStorageConnector; +import org.apache.druid.metadata.TestMetadataStorageTablesConfig; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.ArgumentMatchers; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.junit.MockitoJUnitRunner; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.when; - -@RunWith(MockitoJUnitRunner.class) +@SuppressWarnings("ALL") public class ConfigManagerTest { private static final String CONFIG_KEY = "configX"; @@ -52,33 +45,42 @@ public class ConfigManagerTest private static final byte[] OLD_CONFIG = {1, 2, 3}; private static final TestConfig NEW_CONFIG = new TestConfig("2", "y", 2); - @Mock - private MetadataStorageConnector mockDbConnector; - - @Mock - private MetadataStorageTablesConfig mockMetadataStorageTablesConfig; - - @Mock + private MetadataStorageConnector dbConnector; + private MetadataStorageTablesConfig metadataStorageTablesConfig; private AuditManager mockAuditManager; - - @Mock - private ConfigManagerConfig mockConfigManagerConfig; + private TestConfigManagerConfig configManagerConfig; private ConfigSerde configConfigSerdeFromClass; private ConfigManager configManager; private JacksonConfigManager jacksonConfigManager; - @Before public void setup() { - when(mockMetadataStorageTablesConfig.getConfigTable()).thenReturn(TABLE_NAME); - when(mockConfigManagerConfig.getPollDuration()).thenReturn(new Period()); - configManager = new ConfigManager(mockDbConnector, Suppliers.ofInstance(mockMetadataStorageTablesConfig), Suppliers.ofInstance(mockConfigManagerConfig)); + setup(new TestMetadataStorageConnector()); + } + + public void setup(TestMetadataStorageConnector dbConnector) + { + this.dbConnector = dbConnector; + metadataStorageTablesConfig = new TestMetadataStorageTablesConfig() + { + @Override + public String getConfigTable() + { + return TABLE_NAME; + } + }; + configManagerConfig = new TestConfigManagerConfig(); + configManager = new ConfigManager( + this.dbConnector, + Suppliers.ofInstance(metadataStorageTablesConfig), + Suppliers.ofInstance(configManagerConfig) + ); jacksonConfigManager = new JacksonConfigManager( configManager, new ObjectMapper(), new ObjectMapper().setSerializationInclusion(JsonInclude.Include.NON_NULL), - mockAuditManager + new TestAuditManager() ); configConfigSerdeFromClass = jacksonConfigManager.create(TestConfig.class, null); } @@ -86,6 +88,7 @@ public class ConfigManagerTest @Test public void testSetNewObjectIsNull() { + setup(); ConfigManager.SetResult setResult = configManager.set(CONFIG_KEY, configConfigSerdeFromClass, null); Assert.assertFalse(setResult.isOk()); Assert.assertFalse(setResult.isRetryable()); @@ -95,6 +98,7 @@ public class ConfigManagerTest @Test public void testSetConfigManagerNotStarted() { + setup(); ConfigManager.SetResult setResult = configManager.set(CONFIG_KEY, configConfigSerdeFromClass, NEW_CONFIG); Assert.assertFalse(setResult.isOk()); Assert.assertFalse(setResult.isRetryable()); @@ -104,56 +108,100 @@ public class ConfigManagerTest @Test public void testSetOldObjectNullShouldInsertWithoutSwap() { - configManager.start(); - ConfigManager.SetResult setResult = configManager.set(CONFIG_KEY, configConfigSerdeFromClass, null, NEW_CONFIG); - Assert.assertTrue(setResult.isOk()); - Mockito.verify(mockDbConnector).insertOrUpdate( - ArgumentMatchers.eq(TABLE_NAME), - ArgumentMatchers.anyString(), - ArgumentMatchers.anyString(), - ArgumentMatchers.eq(CONFIG_KEY), - ArgumentMatchers.any(byte[].class) - ); - Mockito.verifyNoMoreInteractions(mockDbConnector); + final AtomicBoolean called = new AtomicBoolean(); + setup(new TestMetadataStorageConnector() + { + + @Override + public Void insertOrUpdate(String tableName, String keyColumn, String valueColumn, String key, byte[] value) + { + Assert.assertFalse(called.getAndSet(true)); + Assert.assertEquals(TABLE_NAME, tableName); + Assert.assertEquals(CONFIG_KEY, key); + return null; + } + }); + + try { + configManager.start(); + ConfigManager.SetResult setResult = configManager.set(CONFIG_KEY, configConfigSerdeFromClass, null, NEW_CONFIG); + Assert.assertTrue(setResult.isOk()); + Assert.assertTrue(called.get()); + } + finally { + configManager.stop(); + } } @Test public void testSetOldObjectNotNullShouldSwap() { - when(mockConfigManagerConfig.isEnableCompareAndSwap()).thenReturn(true); - when(mockDbConnector.compareAndSwap(any(List.class))).thenReturn(true); - final ArgumentCaptor> updateCaptor = ArgumentCaptor.forClass(List.class); - configManager.start(); - ConfigManager.SetResult setResult = configManager.set(CONFIG_KEY, configConfigSerdeFromClass, OLD_CONFIG, NEW_CONFIG); - Assert.assertTrue(setResult.isOk()); - Mockito.verify(mockDbConnector).compareAndSwap( - updateCaptor.capture() - ); - Mockito.verifyNoMoreInteractions(mockDbConnector); - Assert.assertEquals(1, updateCaptor.getValue().size()); - Assert.assertEquals(TABLE_NAME, updateCaptor.getValue().get(0).getTableName()); - Assert.assertEquals(MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, updateCaptor.getValue().get(0).getKeyColumn()); - Assert.assertEquals(MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, updateCaptor.getValue().get(0).getValueColumn()); - Assert.assertEquals(CONFIG_KEY, updateCaptor.getValue().get(0).getKey()); - Assert.assertArrayEquals(OLD_CONFIG, updateCaptor.getValue().get(0).getOldValue()); - Assert.assertArrayEquals(configConfigSerdeFromClass.serialize(NEW_CONFIG), updateCaptor.getValue().get(0).getNewValue()); + final AtomicBoolean called = new AtomicBoolean(); + setup(new TestMetadataStorageConnector() + { + @Override + public boolean compareAndSwap(List updates) + { + Assert.assertFalse(called.getAndSet(true)); + Assert.assertEquals(1, updates.size()); + final MetadataCASUpdate singularUpdate = updates.get(0); + Assert.assertEquals(TABLE_NAME, singularUpdate.getTableName()); + Assert.assertEquals(MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, singularUpdate.getKeyColumn()); + Assert.assertEquals(MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, singularUpdate.getValueColumn()); + Assert.assertEquals(CONFIG_KEY, singularUpdate.getKey()); + Assert.assertArrayEquals(OLD_CONFIG, singularUpdate.getOldValue()); + Assert.assertArrayEquals(configConfigSerdeFromClass.serialize(NEW_CONFIG), singularUpdate.getNewValue()); + return true; + } + }); + try { + configManager.start(); + ConfigManager.SetResult setResult = configManager.set( + CONFIG_KEY, + configConfigSerdeFromClass, + OLD_CONFIG, + NEW_CONFIG + ); + Assert.assertTrue(setResult.isOk()); + Assert.assertTrue(called.get()); + } + finally { + configManager.stop(); + } } @Test public void testSetOldObjectNotNullButCompareAndSwapDisabledShouldInsertWithoutSwap() { - when(mockConfigManagerConfig.isEnableCompareAndSwap()).thenReturn(false); - configManager.start(); - ConfigManager.SetResult setResult = configManager.set(CONFIG_KEY, configConfigSerdeFromClass, OLD_CONFIG, NEW_CONFIG); - Assert.assertTrue(setResult.isOk()); - Mockito.verify(mockDbConnector).insertOrUpdate( - ArgumentMatchers.eq(TABLE_NAME), - ArgumentMatchers.anyString(), - ArgumentMatchers.anyString(), - ArgumentMatchers.eq(CONFIG_KEY), - ArgumentMatchers.any(byte[].class) - ); - Mockito.verifyNoMoreInteractions(mockDbConnector); + final AtomicBoolean called = new AtomicBoolean(); + + setup(new TestMetadataStorageConnector() + { + @Override + public Void insertOrUpdate(String tableName, String keyColumn, String valueColumn, String key, byte[] value) + { + Assert.assertFalse(called.getAndSet(true)); + Assert.assertEquals(TABLE_NAME, tableName); + Assert.assertEquals(CONFIG_KEY, key); + return null; + } + }); + configManagerConfig.enableCompareAndSwap = false; + + try { + configManager.start(); + ConfigManager.SetResult setResult = configManager.set( + CONFIG_KEY, + configConfigSerdeFromClass, + OLD_CONFIG, + NEW_CONFIG + ); + Assert.assertTrue(setResult.isOk()); + Assert.assertTrue(called.get()); + } + finally { + configManager.stop(); + } } static class TestConfig diff --git a/processing/src/test/java/org/apache/druid/common/config/TestConfigManagerConfig.java b/processing/src/test/java/org/apache/druid/common/config/TestConfigManagerConfig.java new file mode 100644 index 00000000000..69ff82ca2a0 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/common/config/TestConfigManagerConfig.java @@ -0,0 +1,40 @@ +/* + * 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.druid.common.config; + +import org.joda.time.Period; + +public class TestConfigManagerConfig extends ConfigManagerConfig +{ + public Period period = null; + public boolean enableCompareAndSwap = true; + + @Override + public Period getPollDuration() + { + return period == null ? super.getPollDuration() : period; + } + + @Override + public boolean isEnableCompareAndSwap() + { + return enableCompareAndSwap; + } +} diff --git a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java new file mode 100644 index 00000000000..028a9d5cc08 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java @@ -0,0 +1,92 @@ +/* + * 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.druid.metadata; + +import javax.annotation.Nullable; + +public class TestMetadataStorageConnector implements MetadataStorageConnector +{ + @Override + public Void insertOrUpdate(String tableName, String keyColumn, String valueColumn, String key, byte[] value) + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public byte[] lookup(String tableName, String keyColumn, String valueColumn, String key) + { + throw new UnsupportedOperationException(); + } + + @Override + public void createDataSourceTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createPendingSegmentsTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createSegmentTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createRulesTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createConfigTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createTaskTables() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createAuditTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void createSupervisorsTable() + { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteAllRecords(String tableName) + { + throw new UnsupportedOperationException(); + } +} diff --git a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java new file mode 100644 index 00000000000..693b36ce24e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java @@ -0,0 +1,43 @@ +/* + * 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.druid.metadata; + +/** + * A config object for tests, use by overriding the specific getter methods and returning what you want + */ +public class TestMetadataStorageTablesConfig extends MetadataStorageTablesConfig +{ + public TestMetadataStorageTablesConfig() + { + super( + "test", + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + } +}