Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

STORM-4154 - fix: Nimbus downtime during topology deployment #3833

Merged
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
add tests for method
  • Loading branch information
DiogoP98 committed Jan 13, 2025
commit bdc3a1362031de429032acd6706500d72805b5f8
Original file line number Diff line number Diff line change
Expand Up @@ -25,24 +25,75 @@

import org.apache.storm.Config;
import org.apache.storm.DaemonConfig;
import org.apache.storm.blobstore.BlobStore;
import org.apache.storm.blobstore.KeySequenceNumber;
import org.apache.storm.blobstore.LocalFsBlobStore;
import org.apache.storm.cluster.IStormClusterState;
import org.apache.storm.generated.InvalidTopologyException;
import org.apache.storm.generated.KeyNotFoundException;
import org.apache.storm.generated.StormTopology;
import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.nimbus.ILeaderElector;
import org.apache.storm.nimbus.NimbusInfo;
import org.apache.storm.scheduler.INimbus;
import org.apache.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy;
import org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy;
import org.apache.storm.scheduler.resource.strategies.scheduling.GenericResourceAwareStrategyOld;
import org.apache.storm.scheduler.resource.strategies.scheduling.RoundRobinResourceAwareStrategy;
import org.apache.storm.security.auth.IGroupMappingServiceProvider;
import org.apache.storm.testing.TestWordSpout;
import org.apache.storm.thrift.TException;
import org.apache.storm.topology.TopologyBuilder;
import org.apache.storm.utils.ServerUtils;
import org.apache.storm.utils.Time;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.Mock;
import org.mockito.MockedConstruction;
import org.mockito.MockitoAnnotations;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.mockConstruction;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;

class NimbusTest {
private static final String BLOB_FILE_KEY = "file-key";

@Mock
private StormMetricsRegistry metricRegistry;
@Mock
private INimbus iNimbus;
@Mock
private IStormClusterState stormClusterState;
@Mock
private NimbusInfo nimbusInfo;
@Mock
private LocalFsBlobStore localBlobStore;
@Mock
private ILeaderElector leaderElector;
@Mock
private IGroupMappingServiceProvider groupMapper;

private Nimbus nimbus;

@BeforeEach
public void setUp() throws Exception {
MockitoAnnotations.openMocks(this).close();

Map<String, Object> conf = Map.of(DaemonConfig.NIMBUS_MONITOR_FREQ_SECS, 10);
nimbus = new Nimbus(conf, iNimbus, stormClusterState, nimbusInfo, localBlobStore, leaderElector, groupMapper, metricRegistry);
}

public class NimbusTest {
@Test
public void testMemoryLoadLargerThanMaxHeapSize() {
// Topology will not be able to be successfully scheduled: Config TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB=128.0 < 129.0,
Expand Down Expand Up @@ -112,4 +163,41 @@ public void validateNoTopoConfOverrides() {
Map<String, Object> normalized = Nimbus.normalizeConf(conf, topoConf, topology);
assertNull(normalized.get(Config.STORM_WORKERS_ARTIFACTS_DIR));
}

@Test
void testCreateStateInZookeeper() throws TException {
nimbus.createStateInZookeeper(BLOB_FILE_KEY);

verify(stormClusterState).setupBlob(eq(BLOB_FILE_KEY), eq(nimbusInfo), any());
}

@Test
void testCreateStateInZookeeperWithoutLocalFsBlobStoreInstanceShouldNotCreate() throws Exception {
BlobStore blobStore = mock(BlobStore.class);
Map<String, Object> conf = Map.of(DaemonConfig.NIMBUS_MONITOR_FREQ_SECS, 10);
nimbus = new Nimbus(conf, iNimbus, stormClusterState, nimbusInfo, blobStore, leaderElector, groupMapper, metricRegistry);

nimbus.createStateInZookeeper(BLOB_FILE_KEY);

verify(stormClusterState, never()).setupBlob(eq(BLOB_FILE_KEY), eq(nimbusInfo), any());
}

@Test
void testCreateStateInZookeeperWhenFailToSetupBlobWithRuntimeExceptionThrowsRuntimeException() {
doThrow(new RuntimeException("Failed to setup blob")).when(stormClusterState).setupBlob(eq(BLOB_FILE_KEY), eq(nimbusInfo), any());

assertThrows(RuntimeException.class, () -> nimbus.createStateInZookeeper(BLOB_FILE_KEY));
verify(stormClusterState).setupBlob(eq(BLOB_FILE_KEY), eq(nimbusInfo), any());
}

@Test
void testCreateStateInZookeeperWhenKeyNotFoundHandlesException() throws Exception {
try (MockedConstruction<KeySequenceNumber> keySequenceNumber = mockConstruction(KeySequenceNumber.class, (mock, context) ->
when(mock.getKeySequenceNumber(any())).thenThrow(new KeyNotFoundException("Failed to setup blob")))) {
nimbus.createStateInZookeeper(BLOB_FILE_KEY);

verify(keySequenceNumber.constructed().get(0)).getKeySequenceNumber(any());
verify(stormClusterState, never()).setupBlob(eq(BLOB_FILE_KEY), eq(nimbusInfo), any());
}
}
}
Loading