mirror of https://github.com/apache/druid.git
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
This commit is contained in:
parent
0335aaa279
commit
fd20bbd30e
|
@ -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 <T> void doAudit(String key, String type, AuditInfo auditInfo, T payload, ConfigSerde<T> configSerde)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doAudit(AuditEntry auditEntry, Handle handler)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AuditEntry> fetchAuditHistory(String key, String type, Interval interval)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AuditEntry> fetchAuditHistory(String type, Interval interval)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AuditEntry> fetchAuditHistory(String key, String type, int limit)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AuditEntry> fetchAuditHistory(String type, int limit)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int removeAuditLogsOlderThan(long timestamp)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
|
@ -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<TestConfig> 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<List<MetadataCASUpdate>> 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<MetadataCASUpdate> 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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue