Pārlūkot izejas kodu

SONAR-9802 replace Hazelcast clients by members

Web Server and Compute Engine processes must use
plain HZ members but not clients so that they can
be involved in distributed computing (contribute to
api/system/info response)
tags/6.6-RC1
Simon Brandhof pirms 6 gadiem
vecāks
revīzija
e4c401f8bc
67 mainītis faili ar 1443 papildinājumiem un 2055 dzēšanām
  1. 11
    11
      server/sonar-ce/src/main/java/org/sonar/ce/CeDistributedInformationImpl.java
  2. 6
    4
      server/sonar-ce/src/main/java/org/sonar/ce/container/ComputeEngineContainerImpl.java
  3. 7
    7
      server/sonar-ce/src/test/java/org/sonar/ce/CeDistributedInformationImplTest.java
  4. 19
    12
      server/sonar-ce/src/test/java/org/sonar/ce/container/ComputeEngineContainerImplTest.java
  5. 0
    22
      server/sonar-ce/src/test/java/org/sonar/ce/container/HazelcastTestHelper.java
  6. 1
    3
      server/sonar-main/src/main/java/org/sonar/application/AppLogging.java
  7. 1
    1
      server/sonar-main/src/main/java/org/sonar/application/AppReloaderImpl.java
  8. 25
    1
      server/sonar-main/src/main/java/org/sonar/application/AppStateFactory.java
  9. 4
    4
      server/sonar-main/src/main/java/org/sonar/application/SchedulerImpl.java
  10. 2
    2
      server/sonar-main/src/main/java/org/sonar/application/cluster/ClusterAppState.java
  11. 207
    52
      server/sonar-main/src/main/java/org/sonar/application/cluster/ClusterAppStateImpl.java
  12. 0
    152
      server/sonar-main/src/main/java/org/sonar/application/cluster/ClusterProperties.java
  13. 0
    427
      server/sonar-main/src/main/java/org/sonar/application/cluster/HazelcastCluster.java
  14. 2
    1
      server/sonar-main/src/main/java/org/sonar/application/cluster/health/DelegateHealthStateRefresherExecutorService.java
  15. 1
    1
      server/sonar-main/src/main/java/org/sonar/application/cluster/health/HealthStateSharing.java
  16. 10
    6
      server/sonar-main/src/main/java/org/sonar/application/cluster/health/HealthStateSharingImpl.java
  17. 5
    4
      server/sonar-main/src/main/java/org/sonar/application/cluster/health/SearchNodeHealthProvider.java
  18. 23
    0
      server/sonar-main/src/main/java/org/sonar/application/cluster/health/package-info.java
  19. 3
    1
      server/sonar-main/src/main/java/org/sonar/application/config/AppSettingsLoaderImpl.java
  20. 60
    90
      server/sonar-main/src/main/java/org/sonar/application/config/ClusterSettings.java
  21. 4
    4
      server/sonar-main/src/main/java/org/sonar/application/es/EsSettings.java
  22. 1
    14
      server/sonar-main/src/test/java/org/sonar/application/AppLoggingTest.java
  23. 1
    1
      server/sonar-main/src/test/java/org/sonar/application/AppReloaderImplTest.java
  24. 16
    4
      server/sonar-main/src/test/java/org/sonar/application/AppStateFactoryTest.java
  25. 8
    8
      server/sonar-main/src/test/java/org/sonar/application/SchedulerImplTest.java
  26. 6
    6
      server/sonar-main/src/test/java/org/sonar/application/TestClusterAppState.java
  27. 23
    76
      server/sonar-main/src/test/java/org/sonar/application/cluster/ClusterAppStateImplTest.java
  28. 0
    128
      server/sonar-main/src/test/java/org/sonar/application/cluster/ClusterPropertiesTest.java
  29. 0
    353
      server/sonar-main/src/test/java/org/sonar/application/cluster/HazelcastClusterTest.java
  30. 0
    84
      server/sonar-main/src/test/java/org/sonar/application/cluster/HazelcastClusterTestHelper.java
  31. 48
    0
      server/sonar-main/src/test/java/org/sonar/application/cluster/HazelcastTesting.java
  32. 1
    1
      server/sonar-main/src/test/java/org/sonar/application/cluster/health/DelegateHealthStateRefresherExecutorServiceTest.java
  33. 10
    9
      server/sonar-main/src/test/java/org/sonar/application/cluster/health/SearchNodeHealthProviderTest.java
  34. 65
    175
      server/sonar-main/src/test/java/org/sonar/application/config/ClusterSettingsLoopbackTest.java
  35. 54
    93
      server/sonar-main/src/test/java/org/sonar/application/config/ClusterSettingsTest.java
  36. 10
    11
      server/sonar-main/src/test/java/org/sonar/application/es/EsSettingsTest.java
  37. 4
    0
      server/sonar-process/pom.xml
  38. 33
    0
      server/sonar-process/src/main/java/org/sonar/process/NetworkUtils.java
  39. 41
    6
      server/sonar-process/src/main/java/org/sonar/process/NetworkUtilsImpl.java
  40. 28
    15
      server/sonar-process/src/main/java/org/sonar/process/ProcessProperties.java
  41. 0
    56
      server/sonar-process/src/main/java/org/sonar/process/cluster/ClusterProperties.java
  42. 1
    2
      server/sonar-process/src/main/java/org/sonar/process/cluster/NodeType.java
  43. 1
    1
      server/sonar-process/src/main/java/org/sonar/process/cluster/health/NodeHealthProvider.java
  44. 10
    10
      server/sonar-process/src/main/java/org/sonar/process/cluster/health/SharedHealthStateImpl.java
  45. 44
    16
      server/sonar-process/src/main/java/org/sonar/process/cluster/hz/HazelcastMember.java
  46. 142
    0
      server/sonar-process/src/main/java/org/sonar/process/cluster/hz/HazelcastMemberBuilder.java
  47. 100
    0
      server/sonar-process/src/main/java/org/sonar/process/cluster/hz/HazelcastMemberImpl.java
  48. 6
    24
      server/sonar-process/src/main/java/org/sonar/process/cluster/hz/HazelcastObjects.java
  49. 23
    0
      server/sonar-process/src/main/java/org/sonar/process/cluster/hz/package-info.java
  50. 100
    0
      server/sonar-process/src/test/java/org/sonar/process/NetworkUtilsImplTest.java
  51. 80
    0
      server/sonar-process/src/test/java/org/sonar/process/cluster/hz/HazelcastMemberBuilderTest.java
  52. 149
    0
      server/sonar-server/src/main/java/org/sonar/server/cluster/StartableHazelcastMember.java
  53. 1
    1
      server/sonar-server/src/main/java/org/sonar/server/cluster/package-info.java
  54. 4
    4
      server/sonar-server/src/main/java/org/sonar/server/es/EsClientProvider.java
  55. 1
    1
      server/sonar-server/src/main/java/org/sonar/server/es/NewIndex.java
  56. 3
    3
      server/sonar-server/src/main/java/org/sonar/server/health/NodeHealthProviderImpl.java
  57. 0
    127
      server/sonar-server/src/main/java/org/sonar/server/hz/HazelcastLocalClient.java
  58. 1
    3
      server/sonar-server/src/main/java/org/sonar/server/platform/WebServer.java
  59. 3
    3
      server/sonar-server/src/main/java/org/sonar/server/platform/WebServerImpl.java
  60. 2
    2
      server/sonar-server/src/main/java/org/sonar/server/platform/platformlevel/PlatformLevel4.java
  61. 4
    4
      server/sonar-server/src/test/java/org/sonar/server/es/EsClientProviderTest.java
  62. 1
    1
      server/sonar-server/src/test/java/org/sonar/server/es/NewIndexTest.java
  63. 2
    2
      server/sonar-server/src/test/java/org/sonar/server/health/NodeHealthModuleTest.java
  64. 3
    3
      server/sonar-server/src/test/java/org/sonar/server/health/NodeHealthProviderImplTest.java
  65. 1
    1
      server/sonar-server/src/test/java/org/sonar/server/rule/index/RuleIndexDefinitionTest.java
  66. 2
    2
      sonar-application/src/main/java/org/sonar/application/App.java
  67. 19
    0
      tests/src/test/java/org/sonarqube/tests/cluster/ClusterTest.java

+ 11
- 11
server/sonar-ce/src/main/java/org/sonar/ce/CeDistributedInformationImpl.java Parādīt failu

@@ -25,27 +25,27 @@ import java.util.Set;
import java.util.concurrent.locks.Lock;
import org.picocontainer.Startable;
import org.sonar.ce.taskprocessor.CeWorkerFactory;
import org.sonar.process.cluster.ClusterObjectKeys;
import org.sonar.process.cluster.HazelcastClient;
import org.sonar.process.cluster.hz.HazelcastObjects;
import org.sonar.process.cluster.hz.HazelcastMember;

import static org.sonar.core.util.stream.MoreCollectors.toSet;
import static org.sonar.process.cluster.ClusterObjectKeys.WORKER_UUIDS;
import static org.sonar.process.cluster.hz.HazelcastObjects.WORKER_UUIDS;

/**
* Provide the set of worker's UUID in a clustered SonarQube instance
*/
public class CeDistributedInformationImpl implements CeDistributedInformation, Startable {
private final HazelcastClient hazelcastClient;
private final HazelcastMember hazelcastMember;
private final CeWorkerFactory ceCeWorkerFactory;

public CeDistributedInformationImpl(HazelcastClient hazelcastClient, CeWorkerFactory ceCeWorkerFactory) {
this.hazelcastClient = hazelcastClient;
public CeDistributedInformationImpl(HazelcastMember hazelcastMember, CeWorkerFactory ceCeWorkerFactory) {
this.hazelcastMember = hazelcastMember;
this.ceCeWorkerFactory = ceCeWorkerFactory;
}

@Override
public Set<String> getWorkerUUIDs() {
Set<String> connectedWorkerUUIDs = hazelcastClient.getMemberUuids();
Set<String> connectedWorkerUUIDs = hazelcastMember.getMemberUuids();

return getClusteredWorkerUUIDs().entrySet().stream()
.filter(e -> connectedWorkerUUIDs.contains(e.getKey()))
@@ -56,12 +56,12 @@ public class CeDistributedInformationImpl implements CeDistributedInformation, S

@Override
public void broadcastWorkerUUIDs() {
getClusteredWorkerUUIDs().put(hazelcastClient.getUUID(), ceCeWorkerFactory.getWorkerUUIDs());
getClusteredWorkerUUIDs().put(hazelcastMember.getUuid(), ceCeWorkerFactory.getWorkerUUIDs());
}

@Override
public Lock acquireCleanJobLock() {
return hazelcastClient.getLock(ClusterObjectKeys.CE_CLEANING_JOB_LOCK);
return hazelcastMember.getLock(HazelcastObjects.CE_CLEANING_JOB_LOCK);
}

@Override
@@ -72,10 +72,10 @@ public class CeDistributedInformationImpl implements CeDistributedInformation, S
@Override
public void stop() {
// Removing the worker UUIDs
getClusteredWorkerUUIDs().remove(hazelcastClient.getUUID());
getClusteredWorkerUUIDs().remove(hazelcastMember.getUuid());
}

private Map<String, Set<String>> getClusteredWorkerUUIDs() {
return hazelcastClient.getReplicatedMap(WORKER_UUIDS);
return hazelcastMember.getReplicatedMap(WORKER_UUIDS);
}
}

+ 6
- 4
server/sonar-ce/src/main/java/org/sonar/ce/container/ComputeEngineContainerImpl.java Parādīt failu

@@ -73,16 +73,17 @@ import org.sonar.db.DatabaseChecker;
import org.sonar.db.DbClient;
import org.sonar.db.DefaultDatabase;
import org.sonar.db.purge.PurgeProfiler;
import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessProperties;
import org.sonar.process.Props;
import org.sonar.process.cluster.ClusterProperties;
import org.sonar.process.logging.LogbackHelper;
import org.sonar.server.cluster.StartableHazelcastMember;
import org.sonar.server.component.ComponentFinder;
import org.sonar.server.component.index.ComponentIndexer;
import org.sonar.server.computation.task.projectanalysis.ProjectAnalysisTaskModule;
import org.sonar.server.debt.DebtModelPluginRepository;
import org.sonar.server.debt.DebtRulesXMLImporter;
import org.sonar.server.event.NewAlerts;
import org.sonar.server.hz.HazelcastLocalClient;
import org.sonar.server.issue.IssueFieldsSetter;
import org.sonar.server.issue.index.IssueIndex;
import org.sonar.server.issue.index.IssueIndexer;
@@ -234,6 +235,7 @@ public class ComputeEngineContainerImpl implements ComputeEngineContainer {
SonarRuntimeImpl.forSonarQube(ApiVersion.load(System2.INSTANCE), SonarQubeSide.COMPUTE_ENGINE),
CeProcessLogging.class,
UuidFactoryImpl.INSTANCE,
NetworkUtils.INSTANCE,
WebServerImpl.class,
LogbackHelper.class,
DefaultDatabase.class,
@@ -417,9 +419,9 @@ public class ComputeEngineContainerImpl implements ComputeEngineContainer {
// cleaning
CeCleaningModule.class);

if (props.valueAsBoolean(ClusterProperties.CLUSTER_ENABLED)) {
if (props.valueAsBoolean(ProcessProperties.CLUSTER_ENABLED)) {
container.add(
HazelcastLocalClient.class,
StartableHazelcastMember.class,
CeDistributedInformationImpl.class);
} else {
container.add(StandaloneCeDistributedInformation.class);

+ 7
- 7
server/sonar-ce/src/test/java/org/sonar/ce/CeDistributedInformationImplTest.java Parādīt failu

@@ -27,13 +27,13 @@ import java.util.Map;
import java.util.Set;
import org.junit.Test;
import org.sonar.ce.taskprocessor.CeWorkerFactory;
import org.sonar.server.hz.HazelcastLocalClient;
import org.sonar.server.cluster.StartableHazelcastMember;

import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.data.MapEntry.entry;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import static org.sonar.process.cluster.ClusterObjectKeys.WORKER_UUIDS;
import static org.sonar.process.cluster.hz.HazelcastObjects.WORKER_UUIDS;

public class CeDistributedInformationImplTest {
private String clientUUID1 = "1";
@@ -45,11 +45,11 @@ public class CeDistributedInformationImplTest {
clientUUID3, ImmutableSet.of("4", "5", "6")
);

private HazelcastLocalClient hzClientWrapper = mock(HazelcastLocalClient.class);
private StartableHazelcastMember hzClientWrapper = mock(StartableHazelcastMember.class);

@Test
public void getWorkerUUIDs_returns_union_of_workers_uuids_of_local_and_cluster_worker_uuids() {
when(hzClientWrapper.getUUID()).thenReturn(clientUUID1);
when(hzClientWrapper.getUuid()).thenReturn(clientUUID1);
when(hzClientWrapper.getMemberUuids()).thenReturn(ImmutableSet.of(clientUUID1, clientUUID2, clientUUID3));
when(hzClientWrapper.getReplicatedMap(WORKER_UUIDS)).thenReturn(workerMap);

@@ -59,7 +59,7 @@ public class CeDistributedInformationImplTest {

@Test
public void getWorkerUUIDs_must_filter_absent_client() {
when(hzClientWrapper.getUUID()).thenReturn(clientUUID1);
when(hzClientWrapper.getUuid()).thenReturn(clientUUID1);
when(hzClientWrapper.getMemberUuids()).thenReturn(ImmutableSet.of(clientUUID1, clientUUID2));
when(hzClientWrapper.getReplicatedMap(WORKER_UUIDS)).thenReturn(workerMap);

@@ -74,7 +74,7 @@ public class CeDistributedInformationImplTest {
connectedClients.add(clientUUID1);
connectedClients.add(clientUUID2);

when(hzClientWrapper.getUUID()).thenReturn(clientUUID1);
when(hzClientWrapper.getUuid()).thenReturn(clientUUID1);
when(hzClientWrapper.getMemberUuids()).thenReturn(connectedClients);
when(hzClientWrapper.getReplicatedMap(WORKER_UUIDS)).thenReturn(modifiableWorkerMap);

@@ -101,7 +101,7 @@ public class CeDistributedInformationImplTest {
Map modifiableWorkerMap = new HashMap();
modifiableWorkerMap.putAll(workerMap);

when(hzClientWrapper.getUUID()).thenReturn(clientUUID1);
when(hzClientWrapper.getUuid()).thenReturn(clientUUID1);
when(hzClientWrapper.getMemberUuids()).thenReturn(connectedClients);
when(hzClientWrapper.getReplicatedMap(WORKER_UUIDS)).thenReturn(modifiableWorkerMap);


+ 19
- 12
server/sonar-ce/src/test/java/org/sonar/ce/container/ComputeEngineContainerImplTest.java Parādīt failu

@@ -19,14 +19,15 @@
*/
package org.sonar.ce.container;

import com.hazelcast.core.HazelcastInstance;
import java.io.File;
import java.io.IOException;
import java.net.InetAddress;
import java.util.Date;
import java.util.Optional;
import java.util.Properties;
import java.util.stream.Collectors;
import org.apache.commons.dbcp.BasicDataSource;
import org.hamcrest.CoreMatchers;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
@@ -45,19 +46,22 @@ import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessId;
import org.sonar.process.ProcessProperties;
import org.sonar.process.Props;
import org.sonar.server.hz.HazelcastLocalClient;
import org.sonar.server.cluster.StartableHazelcastMember;

import static java.lang.String.valueOf;
import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.Assume.assumeThat;
import static org.mockito.Mockito.mock;
import static org.sonar.process.ProcessEntryPoint.PROPERTY_PROCESS_INDEX;
import static org.sonar.process.ProcessEntryPoint.PROPERTY_PROCESS_KEY;
import static org.sonar.process.ProcessEntryPoint.PROPERTY_SHARED_PATH;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_PORT;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_TYPE;
import static org.sonar.process.ProcessProperties.PATH_DATA;
import static org.sonar.process.ProcessProperties.PATH_HOME;
import static org.sonar.process.ProcessProperties.PATH_TEMP;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_LOCALENDPOINT;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_TYPE;

public class ComputeEngineContainerImplTest {
private static final int CONTAINER_ITSELF = 1;
@@ -83,13 +87,16 @@ public class ComputeEngineContainerImplTest {

@Test
public void real_start_with_cluster() throws IOException {
int port = NetworkUtils.INSTANCE.getNextAvailablePort(InetAddress.getLoopbackAddress());
HazelcastInstance hzInstance = HazelcastTestHelper.createHazelcastCluster(NetworkUtils.INSTANCE.getHostname(), port);
Optional<InetAddress> localhost = NetworkUtils.INSTANCE.getLocalNonLoopbackIpv4Address();
// test is ignored if offline
assumeThat(localhost.isPresent(), CoreMatchers.is(true));

Properties properties = getProperties();
properties.setProperty(CLUSTER_NODE_TYPE, "application");
properties.setProperty(PROPERTY_PROCESS_KEY, ProcessId.COMPUTE_ENGINE.getKey());
properties.setProperty(CLUSTER_ENABLED, "true");
properties.setProperty(CLUSTER_LOCALENDPOINT, String.format("%s:%d", hzInstance.getCluster().getLocalMember().getAddress().getHost(), port));
properties.setProperty(CLUSTER_NODE_TYPE, "application");
properties.setProperty(CLUSTER_NODE_HOST, localhost.get().getHostAddress());
properties.setProperty(CLUSTER_NODE_PORT, "" + NetworkUtils.INSTANCE.getNextAvailablePort(localhost.get()));

// required persisted properties
insertProperty(CoreProperties.SERVER_ID, "a_startup_id");
@@ -102,7 +109,7 @@ public class ComputeEngineContainerImplTest {
assertThat(
picoContainer.getComponentAdapters().stream()
.map(ComponentAdapter::getComponentImplementation)
.collect(Collectors.toList())).contains((Class) HazelcastLocalClient.class,
.collect(Collectors.toList())).contains((Class) StartableHazelcastMember.class,
(Class) CeDistributedInformationImpl.class);
underTest.stop();
}
@@ -143,7 +150,7 @@ public class ComputeEngineContainerImplTest {
);
assertThat(picoContainer.getParent().getParent().getParent().getComponentAdapters()).hasSize(
COMPONENTS_IN_LEVEL_1_AT_CONSTRUCTION
+ 25 // level 1
+ 26 // level 1
+ 49 // content of DaoModule
+ 3 // content of EsSearchModule
+ 64 // content of CorePropertyDefinitions
@@ -152,7 +159,7 @@ public class ComputeEngineContainerImplTest {
assertThat(
picoContainer.getComponentAdapters().stream()
.map(ComponentAdapter::getComponentImplementation)
.collect(Collectors.toList())).doesNotContain((Class) HazelcastLocalClient.class,
.collect(Collectors.toList())).doesNotContain((Class) StartableHazelcastMember.class,
(Class) CeDistributedInformationImpl.class).contains(
(Class) StandaloneCeDistributedInformation.class);
assertThat(picoContainer.getParent().getParent().getParent().getParent()).isNull();

+ 0
- 22
server/sonar-ce/src/test/java/org/sonar/ce/container/HazelcastTestHelper.java Parādīt failu

@@ -23,12 +23,9 @@ package org.sonar.ce.container;
import com.hazelcast.config.Config;
import com.hazelcast.config.JoinConfig;
import com.hazelcast.config.NetworkConfig;
import com.hazelcast.core.Client;
import com.hazelcast.core.ClientListener;
import com.hazelcast.core.Hazelcast;
import com.hazelcast.core.HazelcastInstance;
import java.net.InetAddress;
import org.sonar.process.cluster.ClusterObjectKeys;

/**
* TODO move outside main sources
@@ -75,25 +72,6 @@ public class HazelcastTestHelper {
// We are not using the partition group of Hazelcast, so disabling it
hzConfig.getPartitionGroupConfig().setEnabled(false);
HazelcastInstance hzInstance = Hazelcast.newHazelcastInstance(hzConfig);
hzInstance.getClientService().addClientListener(new ConnectedClientListener(hzInstance));
return hzInstance;
}

private static class ConnectedClientListener implements ClientListener {
private final HazelcastInstance hzInstance;

private ConnectedClientListener(HazelcastInstance hzInstance) {
this.hzInstance = hzInstance;
}

@Override
public void clientConnected(Client client) {
hzInstance.getSet(ClusterObjectKeys.LOCAL_MEMBER_UUIDS).add(client.getUuid());
}

@Override
public void clientDisconnected(Client client) {
hzInstance.getSet(ClusterObjectKeys.LOCAL_MEMBER_UUIDS).remove(client.getUuid());
}
}
}

+ 1
- 3
server/sonar-main/src/main/java/org/sonar/application/AppLogging.java Parādīt failu

@@ -34,7 +34,6 @@ import org.sonar.process.logging.RootLoggerConfig;

import static org.slf4j.Logger.ROOT_LOGGER_NAME;
import static org.sonar.application.process.StreamGobbler.LOGGER_GOBBLER;
import static org.sonar.process.cluster.ClusterProperties.HAZELCAST_LOG_LEVEL;
import static org.sonar.process.logging.RootLoggerConfig.newRootLoggerConfigBuilder;

/**
@@ -140,8 +139,7 @@ public class AppLogging {
LogLevelConfig.newBuilder(helper.getRootLoggerName())
.rootLevelFor(ProcessId.APP)
.immutableLevel("com.hazelcast",
Level.toLevel(
appSettings.getProps().nonNullValue(HAZELCAST_LOG_LEVEL)))
Level.toLevel("WARN"))
.build(),
appSettings.getProps());


+ 1
- 1
server/sonar-main/src/main/java/org/sonar/application/AppReloaderImpl.java Parādīt failu

@@ -28,7 +28,7 @@ import org.sonar.process.MessageException;
import org.sonar.process.Props;

import static java.lang.String.format;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.PATH_DATA;
import static org.sonar.process.ProcessProperties.PATH_LOGS;
import static org.sonar.process.ProcessProperties.PATH_TEMP;

+ 25
- 1
server/sonar-main/src/main/java/org/sonar/application/AppStateFactory.java Parādīt failu

@@ -22,6 +22,14 @@ package org.sonar.application;
import org.sonar.application.cluster.ClusterAppStateImpl;
import org.sonar.application.config.AppSettings;
import org.sonar.application.config.ClusterSettings;
import org.sonar.process.ProcessId;
import org.sonar.process.ProcessProperties;
import org.sonar.process.Props;
import org.sonar.process.cluster.NodeType;
import org.sonar.process.cluster.hz.HazelcastMember;
import org.sonar.process.cluster.hz.HazelcastMemberBuilder;

import static java.util.Arrays.asList;

public class AppStateFactory {

@@ -32,6 +40,22 @@ public class AppStateFactory {
}

public AppState create() {
return ClusterSettings.isClusterEnabled(settings) ? new ClusterAppStateImpl(settings) : new AppStateImpl();
if (ClusterSettings.isClusterEnabled(settings)) {
HazelcastMember hzMember = createHzMember(settings.getProps());
return new ClusterAppStateImpl(hzMember);
}
return new AppStateImpl();
}

private static HazelcastMember createHzMember(Props props) {
HazelcastMemberBuilder builder = new HazelcastMemberBuilder()
.setClusterName("sonarqube")
.setNetworkInterface(props.nonNullValue(ProcessProperties.CLUSTER_NODE_HOST))
.setMembers(asList(props.nonNullValue(ProcessProperties.CLUSTER_HOSTS).split(",")))
.setNodeType(NodeType.parse(props.nonNullValue(ProcessProperties.CLUSTER_NODE_TYPE)))
.setNodeName(props.nonNullValue(ProcessProperties.CLUSTER_NODE_NAME))
.setPort(Integer.parseInt(props.nonNullValue(ProcessProperties.CLUSTER_NODE_PORT)))
.setProcessId(ProcessId.APP);
return builder.build();
}
}

+ 4
- 4
server/sonar-main/src/main/java/org/sonar/application/SchedulerImpl.java Parādīt failu

@@ -29,7 +29,7 @@ import java.util.function.Supplier;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.sonar.application.cluster.ClusterAppState;
import org.sonar.application.cluster.SearchNodeHealthProvider;
import org.sonar.application.cluster.health.SearchNodeHealthProvider;
import org.sonar.application.command.CommandFactory;
import org.sonar.application.command.EsCommand;
import org.sonar.application.command.JavaCommand;
@@ -43,8 +43,8 @@ import org.sonar.application.process.ProcessMonitor;
import org.sonar.application.process.SQProcess;
import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessId;
import org.sonar.process.cluster.health.HealthStateSharing;
import org.sonar.process.cluster.health.HealthStateSharingImpl;
import org.sonar.application.cluster.health.HealthStateSharing;
import org.sonar.application.cluster.health.HealthStateSharingImpl;

public class SchedulerImpl implements Scheduler, ProcessEventListener, ProcessLifecycleListener, AppStateListener {

@@ -150,7 +150,7 @@ public class SchedulerImpl implements Scheduler, ProcessEventListener, ProcessLi
&& ClusterSettings.isLocalElasticsearchEnabled(settings)) {
ClusterAppState clusterAppState = (ClusterAppState) appState;
this.healthStateSharing = new HealthStateSharingImpl(
clusterAppState.getHazelcastClient(),
clusterAppState.getHazelcastMember(),
new SearchNodeHealthProvider(settings.getProps(), clusterAppState, NetworkUtils.INSTANCE));
this.healthStateSharing.start();
}

+ 2
- 2
server/sonar-main/src/main/java/org/sonar/application/cluster/ClusterAppState.java Parādīt failu

@@ -20,8 +20,8 @@
package org.sonar.application.cluster;

import org.sonar.application.AppState;
import org.sonar.process.cluster.HazelcastClient;
import org.sonar.process.cluster.hz.HazelcastMember;

public interface ClusterAppState extends AppState {
HazelcastClient getHazelcastClient();
HazelcastMember getHazelcastMember();
}

+ 207
- 52
server/sonar-main/src/main/java/org/sonar/application/cluster/ClusterAppStateImpl.java Parādīt failu

@@ -20,67 +20,103 @@

package org.sonar.application.cluster;

import com.hazelcast.core.EntryEvent;
import com.hazelcast.core.EntryListener;
import com.hazelcast.core.HazelcastInstanceNotActiveException;
import com.hazelcast.core.IAtomicReference;
import com.hazelcast.core.MapEvent;
import com.hazelcast.core.Member;
import com.hazelcast.core.MemberAttributeEvent;
import com.hazelcast.core.MembershipEvent;
import com.hazelcast.core.MembershipListener;
import com.hazelcast.core.ReplicatedMap;
import java.util.ArrayList;
import java.util.EnumMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import javax.annotation.Nonnull;
import java.util.concurrent.locks.Lock;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.sonar.application.AppStateListener;
import org.sonar.application.config.AppSettings;
import org.sonar.process.MessageException;
import org.sonar.process.ProcessId;
import org.sonar.process.cluster.HazelcastClient;
import org.sonar.process.cluster.NodeType;
import org.sonar.process.cluster.hz.HazelcastMember;

import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_LOCALENDPOINT;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_MEMBERUUID;
import static java.lang.String.format;
import static org.sonar.process.cluster.hz.HazelcastObjects.CLUSTER_NAME;
import static org.sonar.process.cluster.hz.HazelcastObjects.LEADER;
import static org.sonar.process.cluster.hz.HazelcastObjects.OPERATIONAL_PROCESSES;
import static org.sonar.process.cluster.hz.HazelcastObjects.SONARQUBE_VERSION;

public class ClusterAppStateImpl implements ClusterAppState {
private static Logger LOGGER = LoggerFactory.getLogger(ClusterAppStateImpl.class);

private final Map<ProcessId, Boolean> localProcesses = new EnumMap<>(ProcessId.class);
private final HazelcastCluster hazelcastCluster;
private static final Logger LOGGER = LoggerFactory.getLogger(ClusterAppStateImpl.class);

public ClusterAppStateImpl(AppSettings appSettings) {
if (!appSettings.getProps().valueAsBoolean(CLUSTER_ENABLED)) {
throw new IllegalStateException("Cluster is not enabled on this instance");
}
private final HazelcastMember hzMember;
private final List<AppStateListener> listeners = new ArrayList<>();
private final Map<ProcessId, Boolean> operationalLocalProcesses = new EnumMap<>(ProcessId.class);
private final ReplicatedMap<ClusterProcess, Boolean> operationalProcesses;
private final String operationalProcessListenerUUID;
private final String nodeDisconnectedListenerUUID;

ClusterProperties clusterProperties = new ClusterProperties(appSettings);
clusterProperties.validate();
public ClusterAppStateImpl(HazelcastMember hzMember) {
this.hzMember = hzMember;

hazelcastCluster = HazelcastCluster.create(clusterProperties);
// Add the local endpoint to be used by processes
appSettings.getProps().set(CLUSTER_LOCALENDPOINT, hazelcastCluster.getLocalEndPoint());
appSettings.getProps().set(CLUSTER_MEMBERUUID, hazelcastCluster.getLocalUUID());
// Get or create the replicated map
operationalProcesses = (ReplicatedMap) hzMember.getReplicatedMap(OPERATIONAL_PROCESSES);
operationalProcessListenerUUID = operationalProcesses.addEntryListener(new OperationalProcessListener());
nodeDisconnectedListenerUUID = hzMember.getCluster().addMembershipListener(new NodeDisconnectedListener());
}

String members = hazelcastCluster.getMembers().stream().collect(Collectors.joining(","));
LOGGER.info("Joined a SonarQube cluster that contains the following hosts : [{}]", members);
@Override
public HazelcastMember getHazelcastMember() {
return hzMember;
}

@Override
public void addListener(@Nonnull AppStateListener listener) {
hazelcastCluster.addListener(listener);
public void addListener(AppStateListener listener) {
listeners.add(listener);
}

@Override
public boolean isOperational(@Nonnull ProcessId processId, boolean local) {
public boolean isOperational(ProcessId processId, boolean local) {
if (local) {
return localProcesses.computeIfAbsent(processId, p -> false);
return operationalLocalProcesses.computeIfAbsent(processId, p -> false);
}
return hazelcastCluster.isOperational(processId);
for (Map.Entry<ClusterProcess, Boolean> entry : operationalProcesses.entrySet()) {
if (entry.getKey().getProcessId().equals(processId) && entry.getValue()) {
return true;
}
}
return false;
}

@Override
public void setOperational(@Nonnull ProcessId processId) {
localProcesses.put(processId, true);
hazelcastCluster.setOperational(processId);
public void setOperational(ProcessId processId) {
operationalLocalProcesses.put(processId, true);
operationalProcesses.put(new ClusterProcess(hzMember.getUuid(), processId), Boolean.TRUE);
}

@Override
public boolean tryToLockWebLeader() {
return hazelcastCluster.tryToLockWebLeader();
IAtomicReference<String> leader = hzMember.getAtomicReference(LEADER);
if (leader.get() == null) {
Lock lock = hzMember.getLock(LEADER);
lock.lock();
try {
if (leader.get() == null) {
leader.set(hzMember.getUuid());
return true;
}
return false;
} finally {
lock.unlock();
}
} else {
return false;
}
}

@Override
@@ -88,42 +124,161 @@ public class ClusterAppStateImpl implements ClusterAppState {
throw new IllegalStateException("state reset is not supported in cluster mode");
}

@Override
public void close() {
hazelcastCluster.close();
}

@Override
public void registerSonarQubeVersion(String sonarqubeVersion) {
hazelcastCluster.registerSonarQubeVersion(sonarqubeVersion);
IAtomicReference<String> sqVersion = hzMember.getAtomicReference(SONARQUBE_VERSION);
if (sqVersion.get() == null) {
Lock lock = hzMember.getLock(SONARQUBE_VERSION);
lock.lock();
try {
if (sqVersion.get() == null) {
sqVersion.set(sonarqubeVersion);
}
} finally {
lock.unlock();
}
}

String clusterVersion = sqVersion.get();
if (!sqVersion.get().equals(sonarqubeVersion)) {
throw new IllegalStateException(
format("The local version %s is not the same as the cluster %s", sonarqubeVersion, clusterVersion));
}
}

@Override
public void registerClusterName(String clusterName) {
hazelcastCluster.registerClusterName(clusterName);
IAtomicReference<String> property = hzMember.getAtomicReference(CLUSTER_NAME);
if (property.get() == null) {
Lock lock = hzMember.getLock(CLUSTER_NAME);
lock.lock();
try {
if (property.get() == null) {
property.set(clusterName);
}
} finally {
lock.unlock();
}
}

String clusterValue = property.get();
if (!property.get().equals(clusterName)) {
throw new MessageException(
format("This node has a cluster name [%s], which does not match [%s] from the cluster", clusterName, clusterValue));
}
}

@Override
public Optional<String> getLeaderHostName() {
return hazelcastCluster.getLeaderHostName();
}

HazelcastCluster getHazelcastCluster() {
return hazelcastCluster;
String leaderId = (String) hzMember.getAtomicReference(LEADER).get();
if (leaderId != null) {
Optional<Member> leader = hzMember.getCluster().getMembers().stream().filter(m -> m.getUuid().equals(leaderId)).findFirst();
if (leader.isPresent()) {
return Optional.of(
format("%s (%s)", leader.get().getStringAttribute(HazelcastMember.Attribute.HOSTNAME), leader.get().getStringAttribute(HazelcastMember.Attribute.IP_ADDRESSES)));
}
}
return Optional.empty();
}

@Override
public HazelcastClient getHazelcastClient() {
return hazelcastCluster.getHazelcastClient();
public void close() {
if (hzMember != null) {
try {
// Removing listeners
operationalProcesses.removeEntryListener(operationalProcessListenerUUID);
hzMember.getCluster().removeMembershipListener(nodeDisconnectedListenerUUID);

// Removing the operationalProcess from the replicated map
operationalProcesses.keySet().forEach(
clusterNodeProcess -> {
if (clusterNodeProcess.getNodeUuid().equals(hzMember.getUuid())) {
operationalProcesses.remove(clusterNodeProcess);
}
});

// Shutdown Hazelcast properly
hzMember.close();
} catch (HazelcastInstanceNotActiveException e) {
// hazelcastCluster may be already closed by the shutdown hook
LOGGER.debug("Unable to close Hazelcast cluster", e);
}
}
}

/**
* Only used for testing purpose
*
* @param logger
*/
static void setLogger(Logger logger) {
ClusterAppStateImpl.LOGGER = logger;
private class OperationalProcessListener implements EntryListener<ClusterProcess, Boolean> {
@Override
public void entryAdded(EntryEvent<ClusterProcess, Boolean> event) {
if (event.getValue()) {
listeners.forEach(appStateListener -> appStateListener.onAppStateOperational(event.getKey().getProcessId()));
}
}

@Override
public void entryRemoved(EntryEvent<ClusterProcess, Boolean> event) {
// Ignore it
}

@Override
public void entryUpdated(EntryEvent<ClusterProcess, Boolean> event) {
if (event.getValue()) {
listeners.forEach(appStateListener -> appStateListener.onAppStateOperational(event.getKey().getProcessId()));
}
}

@Override
public void entryEvicted(EntryEvent<ClusterProcess, Boolean> event) {
// Ignore it
}

@Override
public void mapCleared(MapEvent event) {
// Ignore it
}

@Override
public void mapEvicted(MapEvent event) {
// Ignore it
}
}

private class NodeDisconnectedListener implements MembershipListener {
@Override
public void memberAdded(MembershipEvent membershipEvent) {
// Nothing to do
}

@Override
public void memberRemoved(MembershipEvent membershipEvent) {
removeOperationalProcess(membershipEvent.getMember().getUuid());
if (membershipEvent.getMembers().stream()
.noneMatch(this::isAppNode)) {
purgeSharedMemoryForAppNodes();
}
}

@Override
public void memberAttributeChanged(MemberAttributeEvent memberAttributeEvent) {
// Nothing to do
}

private boolean isAppNode(Member member) {
return NodeType.APPLICATION.getValue().equals(member.getStringAttribute(HazelcastMember.Attribute.NODE_TYPE));
}

private void removeOperationalProcess(String uuid) {
for (ClusterProcess clusterProcess : operationalProcesses.keySet()) {
if (clusterProcess.getNodeUuid().equals(uuid)) {
LOGGER.debug("Set node process off for [{}:{}] : ", clusterProcess.getNodeUuid(), clusterProcess.getProcessId());
hzMember.getReplicatedMap(OPERATIONAL_PROCESSES).put(clusterProcess, Boolean.FALSE);
}
}
}

private void purgeSharedMemoryForAppNodes() {
LOGGER.info("No more application nodes, clearing cluster information about application nodes.");
hzMember.getAtomicReference(LEADER).clear();
hzMember.getAtomicReference(SONARQUBE_VERSION).clear();
}
}
}

+ 0
- 152
server/sonar-main/src/main/java/org/sonar/application/cluster/ClusterProperties.java Parādīt failu

@@ -1,152 +0,0 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/

package org.sonar.application.cluster;

import java.net.InetAddress;
import java.net.NetworkInterface;
import java.net.SocketException;
import java.util.ArrayList;
import java.util.Enumeration;
import java.util.List;
import javax.annotation.Nullable;
import org.apache.commons.lang.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.sonar.application.config.AppSettings;
import org.sonar.process.cluster.NodeType;

import static org.sonar.process.cluster.ClusterProperties.CLUSTER_HOSTS;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_PORT;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_TYPE;

/**
* Properties of the cluster configuration
*/
final class ClusterProperties {
private static final String DEFAULT_PORT = "9003";
private static final Logger LOGGER = LoggerFactory.getLogger(ClusterProperties.class);
static final String HAZELCAST_CLUSTER_NAME = "sonarqube";

private final int port;
private final List<String> hosts;
private final List<String> networkInterfaces;
private final NodeType nodeType;
private final String nodeName;

ClusterProperties(AppSettings appSettings) {
port = appSettings.getProps().valueAsInt(CLUSTER_NODE_PORT);
networkInterfaces = extractNetworkInterfaces(appSettings.getProps().nonNullValue(CLUSTER_NODE_HOST));
hosts = extractHosts(appSettings.getProps().nonNullValue(CLUSTER_HOSTS));
nodeType = NodeType.parse(appSettings.getProps().nonNullValue(CLUSTER_NODE_TYPE));
nodeName = appSettings.getProps().nonNullValue(CLUSTER_NODE_NAME);
}

int getPort() {
return port;
}

public NodeType getNodeType() {
return nodeType;
}

List<String> getHosts() {
return hosts;
}

List<String> getNetworkInterfaces() {
return networkInterfaces;
}

public String getNodeName() {
return nodeName;
}

void validate() {
// Test validity of port
checkArgument(
port > 0 && port < 65_536,
"Cluster port have been set to %d which is outside the range [1-65535].",
port);

// Test the networkInterfaces parameter
try {
List<String> localInterfaces = findAllLocalIPs();

networkInterfaces.forEach(
inet -> checkArgument(
StringUtils.isEmpty(inet) || localInterfaces.contains(inet),
"Interface %s is not available on this machine.",
inet));
} catch (SocketException e) {
LOGGER.warn("Unable to retrieve network networkInterfaces. Interfaces won't be checked", e);
}
}

private static List<String> extractHosts(final String hosts) {
List<String> result = new ArrayList<>();
for (String host : hosts.split(",")) {
if (StringUtils.isNotEmpty(host)) {
if (!host.contains(":")) {
result.add(
String.format("%s:%s", host, DEFAULT_PORT));
} else {
result.add(host);
}
}
}
return result;
}

private static List<String> extractNetworkInterfaces(final String networkInterfaces) {
List<String> result = new ArrayList<>();
for (String iface : networkInterfaces.split(",")) {
if (StringUtils.isNotEmpty(iface)) {
result.add(iface);
}
}
return result;
}

private static List<String> findAllLocalIPs() throws SocketException {
Enumeration<NetworkInterface> netInterfaces = NetworkInterface.getNetworkInterfaces();
List<String> localInterfaces = new ArrayList<>();

while (netInterfaces.hasMoreElements()) {
NetworkInterface networkInterface = netInterfaces.nextElement();
Enumeration<InetAddress> ips = networkInterface.getInetAddresses();
while (ips.hasMoreElements()) {
InetAddress ip = ips.nextElement();
localInterfaces.add(ip.getHostAddress());
}
}
return localInterfaces;
}

private static void checkArgument(boolean expression,
@Nullable String messageTemplate,
@Nullable Object... args) {
if (!expression) {
throw new IllegalArgumentException(String.format(messageTemplate, args));
}
}
}

+ 0
- 427
server/sonar-main/src/main/java/org/sonar/application/cluster/HazelcastCluster.java Parādīt failu

@@ -1,427 +0,0 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/

package org.sonar.application.cluster;

import com.google.common.collect.ImmutableSet;
import com.hazelcast.config.Config;
import com.hazelcast.config.JoinConfig;
import com.hazelcast.config.NetworkConfig;
import com.hazelcast.core.Client;
import com.hazelcast.core.ClientListener;
import com.hazelcast.core.EntryEvent;
import com.hazelcast.core.EntryListener;
import com.hazelcast.core.Hazelcast;
import com.hazelcast.core.HazelcastInstance;
import com.hazelcast.core.HazelcastInstanceNotActiveException;
import com.hazelcast.core.IAtomicReference;
import com.hazelcast.core.ILock;
import com.hazelcast.core.MapEvent;
import com.hazelcast.core.Member;
import com.hazelcast.core.MemberAttributeEvent;
import com.hazelcast.core.MembershipEvent;
import com.hazelcast.core.MembershipListener;
import com.hazelcast.core.ReplicatedMap;
import com.hazelcast.nio.Address;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.locks.Lock;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.sonar.application.AppStateListener;
import org.sonar.process.MessageException;
import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessId;
import org.sonar.process.cluster.NodeType;
import org.sonar.process.cluster.ClusterObjectKeys;
import org.sonar.process.cluster.HazelcastClient;

import static java.lang.String.format;
import static java.util.stream.Collectors.toList;
import static org.sonar.application.cluster.ClusterProperties.HAZELCAST_CLUSTER_NAME;
import static org.sonar.process.cluster.ClusterObjectKeys.CLUSTER_NAME;
import static org.sonar.process.cluster.ClusterObjectKeys.HOSTNAME;
import static org.sonar.process.cluster.ClusterObjectKeys.IP_ADDRESSES;
import static org.sonar.process.cluster.ClusterObjectKeys.LEADER;
import static org.sonar.process.cluster.ClusterObjectKeys.LOCAL_MEMBER_UUIDS;
import static org.sonar.process.cluster.ClusterObjectKeys.NODE_NAME;
import static org.sonar.process.cluster.ClusterObjectKeys.NODE_TYPE;
import static org.sonar.process.cluster.ClusterObjectKeys.OPERATIONAL_PROCESSES;
import static org.sonar.process.cluster.ClusterObjectKeys.SONARQUBE_VERSION;

public class HazelcastCluster implements AutoCloseable {
private static final Logger LOGGER = LoggerFactory.getLogger(HazelcastCluster.class);

private final List<AppStateListener> listeners = new ArrayList<>();
private final ReplicatedMap<ClusterProcess, Boolean> operationalProcesses;
private final String operationalProcessListenerUUID;
private final String clientListenerUUID;
private final String nodeDisconnectedListenerUUID;

protected final HazelcastInstance hzInstance;

private HazelcastCluster(Config hzConfig) {
// Create the Hazelcast instance
hzInstance = Hazelcast.newHazelcastInstance(hzConfig);

// Get or create the replicated map
operationalProcesses = hzInstance.getReplicatedMap(OPERATIONAL_PROCESSES);
operationalProcessListenerUUID = operationalProcesses.addEntryListener(new OperationalProcessListener());
clientListenerUUID = hzInstance.getClientService().addClientListener(new ConnectedClientListener());
nodeDisconnectedListenerUUID = hzInstance.getCluster().addMembershipListener(new NodeDisconnectedListener());
}

String getLocalUUID() {
return hzInstance.getLocalEndpoint().getUuid();
}

String getName() {
return hzInstance.getConfig().getGroupConfig().getName();
}

List<String> getMembers() {
return hzInstance.getCluster().getMembers().stream()
.filter(m -> !m.localMember())
.map(m -> format("%s (%s)", m.getStringAttribute(HOSTNAME), m.getStringAttribute(IP_ADDRESSES)))
.collect(toList());
}

void addListener(AppStateListener listener) {
listeners.add(listener);
}

boolean isOperational(ProcessId processId) {
for (Map.Entry<ClusterProcess, Boolean> entry : operationalProcesses.entrySet()) {
if (entry.getKey().getProcessId().equals(processId) && entry.getValue()) {
return true;
}
}
return false;
}

void setOperational(ProcessId processId) {
operationalProcesses.put(new ClusterProcess(getLocalUUID(), processId), Boolean.TRUE);
}

boolean tryToLockWebLeader() {
IAtomicReference<String> leader = hzInstance.getAtomicReference(LEADER);
if (leader.get() == null) {
ILock lock = hzInstance.getLock(LEADER);
lock.lock();
try {
if (leader.get() == null) {
leader.set(getLocalUUID());
return true;
} else {
return false;
}
} finally {
lock.unlock();
}
} else {
return false;
}
}

public void registerSonarQubeVersion(String sonarqubeVersion) {
IAtomicReference<String> sqVersion = hzInstance.getAtomicReference(SONARQUBE_VERSION);
if (sqVersion.get() == null) {
ILock lock = hzInstance.getLock(SONARQUBE_VERSION);
lock.lock();
try {
if (sqVersion.get() == null) {
sqVersion.set(sonarqubeVersion);
}
} finally {
lock.unlock();
}
}

String clusterVersion = sqVersion.get();
if (!sqVersion.get().equals(sonarqubeVersion)) {
throw new IllegalStateException(
format("The local version %s is not the same as the cluster %s", sonarqubeVersion, clusterVersion));
}
}

public String getSonarQubeVersion() {
IAtomicReference<String> sqVersion = hzInstance.getAtomicReference(SONARQUBE_VERSION);
return sqVersion.get();
}

public void registerClusterName(String nodeValue) {
IAtomicReference<String> property = hzInstance.getAtomicReference(CLUSTER_NAME);
if (property.get() == null) {
ILock lock = hzInstance.getLock(CLUSTER_NAME);
lock.lock();
try {
if (property.get() == null) {
property.set(nodeValue);
}
} finally {
lock.unlock();
}
}

String clusterValue = property.get();
if (!property.get().equals(nodeValue)) {
throw new MessageException(
format("This node has a cluster name [%s], which does not match [%s] from the cluster", nodeValue, clusterValue));
}
}

@Override
public void close() {
if (hzInstance != null) {
try {
// Removing listeners
operationalProcesses.removeEntryListener(operationalProcessListenerUUID);
hzInstance.getClientService().removeClientListener(clientListenerUUID);
hzInstance.getCluster().removeMembershipListener(nodeDisconnectedListenerUUID);

// Removing the operationalProcess from the replicated map
operationalProcesses.keySet().forEach(
clusterNodeProcess -> {
if (clusterNodeProcess.getNodeUuid().equals(getLocalUUID())) {
operationalProcesses.remove(clusterNodeProcess);
}
});

// Shutdown Hazelcast properly
hzInstance.shutdown();
} catch (HazelcastInstanceNotActiveException e) {
// hazelcastCluster may be already closed by the shutdown hook
LOGGER.debug("Unable to close Hazelcast cluster", e);
}
}
}

public static HazelcastCluster create(ClusterProperties clusterProperties) {
Config hzConfig = new Config();
hzConfig.getGroupConfig().setName(HAZELCAST_CLUSTER_NAME);

// Configure the network instance
NetworkConfig netConfig = hzConfig.getNetworkConfig();
netConfig
.setPort(clusterProperties.getPort())
.setReuseAddress(true);

if (!clusterProperties.getNetworkInterfaces().isEmpty()) {
netConfig.getInterfaces()
.setEnabled(true)
.setInterfaces(clusterProperties.getNetworkInterfaces());
}

// Only allowing TCP/IP configuration
JoinConfig joinConfig = netConfig.getJoin();
joinConfig.getAwsConfig().setEnabled(false);
joinConfig.getMulticastConfig().setEnabled(false);
joinConfig.getTcpIpConfig().setEnabled(true);
joinConfig.getTcpIpConfig().setMembers(clusterProperties.getHosts());

// Tweak HazelCast configuration
hzConfig
// Increase the number of tries
.setProperty("hazelcast.tcp.join.port.try.count", "10")
// Don't bind on all interfaces
.setProperty("hazelcast.socket.bind.any", "false")
// Don't phone home
.setProperty("hazelcast.phone.home.enabled", "false")
// Use slf4j for logging
.setProperty("hazelcast.logging.type", "slf4j");

// Trying to resolve the hostname
hzConfig.getMemberAttributeConfig()
.setStringAttribute(NODE_NAME, clusterProperties.getNodeName());
hzConfig.getMemberAttributeConfig()
.setStringAttribute(HOSTNAME, NetworkUtils.INSTANCE.getHostname());
hzConfig.getMemberAttributeConfig()
.setStringAttribute(IP_ADDRESSES, NetworkUtils.INSTANCE.getIPAddresses());
hzConfig.getMemberAttributeConfig()
.setStringAttribute(NODE_TYPE, clusterProperties.getNodeType().getValue());

// We are not using the partition group of Hazelcast, so disabling it
hzConfig.getPartitionGroupConfig().setEnabled(false);
return new HazelcastCluster(hzConfig);
}

Optional<String> getLeaderHostName() {
String leaderId = (String) hzInstance.getAtomicReference(LEADER).get();
if (leaderId != null) {
Optional<Member> leader = hzInstance.getCluster().getMembers().stream().filter(m -> m.getUuid().equals(leaderId)).findFirst();
if (leader.isPresent()) {
return Optional.of(
format("%s (%s)", leader.get().getStringAttribute(HOSTNAME), leader.get().getStringAttribute(IP_ADDRESSES)));
}
}
return Optional.empty();
}

String getLocalEndPoint() {
Address localAddress = hzInstance.getCluster().getLocalMember().getAddress();
return format("%s:%d", localAddress.getHost(), localAddress.getPort());
}

public HazelcastClient getHazelcastClient() {
return new HazelcastInstanceClient(hzInstance);
}

private static class HazelcastInstanceClient implements HazelcastClient {
private final HazelcastInstance hzInstance;

private HazelcastInstanceClient(HazelcastInstance hzInstance) {
this.hzInstance = hzInstance;
}

@Override
public <E> Set<E> getSet(String s) {
return hzInstance.getSet(s);
}

@Override
public <E> List<E> getList(String s) {
return hzInstance.getList(s);
}

@Override
public <K, V> Map<K, V> getMap(String s) {
return hzInstance.getMap(s);
}

@Override
public <K, V> Map<K, V> getReplicatedMap(String s) {
return hzInstance.getReplicatedMap(s);
}

@Override
public String getUUID() {
return hzInstance.getLocalEndpoint().getUuid();
}

@Override
public Set<String> getMemberUuids() {
ImmutableSet.Builder<String> builder = ImmutableSet.builder();
builder.addAll(hzInstance.getSet(ClusterObjectKeys.LOCAL_MEMBER_UUIDS));
hzInstance.getCluster().getMembers().stream().map(Member::getUuid).forEach(builder::add);
return builder.build();
}

@Override
public Lock getLock(String s) {
return hzInstance.getLock(s);
}

@Override
public long getClusterTime() {
return hzInstance.getCluster().getClusterTime();
}
}

private class OperationalProcessListener implements EntryListener<ClusterProcess, Boolean> {
@Override
public void entryAdded(EntryEvent<ClusterProcess, Boolean> event) {
if (event.getValue()) {
listeners.forEach(appStateListener -> appStateListener.onAppStateOperational(event.getKey().getProcessId()));
}
}

@Override
public void entryRemoved(EntryEvent<ClusterProcess, Boolean> event) {
// Ignore it
}

@Override
public void entryUpdated(EntryEvent<ClusterProcess, Boolean> event) {
if (event.getValue()) {
listeners.forEach(appStateListener -> appStateListener.onAppStateOperational(event.getKey().getProcessId()));
}
}

@Override
public void entryEvicted(EntryEvent<ClusterProcess, Boolean> event) {
// Ignore it
}

@Override
public void mapCleared(MapEvent event) {
// Ignore it
}

@Override
public void mapEvicted(MapEvent event) {
// Ignore it
}
}

private class ConnectedClientListener implements ClientListener {
@Override
public void clientConnected(Client client) {
hzInstance.getSet(LOCAL_MEMBER_UUIDS).add(client.getUuid());
}

@Override
public void clientDisconnected(Client client) {
hzInstance.getSet(LOCAL_MEMBER_UUIDS).remove(client.getUuid());
}
}

private class NodeDisconnectedListener implements MembershipListener {
@Override
public void memberAdded(MembershipEvent membershipEvent) {
// Nothing to do
}

@Override
public void memberRemoved(MembershipEvent membershipEvent) {
removeOperationalProcess(membershipEvent.getMember().getUuid());
if (membershipEvent.getMembers().stream()
.noneMatch(this::isAppNode)) {
purgeSharedMemoryForAppNodes();
}
}

@Override
public void memberAttributeChanged(MemberAttributeEvent memberAttributeEvent) {
// Nothing to do
}

private boolean isAppNode(Member member) {
return NodeType.APPLICATION.getValue().equals(member.getStringAttribute(NODE_TYPE));
}

private void removeOperationalProcess(String uuid) {
for (ClusterProcess clusterProcess : operationalProcesses.keySet()) {
if (clusterProcess.getNodeUuid().equals(uuid)) {
LOGGER.debug("Set node process off for [{}:{}] : ", clusterProcess.getNodeUuid(), clusterProcess.getProcessId());
hzInstance.getReplicatedMap(OPERATIONAL_PROCESSES).put(clusterProcess, Boolean.FALSE);
}
}
}

private void purgeSharedMemoryForAppNodes() {
LOGGER.info("No more application nodes, clearing cluster information about application nodes.");
hzInstance.getAtomicReference(LEADER).clear();
hzInstance.getAtomicReference(SONARQUBE_VERSION).clear();
}
}
}

server/sonar-process/src/main/java/org/sonar/process/cluster/health/DelegateHealthStateRefresherExecutorService.java → server/sonar-main/src/main/java/org/sonar/application/cluster/health/DelegateHealthStateRefresherExecutorService.java Parādīt failu

@@ -17,7 +17,7 @@
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.process.cluster.health;
package org.sonar.application.cluster.health;

import java.util.Collection;
import java.util.List;
@@ -28,6 +28,7 @@ import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.sonar.process.cluster.health.HealthStateRefresherExecutorService;

class DelegateHealthStateRefresherExecutorService implements HealthStateRefresherExecutorService {
private final ScheduledExecutorService delegate;

server/sonar-process/src/main/java/org/sonar/process/cluster/health/HealthStateSharing.java → server/sonar-main/src/main/java/org/sonar/application/cluster/health/HealthStateSharing.java Parādīt failu

@@ -17,7 +17,7 @@
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.process.cluster.health;
package org.sonar.application.cluster.health;

public interface HealthStateSharing {
void start();

server/sonar-process/src/main/java/org/sonar/process/cluster/health/HealthStateSharingImpl.java → server/sonar-main/src/main/java/org/sonar/application/cluster/health/HealthStateSharingImpl.java Parādīt failu

@@ -17,7 +17,7 @@
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.process.cluster.health;
package org.sonar.application.cluster.health;

import com.google.common.util.concurrent.ThreadFactoryBuilder;
import java.util.concurrent.Executors;
@@ -25,20 +25,24 @@ import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.sonar.process.cluster.HazelcastClient;
import org.sonar.process.cluster.health.HealthStateRefresher;
import org.sonar.process.cluster.health.HealthStateRefresherExecutorService;
import org.sonar.process.cluster.health.NodeHealthProvider;
import org.sonar.process.cluster.health.SharedHealthStateImpl;
import org.sonar.process.cluster.hz.HazelcastMember;

import static java.lang.String.format;

public class HealthStateSharingImpl implements HealthStateSharing {
private static final Logger LOG = LoggerFactory.getLogger(HealthStateSharingImpl.class);

private final HazelcastClient hazelcastClient;
private final HazelcastMember hzMember;
private final NodeHealthProvider nodeHealthProvider;
private HealthStateRefresherExecutorService executorService;
private HealthStateRefresher healthStateRefresher;

public HealthStateSharingImpl(HazelcastClient hazelcastClient, NodeHealthProvider nodeHealthProvider) {
this.hazelcastClient = hazelcastClient;
public HealthStateSharingImpl(HazelcastMember hzMember, NodeHealthProvider nodeHealthProvider) {
this.hzMember = hzMember;
this.nodeHealthProvider = nodeHealthProvider;
}

@@ -50,7 +54,7 @@ public class HealthStateSharingImpl implements HealthStateSharing {
.setDaemon(false)
.setNameFormat("health_state_refresh-%d")
.build()));
healthStateRefresher = new HealthStateRefresher(executorService, nodeHealthProvider, new SharedHealthStateImpl(hazelcastClient));
healthStateRefresher = new HealthStateRefresher(executorService, nodeHealthProvider, new SharedHealthStateImpl(hzMember));
healthStateRefresher.start();
}


server/sonar-main/src/main/java/org/sonar/application/cluster/SearchNodeHealthProvider.java → server/sonar-main/src/main/java/org/sonar/application/cluster/health/SearchNodeHealthProvider.java Parādīt failu

@@ -17,8 +17,9 @@
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.application.cluster;
package org.sonar.application.cluster.health;

import org.sonar.application.cluster.ClusterAppState;
import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessId;
import org.sonar.process.Props;
@@ -26,9 +27,9 @@ import org.sonar.process.cluster.health.NodeDetails;
import org.sonar.process.cluster.health.NodeHealth;
import org.sonar.process.cluster.health.NodeHealthProvider;

import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_PORT;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_PORT;

public class SearchNodeHealthProvider implements NodeHealthProvider {


+ 23
- 0
server/sonar-main/src/main/java/org/sonar/application/cluster/health/package-info.java Parādīt failu

@@ -0,0 +1,23 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
@ParametersAreNonnullByDefault
package org.sonar.application.cluster.health;

import javax.annotation.ParametersAreNonnullByDefault;

+ 3
- 1
server/sonar-main/src/main/java/org/sonar/application/config/AppSettingsLoaderImpl.java Parādīt failu

@@ -30,6 +30,7 @@ import java.util.Properties;
import java.util.function.Consumer;
import org.slf4j.LoggerFactory;
import org.sonar.process.ConfigurationUtils;
import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessProperties;
import org.sonar.process.Props;

@@ -42,7 +43,8 @@ public class AppSettingsLoaderImpl implements AppSettingsLoader {
private final Consumer<Props>[] consumers;

public AppSettingsLoaderImpl(String[] cliArguments) {
this(cliArguments, detectHomeDir(), new FileSystemSettings(), new JdbcSettings(), new ClusterSettings());
this(cliArguments, detectHomeDir(),
new FileSystemSettings(), new JdbcSettings(), new ClusterSettings(NetworkUtils.INSTANCE));
}

AppSettingsLoaderImpl(String[] cliArguments, File homeDir, Consumer<Props>... consumers) {

+ 60
- 90
server/sonar-main/src/main/java/org/sonar/application/config/ClusterSettings.java Parādīt failu

@@ -19,11 +19,7 @@
*/
package org.sonar.application.config;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.net.HostAndPort;
import com.google.common.net.InetAddresses;
import java.net.InetAddress;
import java.net.NetworkInterface;
import java.net.SocketException;
import java.net.UnknownHostException;
import java.util.Arrays;
@@ -31,142 +27,116 @@ import java.util.List;
import java.util.function.Consumer;
import org.apache.commons.lang.StringUtils;
import org.sonar.process.MessageException;
import org.sonar.process.cluster.NodeType;
import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessId;
import org.sonar.process.Props;
import org.sonar.process.cluster.NodeType;

import static com.google.common.net.InetAddresses.forString;
import static java.lang.String.format;
import static java.util.Arrays.asList;
import static java.util.Arrays.stream;
import static java.util.Collections.singletonList;
import static java.util.stream.Collectors.joining;
import static org.apache.commons.lang.StringUtils.isBlank;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_HOSTS;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_TYPE;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_SEARCH_HOSTS;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_WEB_LEADER;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_HOSTS;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_TYPE;
import static org.sonar.process.ProcessProperties.CLUSTER_SEARCH_HOSTS;
import static org.sonar.process.ProcessProperties.CLUSTER_WEB_STARTUP_LEADER;
import static org.sonar.process.ProcessProperties.JDBC_URL;
import static org.sonar.process.ProcessProperties.SEARCH_HOST;

public class ClusterSettings implements Consumer<Props> {

private final NetworkUtils network;

public ClusterSettings(NetworkUtils network) {
this.network = network;
}

@Override
public void accept(Props props) {
if (isClusterEnabled(props)) {
checkProperties(props);
checkClusterProperties(props);
}
}

private static void checkProperties(Props props) {
// Cluster web leader is not allowed
if (props.value(CLUSTER_WEB_LEADER) != null) {
throw new MessageException(format("Property [%s] is forbidden", CLUSTER_WEB_LEADER));
}

// Mandatory properties
ensureMandatoryProperty(props, CLUSTER_NODE_TYPE);
String nodeTypeValue = props.nonNullValue(CLUSTER_NODE_TYPE);
if (!NodeType.isValid(nodeTypeValue)) {
throw new MessageException(format("Invalid value for property [%s]: [%s], only [%s] are allowed", CLUSTER_NODE_TYPE, nodeTypeValue,
Arrays.stream(NodeType.values()).map(NodeType::getValue).collect(joining(", "))));
private void checkClusterProperties(Props props) {
// for internal use
if (props.value(CLUSTER_WEB_STARTUP_LEADER) != null) {
throw new MessageException(format("Property [%s] is forbidden", CLUSTER_WEB_STARTUP_LEADER));
}
ensureMandatoryProperty(props, CLUSTER_HOSTS);
ensureMandatoryProperty(props, CLUSTER_SEARCH_HOSTS);

NodeType nodeType = NodeType.parse(nodeTypeValue);
NodeType nodeType = toNodeType(props);
switch (nodeType) {
case APPLICATION:
ensureNotH2(props);
ensureMandatoryProperty(props, "sonar.auth.jwtBase64Hs256Secret");
requireValue(props, "sonar.auth.jwtBase64Hs256Secret");
break;
case SEARCH:
ensureMandatoryProperty(props, SEARCH_HOST);
ensureNotLoopback(props, SEARCH_HOST);
requireValue(props, SEARCH_HOST);
ensureLocalButNotLoopbackAddress(props, SEARCH_HOST);
break;
default:
throw new IllegalArgumentException("Unsupported node type: " + nodeType);
throw new UnsupportedOperationException("Unknown value: " + nodeType);
}

// Loopback interfaces are forbidden for the ports accessed
// by other nodes of cluster
ensureNotLoopback(props, CLUSTER_HOSTS);
ensureNotLoopback(props, CLUSTER_NODE_HOST);
ensureNotLoopback(props, CLUSTER_SEARCH_HOSTS);

ensureLocalAddress(props, SEARCH_HOST);
ensureLocalAddress(props, CLUSTER_NODE_HOST);
ensureNotLoopbackAddresses(props, CLUSTER_HOSTS);
requireValue(props, CLUSTER_NODE_HOST);
ensureLocalButNotLoopbackAddress(props, CLUSTER_NODE_HOST);
ensureNotLoopbackAddresses(props, CLUSTER_SEARCH_HOSTS);
}

private static void ensureNotH2(Props props) {
String jdbcUrl = props.value(JDBC_URL);
if (isBlank(jdbcUrl) || jdbcUrl.startsWith("jdbc:h2:")) {
throw new MessageException("Embedded database is not supported in cluster mode");
private static NodeType toNodeType(Props props) {
String nodeTypeValue = requireValue(props, CLUSTER_NODE_TYPE);
if (!NodeType.isValid(nodeTypeValue)) {
throw new MessageException(format("Invalid value for property %s: [%s], only [%s] are allowed", CLUSTER_NODE_TYPE, nodeTypeValue,
Arrays.stream(NodeType.values()).map(NodeType::getValue).collect(joining(", "))));
}
return NodeType.parse(nodeTypeValue);
}

private static void ensureMandatoryProperty(Props props, String key) {
if (isBlank(props.value(key))) {
throw new MessageException(format("Property [%s] is mandatory", key));
private static String requireValue(Props props, String key) {
String value = props.value(key);
if (isBlank(value)) {
throw new MessageException(format("Property %s is mandatory", key));
}
return value;
}

@VisibleForTesting
private static void ensureNotLoopback(Props props, String key) {
String ipList = props.value(key);
if (ipList == null) {
return;
private static void ensureNotH2(Props props) {
String jdbcUrl = props.value(JDBC_URL);
if (isBlank(jdbcUrl) || jdbcUrl.startsWith("jdbc:h2:")) {
throw new MessageException("Embedded database is not supported in cluster mode");
}

stream(ipList.split(","))
.filter(StringUtils::isNotBlank)
.map(StringUtils::trim)
.forEach(ip -> {
InetAddress inetAddress = convertToInetAddress(ip, key);
if (inetAddress.isLoopbackAddress()) {
throw new MessageException(format("The interface address [%s] of [%s] must not be a loopback address", ip, key));
}
});
}

private static void ensureLocalAddress(Props props, String key) {
String ipList = props.value(key);

if (ipList == null) {
return;
}

stream(ipList.split(","))
private void ensureNotLoopbackAddresses(Props props, String propertyKey) {
stream(requireValue(props, propertyKey).split(","))
.filter(StringUtils::isNotBlank)
.map(StringUtils::trim)
.map(s -> StringUtils.substringBefore(s, ":"))
.forEach(ip -> {
InetAddress inetAddress = convertToInetAddress(ip, key);
try {
if (NetworkInterface.getByInetAddress(inetAddress) == null) {
throw new MessageException(format("The interface address [%s] of [%s] is not a local address", ip, key));
if (network.isLoopbackInetAddress(network.toInetAddress(ip))) {
throw new MessageException(format("Property %s must not be a loopback address: %s", propertyKey, ip));
}
} catch (SocketException e) {
throw new MessageException(format("The interface address [%s] of [%s] is not a local address", ip, key));
} catch (UnknownHostException e) {
throw new MessageException(format("Property %s must not a valid address: %s [%s]", propertyKey, ip, e.getMessage()));
}
});
}

private static InetAddress convertToInetAddress(String text, String key) {
InetAddress inetAddress;
HostAndPort hostAndPort = HostAndPort.fromString(text);
if (!InetAddresses.isInetAddress(hostAndPort.getHostText())) {
try {
inetAddress = InetAddress.getByName(hostAndPort.getHostText());
} catch (UnknownHostException e) {
throw new MessageException(format("The interface address [%s] of [%s] cannot be resolved : %s", text, key, e.getMessage()));
private void ensureLocalButNotLoopbackAddress(Props props, String propertyKey) {
String propertyValue = props.nonNullValue(propertyKey).trim();
try {
InetAddress address = network.toInetAddress(propertyValue);
if (!network.isLocalInetAddress(address) || network.isLoopbackInetAddress(address)) {
throw new MessageException(format("Property %s must be a local non-loopback address: %s", propertyKey, propertyValue));
}
} else {
inetAddress = forString(hostAndPort.getHostText());
} catch (UnknownHostException | SocketException e) {
throw new MessageException(format("Property %s must be a local non-loopback address: %s [%s]", propertyKey, propertyValue, e.getMessage()));
}

return inetAddress;
}

public static boolean isClusterEnabled(AppSettings settings) {
@@ -181,7 +151,7 @@ public class ClusterSettings implements Consumer<Props> {
if (!isClusterEnabled(settings)) {
return asList(ProcessId.ELASTICSEARCH, ProcessId.WEB_SERVER, ProcessId.COMPUTE_ENGINE);
}
NodeType nodeType = NodeType.parse(settings.getValue(CLUSTER_NODE_TYPE).orElse(null));
NodeType nodeType = NodeType.parse(settings.getValue(CLUSTER_NODE_TYPE).orElse(""));
switch (nodeType) {
case APPLICATION:
return asList(ProcessId.WEB_SERVER, ProcessId.COMPUTE_ENGINE);
@@ -195,7 +165,7 @@ public class ClusterSettings implements Consumer<Props> {
public static boolean isLocalElasticsearchEnabled(AppSettings settings) {
// elasticsearch is enabled on "search" nodes, but disabled on "application" nodes
if (isClusterEnabled(settings.getProps())) {
return NodeType.parse(settings.getValue(CLUSTER_NODE_TYPE).orElse(null)) == NodeType.SEARCH;
return NodeType.parse(settings.getValue(CLUSTER_NODE_TYPE).orElse("")) == NodeType.SEARCH;
}

// elasticsearch is enabled in standalone mode

+ 4
- 4
server/sonar-main/src/main/java/org/sonar/application/es/EsSettings.java Parādīt failu

@@ -31,10 +31,10 @@ import org.sonar.process.Props;
import org.sonar.process.System2;

import static java.lang.String.valueOf;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_SEARCH_HOSTS;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_NAME;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.ProcessProperties.CLUSTER_SEARCH_HOSTS;

public class EsSettings {


+ 1
- 14
server/sonar-main/src/test/java/org/sonar/application/AppLoggingTest.java Parādīt failu

@@ -48,8 +48,7 @@ import org.sonar.process.logging.LogbackHelper;
import static org.assertj.core.api.Assertions.assertThat;
import static org.slf4j.Logger.ROOT_LOGGER_NAME;
import static org.sonar.application.process.StreamGobbler.LOGGER_GOBBLER;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.HAZELCAST_LOG_LEVEL;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;

public class AppLoggingTest {

@@ -257,18 +256,6 @@ public class AppLoggingTest {
LoggerFactory.getLogger("com.hazelcast").isInfoEnabled()).isEqualTo(false);
}

@Test
public void configure_logging_for_hazelcast() throws IOException {
settings.getProps().set(CLUSTER_ENABLED, "true");
settings.getProps().set(HAZELCAST_LOG_LEVEL, "INFO");
underTest.configure();

assertThat(
LoggerFactory.getLogger("com.hazelcast").isInfoEnabled()).isEqualTo(true);
assertThat(
LoggerFactory.getLogger("com.hazelcast").isDebugEnabled()).isEqualTo(false);
}

private void emulateRunFromSonarScript() {
settings.getProps().set("sonar.wrapped", "true");
}

+ 1
- 1
server/sonar-main/src/test/java/org/sonar/application/AppReloaderImplTest.java Parādīt failu

@@ -35,7 +35,7 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyZeroInteractions;
import static org.mockito.Mockito.when;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;

public class AppReloaderImplTest {


+ 16
- 4
server/sonar-main/src/test/java/org/sonar/application/AppStateFactoryTest.java Parādīt failu

@@ -19,14 +19,21 @@
*/
package org.sonar.application;

import java.net.InetAddress;
import java.util.Optional;
import org.hamcrest.CoreMatchers;
import org.junit.Test;
import org.sonar.application.cluster.ClusterAppStateImpl;
import org.sonar.application.config.TestAppSettings;
import org.sonar.process.NetworkUtils;

import static org.assertj.core.api.Assertions.assertThat;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_TYPE;
import static org.junit.Assume.assumeThat;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_HOSTS;
import static org.sonar.process.ProcessProperties.CLUSTER_NAME;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_TYPE;

public class AppStateFactoryTest {

@@ -35,13 +42,18 @@ public class AppStateFactoryTest {

@Test
public void create_cluster_implementation_if_cluster_is_enabled() {
Optional<InetAddress> ip = NetworkUtils.INSTANCE.getLocalNonLoopbackIpv4Address();
assumeThat(ip.isPresent(), CoreMatchers.is(true));

settings.set(CLUSTER_ENABLED, "true");
settings.set(CLUSTER_NODE_TYPE, "application");
settings.set(CLUSTER_NODE_HOST, ip.get().getHostAddress());
settings.set(CLUSTER_HOSTS, ip.get().getHostAddress());
settings.set(CLUSTER_NAME, "foo");

AppState appState = underTest.create();
assertThat(appState).isInstanceOf(ClusterAppStateImpl.class);
((ClusterAppStateImpl) appState).close();
appState.close();
}

@Test

+ 8
- 8
server/sonar-main/src/test/java/org/sonar/application/SchedulerImplTest.java Parādīt failu

@@ -46,7 +46,7 @@ import org.sonar.application.config.TestAppSettings;
import org.sonar.application.process.ProcessLauncher;
import org.sonar.application.process.ProcessMonitor;
import org.sonar.process.ProcessId;
import org.sonar.process.cluster.HazelcastClient;
import org.sonar.process.cluster.hz.HazelcastMember;

import static java.util.Collections.synchronizedList;
import static org.apache.commons.lang.RandomStringUtils.randomAlphanumeric;
@@ -58,11 +58,11 @@ import static org.mockito.Mockito.verify;
import static org.sonar.process.ProcessId.COMPUTE_ENGINE;
import static org.sonar.process.ProcessId.ELASTICSEARCH;
import static org.sonar.process.ProcessId.WEB_SERVER;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_PORT;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_TYPE;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_PORT;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_TYPE;

public class SchedulerImplTest {

@@ -83,8 +83,8 @@ public class SchedulerImplTest {
private TestCommandFactory javaCommandFactory = new TestCommandFactory();
private TestProcessLauncher processLauncher = new TestProcessLauncher();
private TestAppState appState = new TestAppState();
private HazelcastClient hazelcastClient = mock(HazelcastClient.class);
private TestClusterAppState clusterAppState = new TestClusterAppState(hazelcastClient);
private HazelcastMember hazelcastMember = mock(HazelcastMember.class);
private TestClusterAppState clusterAppState = new TestClusterAppState(hazelcastMember);
private List<ProcessId> orderedStops = synchronizedList(new ArrayList<>());

@Before

+ 6
- 6
server/sonar-main/src/test/java/org/sonar/application/TestClusterAppState.java Parādīt failu

@@ -20,17 +20,17 @@
package org.sonar.application;

import org.sonar.application.cluster.ClusterAppState;
import org.sonar.process.cluster.HazelcastClient;
import org.sonar.process.cluster.hz.HazelcastMember;

public class TestClusterAppState extends TestAppState implements ClusterAppState {
private final HazelcastClient hazelcastClient;
private final HazelcastMember hazelcastMember;

public TestClusterAppState(HazelcastClient hazelcastClient) {
this.hazelcastClient = hazelcastClient;
public TestClusterAppState(HazelcastMember hazelcastMember) {
this.hazelcastMember = hazelcastMember;
}

@Override
public HazelcastClient getHazelcastClient() {
return hazelcastClient;
public HazelcastMember getHazelcastMember() {
return hazelcastMember;
}
}

+ 23
- 76
server/sonar-main/src/test/java/org/sonar/application/cluster/ClusterAppStateImplTest.java Parādīt failu

@@ -19,32 +19,23 @@
*/
package org.sonar.application.cluster;

import com.hazelcast.core.HazelcastInstance;
import java.io.IOException;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.DisableOnDebug;
import org.junit.rules.ExpectedException;
import org.junit.rules.TestRule;
import org.junit.rules.Timeout;
import org.slf4j.Logger;
import org.sonar.application.AppStateListener;
import org.sonar.application.config.TestAppSettings;
import org.sonar.process.MessageException;
import org.sonar.process.ProcessId;

import static org.apache.commons.lang.RandomStringUtils.randomAlphanumeric;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Matchers.anyString;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.timeout;
import static org.mockito.Mockito.verify;
import static org.sonar.application.cluster.HazelcastClusterTestHelper.createHazelcastClient;
import static org.sonar.application.cluster.HazelcastClusterTestHelper.newApplicationSettings;
import static org.sonar.process.cluster.ClusterObjectKeys.CLUSTER_NAME;
import static org.sonar.process.cluster.ClusterObjectKeys.SONARQUBE_VERSION;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.application.cluster.HazelcastTesting.newHzMember;
import static org.sonar.process.cluster.hz.HazelcastObjects.CLUSTER_NAME;
import static org.sonar.process.cluster.hz.HazelcastObjects.SONARQUBE_VERSION;

public class ClusterAppStateImplTest {

@@ -54,46 +45,18 @@ public class ClusterAppStateImplTest {
@Rule
public TestRule safeguardTimeout = new DisableOnDebug(Timeout.seconds(60));

@Test
public void instantiation_throws_ISE_if_cluster_mode_is_disabled() throws Exception {
TestAppSettings settings = new TestAppSettings();
settings.set(CLUSTER_ENABLED, "false");

expectedException.expect(IllegalStateException.class);
expectedException.expectMessage("Cluster is not enabled on this instance");

new ClusterAppStateImpl(settings);
}

@Test
public void tryToLockWebLeader_returns_true_only_for_the_first_call() throws Exception {
TestAppSettings settings = newApplicationSettings();

try (ClusterAppStateImpl underTest = new ClusterAppStateImpl(settings)) {
try (ClusterAppStateImpl underTest = new ClusterAppStateImpl(newHzMember())) {
assertThat(underTest.tryToLockWebLeader()).isEqualTo(true);
assertThat(underTest.tryToLockWebLeader()).isEqualTo(false);
}
}

@Test
public void log_when_sonarqube_is_joining_a_cluster() throws IOException, InterruptedException, IllegalAccessException, NoSuchFieldException {
// Now launch an instance that try to be part of the hzInstance cluster
TestAppSettings settings = newApplicationSettings();

Logger logger = mock(Logger.class);
ClusterAppStateImpl.setLogger(logger);

try (ClusterAppStateImpl appStateCluster = new ClusterAppStateImpl(settings)) {
verify(logger).info(
eq("Joined a SonarQube cluster that contains the following hosts : [{}]"),
anyString());
}
}

@Test
public void test_listeners() throws InterruptedException {
AppStateListener listener = mock(AppStateListener.class);
try (ClusterAppStateImpl underTest = new ClusterAppStateImpl(newApplicationSettings())) {
try (ClusterAppStateImpl underTest = new ClusterAppStateImpl(newHzMember())) {
underTest.addListener(listener);

underTest.setOperational(ProcessId.ELASTICSEARCH);
@@ -108,77 +71,61 @@ public class ClusterAppStateImplTest {

@Test
public void registerSonarQubeVersion_publishes_version_on_first_call() {
TestAppSettings settings = newApplicationSettings();

try (ClusterAppStateImpl appStateCluster = new ClusterAppStateImpl(settings)) {
appStateCluster.registerSonarQubeVersion("6.4.1.5");
try (ClusterAppStateImpl underTest = new ClusterAppStateImpl(newHzMember())) {
underTest.registerSonarQubeVersion("6.4.1.5");

HazelcastInstance hzInstance = createHazelcastClient(appStateCluster);
assertThat(hzInstance.getAtomicReference(SONARQUBE_VERSION).get())
.isNotNull()
.isInstanceOf(String.class)
assertThat(underTest.getHazelcastMember().getAtomicReference(SONARQUBE_VERSION).get())
.isEqualTo("6.4.1.5");
}
}

@Test
public void registerClusterName_publishes_clusterName_on_first_call() {
TestAppSettings settings = newApplicationSettings();
String clusterName = randomAlphanumeric(20);
try (ClusterAppStateImpl underTest = new ClusterAppStateImpl(newHzMember())) {
underTest.registerClusterName("foo");

try (ClusterAppStateImpl appStateCluster = new ClusterAppStateImpl(settings)) {
appStateCluster.registerClusterName(clusterName);

HazelcastInstance hzInstance = createHazelcastClient(appStateCluster);
assertThat(hzInstance.getAtomicReference(CLUSTER_NAME).get())
.isNotNull()
.isInstanceOf(String.class)
.isEqualTo(clusterName);
assertThat(underTest.getHazelcastMember().getAtomicReference(CLUSTER_NAME).get())
.isEqualTo("foo");
}
}

@Test
public void reset_throws_always_ISE() {
TestAppSettings settings = newApplicationSettings();

try (ClusterAppStateImpl appStateCluster = new ClusterAppStateImpl(settings)) {
public void reset_always_throws_ISE() {
try (ClusterAppStateImpl underTest = new ClusterAppStateImpl(newHzMember())) {
expectedException.expect(IllegalStateException.class);
expectedException.expectMessage("state reset is not supported in cluster mode");
appStateCluster.reset();

underTest.reset();
}
}

@Test
public void registerSonarQubeVersion_throws_ISE_if_initial_version_is_different() throws Exception {
// Now launch an instance that try to be part of the hzInstance cluster
TestAppSettings settings = newApplicationSettings();

try (ClusterAppStateImpl appStateCluster = new ClusterAppStateImpl(settings)) {
try (ClusterAppStateImpl underTest = new ClusterAppStateImpl(newHzMember())) {
// Register first version
appStateCluster.registerSonarQubeVersion("1.0.0");
underTest.getHazelcastMember().getAtomicReference(SONARQUBE_VERSION).set("6.6.0.1111");

expectedException.expect(IllegalStateException.class);
expectedException.expectMessage("The local version 2.0.0 is not the same as the cluster 1.0.0");
expectedException.expectMessage("The local version 6.7.0.9999 is not the same as the cluster 6.6.0.1111");

// Registering a second different version must trigger an exception
appStateCluster.registerSonarQubeVersion("2.0.0");
underTest.registerSonarQubeVersion("6.7.0.9999");
}
}

@Test
public void registerClusterName_throws_MessageException_if_clusterName_is_different() throws Exception {
// Now launch an instance that try to be part of the hzInstance cluster
TestAppSettings settings = newApplicationSettings();

try (ClusterAppStateImpl appStateCluster = new ClusterAppStateImpl(settings)) {
try (ClusterAppStateImpl underTest = new ClusterAppStateImpl(newHzMember())) {
// Register first version
appStateCluster.registerClusterName("goodClusterName");
underTest.getHazelcastMember().getAtomicReference(CLUSTER_NAME).set("goodClusterName");

expectedException.expect(MessageException.class);
expectedException.expectMessage("This node has a cluster name [badClusterName], which does not match [goodClusterName] from the cluster");

// Registering a second different cluster name must trigger an exception
appStateCluster.registerClusterName("badClusterName");
underTest.registerClusterName("badClusterName");
}
}
}

+ 0
- 128
server/sonar-main/src/test/java/org/sonar/application/cluster/ClusterPropertiesTest.java Parādīt failu

@@ -1,128 +0,0 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/

package org.sonar.application.cluster;

import java.util.Arrays;
import java.util.Collections;
import java.util.stream.Stream;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.sonar.application.config.AppSettings;
import org.sonar.application.config.TestAppSettings;

import static org.assertj.core.api.Assertions.assertThat;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_HOSTS;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_PORT;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_TYPE;

public class ClusterPropertiesTest {
@Rule
public ExpectedException expectedException = ExpectedException.none();

private AppSettings appSettings = new TestAppSettings();

@Test
public void test_default_values() throws Exception {
appSettings.getProps().set(CLUSTER_ENABLED, "true");
appSettings.getProps().set(CLUSTER_NODE_TYPE, "application");
ClusterProperties props = new ClusterProperties(appSettings);

assertThat(props.getNetworkInterfaces())
.isEqualTo(Collections.emptyList());
assertThat(props.getPort())
.isEqualTo(9003);
assertThat(props.getHosts())
.isEqualTo(Collections.emptyList());
}

@Test
public void test_port_parameter() {
appSettings.getProps().set(CLUSTER_ENABLED, "true");
appSettings.getProps().set(CLUSTER_NAME, "sonarqube");
appSettings.getProps().set(CLUSTER_NODE_TYPE, "application");

Stream.of("-50", "0", "65536", "128563").forEach(
port -> {
appSettings.getProps().set(CLUSTER_NODE_PORT, port);

ClusterProperties clusterProperties = new ClusterProperties(appSettings);
expectedException.expect(IllegalArgumentException.class);
expectedException.expectMessage(
String.format("Cluster port have been set to %s which is outside the range [1-65535].", port));
clusterProperties.validate();

});
}

@Test
public void test_interfaces_parameter() {
appSettings.getProps().set(CLUSTER_ENABLED, "true");
appSettings.getProps().set(CLUSTER_NAME, "sonarqube");
appSettings.getProps().set(CLUSTER_NODE_HOST, "8.8.8.8"); // This IP belongs to Google
appSettings.getProps().set(CLUSTER_NODE_TYPE, "application");

ClusterProperties clusterProperties = new ClusterProperties(appSettings);
expectedException.expect(IllegalArgumentException.class);
expectedException.expectMessage(
String.format("Interface %s is not available on this machine.", "8.8.8.8"));
clusterProperties.validate();
}

@Test
public void validate_does_not_fail_if_cluster_enabled_and_name_specified() {
appSettings.getProps().set(CLUSTER_ENABLED, "true");
appSettings.getProps().set(CLUSTER_NAME, "sonarqube");
appSettings.getProps().set(CLUSTER_NODE_TYPE, "application");

ClusterProperties clusterProperties = new ClusterProperties(appSettings);
clusterProperties.validate();
}

@Test
public void test_members() {
appSettings.getProps().set(CLUSTER_ENABLED, "true");
appSettings.getProps().set(CLUSTER_NAME, "sonarqube");
appSettings.getProps().set(CLUSTER_NODE_TYPE, "application");

assertThat(
new ClusterProperties(appSettings).getHosts()).isEqualTo(
Collections.emptyList());

appSettings.getProps().set(CLUSTER_HOSTS, "192.168.1.1");
assertThat(
new ClusterProperties(appSettings).getHosts()).isEqualTo(
Arrays.asList("192.168.1.1:9003"));

appSettings.getProps().set(CLUSTER_HOSTS, "192.168.1.2:5501");
assertThat(
new ClusterProperties(appSettings).getHosts()).containsExactlyInAnyOrder(
"192.168.1.2:5501");

appSettings.getProps().set(CLUSTER_HOSTS, "192.168.1.2:5501,192.168.1.1");
assertThat(
new ClusterProperties(appSettings).getHosts()).containsExactlyInAnyOrder(
"192.168.1.2:5501", "192.168.1.1:9003");
}
}

+ 0
- 353
server/sonar-main/src/test/java/org/sonar/application/cluster/HazelcastClusterTest.java Parādīt failu

@@ -1,353 +0,0 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.application.cluster;

import ch.qos.logback.classic.LoggerContext;
import ch.qos.logback.classic.spi.ILoggingEvent;
import ch.qos.logback.core.AppenderBase;
import com.hazelcast.core.HazelcastInstance;
import com.hazelcast.core.ItemEvent;
import com.hazelcast.core.ItemListener;
import com.hazelcast.core.ReplicatedMap;
import java.net.InetAddress;
import java.util.AbstractMap;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.junit.After;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.DisableOnDebug;
import org.junit.rules.ExpectedException;
import org.junit.rules.TestRule;
import org.junit.rules.Timeout;
import org.slf4j.LoggerFactory;
import org.sonar.process.NetworkUtils;
import org.sonar.application.AppStateListener;
import org.sonar.application.config.TestAppSettings;
import org.sonar.process.cluster.ClusterObjectKeys;
import org.sonar.process.ProcessId;

import static java.lang.String.format;
import static junit.framework.TestCase.fail;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.within;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.timeout;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.sonar.application.cluster.HazelcastClusterTestHelper.closeAllHazelcastClients;
import static org.sonar.application.cluster.HazelcastClusterTestHelper.createHazelcastClient;
import static org.sonar.application.cluster.HazelcastClusterTestHelper.newApplicationSettings;
import static org.sonar.application.cluster.HazelcastClusterTestHelper.newSearchSettings;
import static org.sonar.process.cluster.ClusterObjectKeys.LEADER;
import static org.sonar.process.cluster.ClusterObjectKeys.OPERATIONAL_PROCESSES;
import static org.sonar.process.cluster.ClusterObjectKeys.SONARQUBE_VERSION;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_HOSTS;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_PORT;

public class HazelcastClusterTest {
@Rule
public TestRule safeguardTimeout = new DisableOnDebug(Timeout.seconds(60));

@Rule
public ExpectedException expectedException = ExpectedException.none();

@After
public void closeHazelcastClients() {
closeAllHazelcastClients();
}

@Test
public void test_two_tryToLockWebLeader_must_return_true() {
ClusterProperties clusterProperties = new ClusterProperties(newApplicationSettings());
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
assertThat(hzCluster.tryToLockWebLeader()).isEqualTo(true);
assertThat(hzCluster.tryToLockWebLeader()).isEqualTo(false);
}
}

@Test
public void when_another_process_locked_webleader_tryToLockWebLeader_must_return_false() {
ClusterProperties clusterProperties = new ClusterProperties(newApplicationSettings());
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
HazelcastInstance hzInstance = createHazelcastClient(hzCluster);
hzInstance.getAtomicReference(LEADER).set("aaaa");
assertThat(hzCluster.tryToLockWebLeader()).isEqualTo(false);
}
}

@Test
public void when_no_leader_getLeaderHostName_must_return_NO_LEADER() {
ClusterProperties clusterProperties = new ClusterProperties(newApplicationSettings());
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
assertThat(hzCluster.getLeaderHostName()).isEmpty();
}
}

@Test
public void when_no_leader_getLeaderHostName_must_return_the_hostname() {
ClusterProperties clusterProperties = new ClusterProperties(newApplicationSettings());
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
assertThat(hzCluster.tryToLockWebLeader()).isTrue();
assertThat(hzCluster.getLeaderHostName().get()).isEqualTo(
format("%s (%s)", NetworkUtils.INSTANCE.getHostname(), NetworkUtils.INSTANCE.getIPAddresses()));
}
}

@Test
public void members_must_be_empty_when_there_is_no_other_node() {
ClusterProperties clusterProperties = new ClusterProperties(newApplicationSettings());
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
assertThat(hzCluster.getMembers()).isEmpty();
}
}

@Test
public void set_operational_is_writing_to_cluster() {
ClusterProperties clusterProperties = new ClusterProperties(newApplicationSettings());
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
hzCluster.setOperational(ProcessId.ELASTICSEARCH);

assertThat(hzCluster.isOperational(ProcessId.ELASTICSEARCH)).isTrue();
assertThat(hzCluster.isOperational(ProcessId.WEB_SERVER)).isFalse();
assertThat(hzCluster.isOperational(ProcessId.COMPUTE_ENGINE)).isFalse();

// Connect via Hazelcast client to test values
HazelcastInstance hzInstance = createHazelcastClient(hzCluster);
ReplicatedMap<ClusterProcess, Boolean> operationalProcesses = hzInstance.getReplicatedMap(OPERATIONAL_PROCESSES);
assertThat(operationalProcesses)
.containsExactly(new AbstractMap.SimpleEntry<>(new ClusterProcess(hzCluster.getLocalUUID(), ProcessId.ELASTICSEARCH), Boolean.TRUE));
}
}

@Test
public void hazelcast_cluster_name_is_hardcoded_and_not_affected_by_settings() {
TestAppSettings testAppSettings = newApplicationSettings();
testAppSettings.set(CLUSTER_NAME, "a_cluster_");
ClusterProperties clusterProperties = new ClusterProperties(testAppSettings);
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
assertThat(hzCluster.getName()).isEqualTo("sonarqube");
}
}

@Test
public void cluster_must_keep_a_list_of_clients() throws InterruptedException {
TestAppSettings testAppSettings = newApplicationSettings();
testAppSettings.set(CLUSTER_NAME, "a_cluster_");
ClusterProperties clusterProperties = new ClusterProperties(testAppSettings);
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
assertThat(hzCluster.hzInstance.getSet(ClusterObjectKeys.LOCAL_MEMBER_UUIDS)).isEmpty();
HazelcastInstance hzClient = createHazelcastClient(hzCluster);
assertThat(hzCluster.hzInstance.getSet(ClusterObjectKeys.LOCAL_MEMBER_UUIDS)).containsExactly(hzClient.getLocalEndpoint().getUuid());

CountDownLatch latch = new CountDownLatch(1);
hzCluster.hzInstance.getSet(ClusterObjectKeys.LOCAL_MEMBER_UUIDS).addItemListener(new ItemListener<Object>() {
@Override
public void itemAdded(ItemEvent<Object> item) {
}

@Override
public void itemRemoved(ItemEvent<Object> item) {
latch.countDown();
}
}, false);

hzClient.shutdown();
if (latch.await(5, TimeUnit.SECONDS)) {
assertThat(hzCluster.hzInstance.getSet(ClusterObjectKeys.LOCAL_MEMBER_UUIDS).size()).isEqualTo(0);
} else {
fail("The client UUID have not been removed from the Set within 5 seconds' time lapse");
}
}
}

@Test
public void localUUID_must_not_be_empty() {
ClusterProperties clusterProperties = new ClusterProperties(newApplicationSettings());
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
assertThat(hzCluster.getLocalUUID()).isNotEmpty();
}
}

@Test
public void when_a_process_is_set_operational_listener_must_be_triggered() {
ClusterProperties clusterProperties = new ClusterProperties(newApplicationSettings());
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
AppStateListener listener = mock(AppStateListener.class);
hzCluster.addListener(listener);

// ElasticSearch is not operational
assertThat(hzCluster.isOperational(ProcessId.ELASTICSEARCH)).isFalse();

// Simulate a node that set ElasticSearch operational
HazelcastInstance hzInstance = createHazelcastClient(hzCluster);
ReplicatedMap<ClusterProcess, Boolean> operationalProcesses = hzInstance.getReplicatedMap(OPERATIONAL_PROCESSES);
operationalProcesses.put(new ClusterProcess(UUID.randomUUID().toString(), ProcessId.ELASTICSEARCH), Boolean.TRUE);
verify(listener, timeout(20_000)).onAppStateOperational(ProcessId.ELASTICSEARCH);
verifyNoMoreInteractions(listener);

// ElasticSearch is operational
assertThat(hzCluster.isOperational(ProcessId.ELASTICSEARCH)).isTrue();
}
}

@Test
public void registerSonarQubeVersion_publishes_version_on_first_call() {
ClusterProperties clusterProperties = new ClusterProperties(newApplicationSettings());
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
hzCluster.registerSonarQubeVersion("1.0.0.0");

HazelcastInstance hzInstance = createHazelcastClient(hzCluster);
assertThat(hzInstance.getAtomicReference(SONARQUBE_VERSION).get()).isEqualTo("1.0.0.0");
}
}

@Test
public void registerSonarQubeVersion_throws_ISE_if_initial_version_is_different() throws Exception {
ClusterProperties clusterProperties = new ClusterProperties(newApplicationSettings());
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
// Register first version
hzCluster.registerSonarQubeVersion("1.0.0");

expectedException.expect(IllegalStateException.class);
expectedException.expectMessage("The local version 2.0.0 is not the same as the cluster 1.0.0");

// Registering a second different version must trigger an exception
hzCluster.registerSonarQubeVersion("2.0.0");
}
}

@Test
public void simulate_network_cluster() throws InterruptedException {
TestAppSettings settings = newApplicationSettings();
settings.set(CLUSTER_NODE_HOST, InetAddress.getLoopbackAddress().getHostAddress());
AppStateListener listener = mock(AppStateListener.class);

try (ClusterAppStateImpl appStateCluster = new ClusterAppStateImpl(settings)) {
appStateCluster.addListener(listener);

HazelcastInstance hzInstance = createHazelcastClient(appStateCluster.getHazelcastCluster());
String uuid = UUID.randomUUID().toString();
ReplicatedMap<ClusterProcess, Boolean> replicatedMap = hzInstance.getReplicatedMap(OPERATIONAL_PROCESSES);
// process is not up yet --> no events are sent to listeners
replicatedMap.put(
new ClusterProcess(uuid, ProcessId.ELASTICSEARCH),
Boolean.FALSE);

// process is up yet --> notify listeners
replicatedMap.replace(
new ClusterProcess(uuid, ProcessId.ELASTICSEARCH),
Boolean.TRUE);

// should be called only once
verify(listener, timeout(20_000)).onAppStateOperational(ProcessId.ELASTICSEARCH);
verifyNoMoreInteractions(listener);

hzInstance.shutdown();
}
}

@Test
public void hazelcast_must_log_through_sl4fj() {
MemoryAppender<ILoggingEvent> memoryAppender = new MemoryAppender<>();
LoggerContext lc = (LoggerContext) LoggerFactory.getILoggerFactory();
lc.reset();
memoryAppender.setContext(lc);
memoryAppender.start();
lc.getLogger("com.hazelcast").addAppender(memoryAppender);

try (ClusterAppStateImpl appStateCluster = new ClusterAppStateImpl(newApplicationSettings())) {
}

assertThat(memoryAppender.events).isNotEmpty();
memoryAppender.events.stream().forEach(
e -> assertThat(e.getLoggerName()).startsWith("com.hazelcast"));
}

@Test
public void getClusterTime_returns_time_of_cluster() {
ClusterProperties clusterProperties = new ClusterProperties(newApplicationSettings());
try (HazelcastCluster hzCluster = HazelcastCluster.create(clusterProperties)) {
assertThat(hzCluster.getHazelcastClient().getClusterTime())
.isCloseTo(hzCluster.hzInstance.getCluster().getClusterTime(), within(1000L));
}
}

@Test
public void removing_the_last_application_node_must_clear_web_leader() throws InterruptedException {
try (ClusterAppStateImpl appStateCluster = new ClusterAppStateImpl(newSearchSettings())) {
TestAppSettings appSettings = newApplicationSettings();
appSettings.set(CLUSTER_HOSTS, appStateCluster.getHazelcastCluster().getLocalEndPoint());
appSettings.set(CLUSTER_NODE_PORT, "9004");
ClusterProperties clusterProperties = new ClusterProperties(appSettings);
// Simulate a connection from an application node
HazelcastCluster appNode = HazelcastCluster.create(clusterProperties);
appNode.tryToLockWebLeader();
appNode.setOperational(ProcessId.WEB_SERVER);
appNode.setOperational(ProcessId.COMPUTE_ENGINE);
appNode.registerSonarQubeVersion("6.6.0.22999");

assertThat(appStateCluster.getLeaderHostName()).isPresent();
assertThat(appStateCluster.isOperational(ProcessId.WEB_SERVER, false)).isTrue();
assertThat(appStateCluster.isOperational(ProcessId.COMPUTE_ENGINE, false)).isTrue();
assertThat(appStateCluster.getHazelcastCluster().getSonarQubeVersion()).isEqualTo("6.6.0.22999");

// Shutdown the node
appNode.close();

// Propagation of all information take some time, let's wait 5s maximum
int counter = 10;
while (appStateCluster.getHazelcastCluster().getSonarQubeVersion() != null && counter > 0) {
Thread.sleep(500);
counter--;
}

assertThat(appStateCluster.getLeaderHostName()).isNotPresent();
assertThat(appStateCluster.isOperational(ProcessId.WEB_SERVER, false)).isFalse();
assertThat(appStateCluster.isOperational(ProcessId.COMPUTE_ENGINE, false)).isFalse();
assertThat(appStateCluster.getHazelcastCluster().getSonarQubeVersion()).isNull();
}

}

@Test
public void configuration_tweaks_of_hazelcast_must_be_present() {
try (HazelcastCluster hzCluster = HazelcastCluster.create(new ClusterProperties(newApplicationSettings()))) {
assertThat(hzCluster.hzInstance.getConfig().getProperty("hazelcast.tcp.join.port.try.count")).isEqualTo("10");
assertThat(hzCluster.hzInstance.getConfig().getProperty("hazelcast.phone.home.enabled")).isEqualTo("false");
assertThat(hzCluster.hzInstance.getConfig().getProperty("hazelcast.logging.type")).isEqualTo("slf4j");
assertThat(hzCluster.hzInstance.getConfig().getProperty("hazelcast.socket.bind.any")).isEqualTo("false");
}
}

private class MemoryAppender<E> extends AppenderBase<E> {
private final List<E> events = new ArrayList();

@Override
protected void append(E eventObject) {
events.add(eventObject);
}
}
}

+ 0
- 84
server/sonar-main/src/test/java/org/sonar/application/cluster/HazelcastClusterTestHelper.java Parādīt failu

@@ -1,84 +0,0 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/

package org.sonar.application.cluster;

import com.hazelcast.client.HazelcastClient;
import com.hazelcast.client.config.ClientConfig;
import com.hazelcast.core.HazelcastInstance;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.List;
import org.sonar.application.config.TestAppSettings;

import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_TYPE;

public class HazelcastClusterTestHelper {

// Be careful this test won't work if parallel tests is used
private static final List<HazelcastInstance> HAZELCAST_INSTANCES = new ArrayList<>();

static HazelcastInstance createHazelcastClient(HazelcastCluster hzCluster) {
ClientConfig clientConfig = new ClientConfig();
InetSocketAddress socketAddress = (InetSocketAddress) hzCluster.hzInstance.getLocalEndpoint().getSocketAddress();

clientConfig.getNetworkConfig().getAddresses().add(
String.format("%s:%d",
socketAddress.getHostString(),
socketAddress.getPort()));
clientConfig.getGroupConfig().setName(hzCluster.getName());
HazelcastInstance hazelcastInstance = HazelcastClient.newHazelcastClient(clientConfig);
HAZELCAST_INSTANCES.add(hazelcastInstance);
return hazelcastInstance;
}

static HazelcastInstance createHazelcastClient(ClusterAppStateImpl appStateCluster) {
return createHazelcastClient(appStateCluster.getHazelcastCluster());
}

static void closeAllHazelcastClients() {
HAZELCAST_INSTANCES.stream().forEach(
hz -> {
try {
hz.shutdown();
} catch (Exception ex) {
// Ignore it
}
});
}

static TestAppSettings newApplicationSettings() {
TestAppSettings settings = new TestAppSettings();
settings.set(CLUSTER_ENABLED, "true");
settings.set(CLUSTER_NAME, "sonarqube");
settings.set(CLUSTER_NODE_TYPE, "application");
return settings;
}

static TestAppSettings newSearchSettings() {
TestAppSettings settings = new TestAppSettings();
settings.set(CLUSTER_ENABLED, "true");
settings.set(CLUSTER_NAME, "sonarqube");
settings.set(CLUSTER_NODE_TYPE, "search");
return settings;
}
}

+ 48
- 0
server/sonar-main/src/test/java/org/sonar/application/cluster/HazelcastTesting.java Parādīt failu

@@ -0,0 +1,48 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.application.cluster;

import java.net.InetAddress;
import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessId;
import org.sonar.process.cluster.NodeType;
import org.sonar.process.cluster.hz.HazelcastMember;
import org.sonar.process.cluster.hz.HazelcastMemberBuilder;

public class HazelcastTesting {

private HazelcastTesting() {
// do not instantiate
}

public static HazelcastMember newHzMember() {
// use loopback for support of offline builds
InetAddress loopback = InetAddress.getLoopbackAddress();

return new HazelcastMemberBuilder()
.setNodeType(NodeType.APPLICATION)
.setProcessId(ProcessId.COMPUTE_ENGINE)
.setClusterName("foo")
.setNodeName("bar")
.setPort(NetworkUtils.INSTANCE.getNextAvailablePort(loopback))
.setNetworkInterface(loopback.getHostAddress())
.build();
}
}

server/sonar-process/src/test/java/org/sonar/process/cluster/health/DelegateHealthStateRefresherExecutorServiceTest.java → server/sonar-main/src/test/java/org/sonar/application/cluster/health/DelegateHealthStateRefresherExecutorServiceTest.java Parādīt failu

@@ -17,7 +17,7 @@
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.process.cluster.health;
package org.sonar.application.cluster.health;

import java.util.Collection;
import java.util.Random;

server/sonar-main/src/test/java/org/sonar/application/cluster/SearchNodeHealthProviderTest.java → server/sonar-main/src/test/java/org/sonar/application/cluster/health/SearchNodeHealthProviderTest.java Parādīt failu

@@ -17,7 +17,7 @@
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.application.cluster;
package org.sonar.application.cluster.health;

import java.util.Properties;
import java.util.Random;
@@ -25,10 +25,11 @@ import javax.annotation.Nullable;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.sonar.application.cluster.ClusterAppState;
import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessId;
import org.sonar.process.Props;
import org.sonar.process.cluster.ClusterProperties;
import org.sonar.process.ProcessProperties;
import org.sonar.process.cluster.health.NodeHealth;

import static java.lang.String.valueOf;
@@ -36,9 +37,9 @@ import static org.apache.commons.lang.RandomStringUtils.randomAlphanumeric;
import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_PORT;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_PORT;

public class SearchNodeHealthProviderTest {
@Rule
@@ -62,7 +63,7 @@ public class SearchNodeHealthProviderTest {
@Test
public void constructor_throws_NPE_if_NetworkUtils_getHostname_returns_null_and_property_is_not_set() {
Properties properties = new Properties();
properties.put(ClusterProperties.CLUSTER_NODE_NAME, randomAlphanumeric(3));
properties.put(ProcessProperties.CLUSTER_NODE_NAME, randomAlphanumeric(3));
Props props = new Props(properties);

expectedException.expect(NullPointerException.class);
@@ -73,7 +74,7 @@ public class SearchNodeHealthProviderTest {
@Test
public void constructor_throws_IAE_if_property_node_port_is_not_set() {
Properties properties = new Properties();
properties.put(ClusterProperties.CLUSTER_NODE_NAME, randomAlphanumeric(3));
properties.put(ProcessProperties.CLUSTER_NODE_NAME, randomAlphanumeric(3));
when(networkUtils.getHostname()).thenReturn(randomAlphanumeric(34));
Props props = new Props(properties);

@@ -87,8 +88,8 @@ public class SearchNodeHealthProviderTest {
public void constructor_throws_FormatException_if_property_node_port_is_not_an_integer() {
String port = randomAlphanumeric(3);
Properties properties = new Properties();
properties.put(ClusterProperties.CLUSTER_NODE_NAME, randomAlphanumeric(3));
properties.put(ClusterProperties.CLUSTER_NODE_PORT, port);
properties.put(ProcessProperties.CLUSTER_NODE_NAME, randomAlphanumeric(3));
properties.put(ProcessProperties.CLUSTER_NODE_PORT, port);
when(networkUtils.getHostname()).thenReturn(randomAlphanumeric(34));
Props props = new Props(properties);


+ 65
- 175
server/sonar-main/src/test/java/org/sonar/application/config/ClusterSettingsLoopbackTest.java Parādīt failu

@@ -20,207 +20,97 @@

package org.sonar.application.config;

import java.net.NetworkInterface;
import java.net.InetAddress;
import java.net.SocketException;
import java.util.Collections;
import java.util.Enumeration;
import java.util.Optional;
import org.hamcrest.CoreMatchers;
import org.junit.Before;
import org.junit.Rule;
import org.junit.experimental.theories.DataPoints;
import org.junit.experimental.theories.FromDataPoints;
import org.junit.experimental.theories.Theories;
import org.junit.experimental.theories.Theory;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.runner.RunWith;
import org.sonar.process.MessageException;

import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_HOSTS;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_TYPE;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_SEARCH_HOSTS;
import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessProperties;

import static org.junit.Assume.assumeThat;
import static org.mockito.Mockito.spy;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_HOSTS;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_TYPE;
import static org.sonar.process.ProcessProperties.CLUSTER_SEARCH_HOSTS;
import static org.sonar.process.ProcessProperties.JDBC_URL;
import static org.sonar.process.ProcessProperties.SEARCH_HOST;

@RunWith(Theories.class)
public class ClusterSettingsLoopbackTest {

private TestAppSettings settings;
private static final String LOOPBACK_FORBIDDEN = " must not be a loopback address";
private static final String NOT_LOCAL_ADDRESS = " is not a local address";
private static final String NOT_RESOLVABLE = " cannot be resolved";

@DataPoints("parameter")
public static final ValueAndResult[] VALID_SINGLE_IP = {
// Valid IPs
new ValueAndResult("1.2.3.4", NOT_LOCAL_ADDRESS),
new ValueAndResult("1.2.3.4:9001", NOT_LOCAL_ADDRESS),
new ValueAndResult("2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccb", NOT_LOCAL_ADDRESS),
new ValueAndResult("[2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccb]:9001", NOT_LOCAL_ADDRESS),

// Valid Name
new ValueAndResult("www.sonarqube.org", NOT_LOCAL_ADDRESS),
new ValueAndResult("www.google.fr", NOT_LOCAL_ADDRESS),
new ValueAndResult("www.google.com, www.sonarsource.com, wwww.sonarqube.org", NOT_LOCAL_ADDRESS),

new ValueAndResult("...", NOT_RESOLVABLE),
new ValueAndResult("භඦආ\uD801\uDC8C\uD801\uDC8B", NOT_RESOLVABLE),

// Valide IPs List
new ValueAndResult("1.2.3.4,2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccb", NOT_LOCAL_ADDRESS),
new ValueAndResult("1.2.3.4:9001,[2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccb]:9001", NOT_LOCAL_ADDRESS),
new ValueAndResult("2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccb,1.2.3.4:9001", NOT_LOCAL_ADDRESS),
new ValueAndResult("[2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccb]:9001,2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccc", NOT_LOCAL_ADDRESS),

// Loopback IPs
new ValueAndResult("localhost", LOOPBACK_FORBIDDEN),
new ValueAndResult("127.0.0.1", LOOPBACK_FORBIDDEN),
new ValueAndResult("127.1.1.1", LOOPBACK_FORBIDDEN),
new ValueAndResult("127.243.136.241", LOOPBACK_FORBIDDEN),
new ValueAndResult("::1", LOOPBACK_FORBIDDEN),
new ValueAndResult("0:0:0:0:0:0:0:1", LOOPBACK_FORBIDDEN),
new ValueAndResult("localhost:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("127.0.0.1:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("127.1.1.1:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("127.243.136.241:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("[::1]:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("[0:0:0:0:0:0:0:1]:9001", LOOPBACK_FORBIDDEN),

// Loopback IPs list
new ValueAndResult("127.0.0.1,192.168.11.25", LOOPBACK_FORBIDDEN),
new ValueAndResult("192.168.11.25,127.1.1.1", LOOPBACK_FORBIDDEN),
new ValueAndResult("2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccb,0:0:0:0:0:0:0:1", LOOPBACK_FORBIDDEN),
new ValueAndResult("0:0:0:0:0:0:0:1,2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccb", LOOPBACK_FORBIDDEN),
new ValueAndResult("2a01:e34:ef1f:dbb0:c3f6:a978:c5c0:9ccb,::1", LOOPBACK_FORBIDDEN),
new ValueAndResult("::1,2a01:e34:ef1f:dbb0:c3f6:a978:c5c0:9ccb", LOOPBACK_FORBIDDEN),
new ValueAndResult("::1,2a01:e34:ef1f:dbb0:c3f6:a978:c5c0:9ccb,2a01:e34:ef1f:dbb0:b3f6:a978:c5c0:9ccb", LOOPBACK_FORBIDDEN),
new ValueAndResult("localhost:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("127.0.0.1:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("127.1.1.1:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("127.243.136.241:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("[::1]:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("[0:0:0:0:0:0:0:1]:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("127.0.0.1,192.168.11.25:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("192.168.11.25:9001,127.1.1.1:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccb,[0:0:0:0:0:0:0:1]:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("[0:0:0:0:0:0:0:1]:9001,[2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccb]:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("[2a01:e34:ef1f:dbb0:c3f6:a978:c5c0:9ccb]:9001,[::1]:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("[::1]:9001,[2a01:e34:ef1f:dbb0:c3f6:a978:c5c0:9ccb]:9001", LOOPBACK_FORBIDDEN),
new ValueAndResult("[::1]:9001,[2a01:e34:ef1f:dbb0:c3f6:a978:c5c0:9ccb]:9001,[2a01:e34:ef1f:dbb0:b3f6:a978:c5c0:9ccb]:9001", LOOPBACK_FORBIDDEN)
};

@DataPoints("key")
public static final Key[] KEYS = {
new Key(SEARCH_HOST, false, false),
new Key(CLUSTER_NODE_HOST, true, false),
new Key(CLUSTER_SEARCH_HOSTS, true, true),
new Key(CLUSTER_HOSTS, true, true)
};

@DataPoints("unresolvable_hosts")
public static final String[] UNRESOLVABLE_HOSTS = {
};

@Rule
public ExpectedException expectedException = ExpectedException.none();

private InetAddress loopback = InetAddress.getLoopbackAddress();
private InetAddress nonLoopbackLocal;
private NetworkUtils network = spy(NetworkUtils.INSTANCE);

@Before
public void resetSettings() {
settings = getClusterSettings();
}
public void setUp() throws Exception {
Optional<InetAddress> opt = network.getLocalNonLoopbackIpv4Address();
assumeThat(opt.isPresent(), CoreMatchers.is(true));

@Theory
public void accept_throws_MessageException(
@FromDataPoints("key") Key propertyKey,
@FromDataPoints("parameter") ValueAndResult valueAndResult) {
// Skip the test if the value is a list and if the key is not accepting a list
if (settings == null) {
System.out.println("No network found, skipping the test");
return;
}
if ((valueAndResult.isList() && propertyKey.acceptList) || !valueAndResult.isList()) {
settings.set(propertyKey.getKey(), valueAndResult.getValue());

// If the key accept non local IPs there won't be any exception
if (!propertyKey.acceptNonLocal || valueAndResult.getMessage() != NOT_LOCAL_ADDRESS) {
expectedException.expect(MessageException.class);
expectedException.expectMessage(valueAndResult.getMessage());
}

new ClusterSettings().accept(settings.getProps());
}
nonLoopbackLocal = opt.get();
}

private static TestAppSettings getClusterSettings() {
String localAddress = null;
try {
Enumeration<NetworkInterface> nets = NetworkInterface.getNetworkInterfaces();
for (NetworkInterface networkInterface : Collections.list(nets)) {
if (!networkInterface.isLoopback() && networkInterface.isUp()) {
localAddress = networkInterface.getInetAddresses().nextElement().getHostAddress();
}
}
if (localAddress == null) {
return null;
}

} catch (SocketException e) {
return null;
}

TestAppSettings testAppSettings = new TestAppSettings()
.set(CLUSTER_ENABLED, "true")
.set(CLUSTER_NODE_TYPE, "search")
.set(CLUSTER_SEARCH_HOSTS, "192.168.233.1, 192.168.233.2,192.168.233.3")
.set(CLUSTER_HOSTS, "192.168.233.1, 192.168.233.2,192.168.233.3")
.set(SEARCH_HOST, localAddress)
.set(JDBC_URL, "jdbc:mysql://localhost:3306/sonar?useUnicode=true&characterEncoding=utf8&rewriteBatchedStatements=true&useConfigs=maxPerformance");
return testAppSettings;
@Test
public void ClusterSettings_throws_MessageException_if_host_of_search_node_is_loopback() throws Exception {
verifySearchFailureIfLoopback(ProcessProperties.CLUSTER_NODE_HOST);
verifySearchFailureIfLoopback(ProcessProperties.CLUSTER_SEARCH_HOSTS);
verifySearchFailureIfLoopback(ProcessProperties.CLUSTER_HOSTS);
verifySearchFailureIfLoopback(ProcessProperties.SEARCH_HOST);
}

private static class Key {
private final String key;
private final boolean acceptList;
private final boolean acceptNonLocal;

private Key(String key, boolean acceptList, boolean acceptNonLocal) {
this.key = key;
this.acceptList = acceptList;
this.acceptNonLocal = acceptNonLocal;
}

public String getKey() {
return key;
}
@Test
public void ClusterSettings_throws_MessageException_if_host_of_app_node_is_loopback() throws Exception {
verifyAppFailureIfLoopback(ProcessProperties.CLUSTER_NODE_HOST);
verifyAppFailureIfLoopback(ProcessProperties.CLUSTER_SEARCH_HOSTS);
verifyAppFailureIfLoopback(ProcessProperties.CLUSTER_HOSTS);
}

public boolean acceptList() {
return acceptList;
}
private void verifySearchFailureIfLoopback(String propertyKey) throws Exception {
TestAppSettings settings = newSettingsForSearchNode();
verifyFailure(propertyKey, settings);
}

public boolean acceptNonLocal() {
return acceptNonLocal;
}
private void verifyAppFailureIfLoopback(String propertyKey) throws Exception {
TestAppSettings settings = newSettingsForAppNode();
verifyFailure(propertyKey, settings);
}

private static class ValueAndResult {
private final String value;
private final String message;
private void verifyFailure(String propertyKey, TestAppSettings settings) {
settings.set(propertyKey, loopback.getHostAddress());

private ValueAndResult(String value, String message) {
this.value = value;
this.message = message;
}
expectedException.expect(MessageException.class);
expectedException.expectMessage("Property " + propertyKey + " must be a local non-loopback address: " + loopback.getHostAddress());

public String getValue() {
return value;
}
new ClusterSettings(network).accept(settings.getProps());
}

public String getMessage() {
return message;
}
private TestAppSettings newSettingsForAppNode() throws SocketException {
return new TestAppSettings()
.set(CLUSTER_ENABLED, "true")
.set(CLUSTER_NODE_TYPE, "application")
.set(CLUSTER_NODE_HOST, nonLoopbackLocal.getHostAddress())
.set(CLUSTER_HOSTS, nonLoopbackLocal.getHostAddress())
.set(CLUSTER_SEARCH_HOSTS, nonLoopbackLocal.getHostAddress())
.set("sonar.auth.jwtBase64Hs256Secret", "abcde")
.set(JDBC_URL, "jdbc:mysql://localhost:3306/sonar");
}

public boolean isList() {
return value != null && value.contains(",");
}
private TestAppSettings newSettingsForSearchNode() throws SocketException {
return new TestAppSettings()
.set(CLUSTER_ENABLED, "true")
.set(CLUSTER_NODE_TYPE, "search")
.set(CLUSTER_NODE_HOST, nonLoopbackLocal.getHostAddress())
.set(CLUSTER_HOSTS, nonLoopbackLocal.getHostAddress())
.set(CLUSTER_SEARCH_HOSTS, nonLoopbackLocal.getHostAddress())
.set(SEARCH_HOST, nonLoopbackLocal.getHostAddress());
}
}

+ 54
- 93
server/sonar-main/src/test/java/org/sonar/application/config/ClusterSettingsTest.java Parādīt failu

@@ -19,26 +19,27 @@
*/
package org.sonar.application.config;

import java.net.Inet4Address;
import java.net.InetAddress;
import java.net.NetworkInterface;
import java.net.SocketException;
import java.util.Collections;
import java.util.Enumeration;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.sonar.process.MessageException;
import org.sonar.process.NetworkUtils;

import static java.lang.String.format;
import static org.assertj.core.api.Assertions.assertThat;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_HOSTS;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_TYPE;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_SEARCH_HOSTS;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import static org.sonar.process.ProcessId.COMPUTE_ENGINE;
import static org.sonar.process.ProcessId.ELASTICSEARCH;
import static org.sonar.process.ProcessId.WEB_SERVER;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_HOSTS;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_TYPE;
import static org.sonar.process.ProcessProperties.CLUSTER_SEARCH_HOSTS;
import static org.sonar.process.ProcessProperties.JDBC_URL;
import static org.sonar.process.ProcessProperties.SEARCH_HOST;

@@ -47,8 +48,17 @@ public class ClusterSettingsTest {
@Rule
public ExpectedException expectedException = ExpectedException.none();

private InetAddress nonLoopbackLocal = InetAddress.getLoopbackAddress();
private NetworkUtils network = spy(NetworkUtils.INSTANCE);

@Before
public void setUp() throws Exception {
when(network.isLocalInetAddress(nonLoopbackLocal)).thenReturn(true);
when(network.isLoopbackInetAddress(nonLoopbackLocal)).thenReturn(false);
}

@Test
public void test_isClusterEnabled() {
public void test_isClusterEnabled() throws Exception {
TestAppSettings settings = newSettingsForAppNode().set(CLUSTER_ENABLED, "true");
assertThat(ClusterSettings.isClusterEnabled(settings)).isTrue();

@@ -68,19 +78,7 @@ public class ClusterSettingsTest {
}

@Test
public void getEnabledProcesses_fails_if_no_node_type_is_set_for_a_cluster_node() {
TestAppSettings settings = new TestAppSettings();
settings.set(CLUSTER_ENABLED, "true");
settings.set(CLUSTER_NODE_TYPE, "foo");

expectedException.expect(IllegalArgumentException.class);
expectedException.expectMessage("Invalid value for [sonar.cluster.node.type]: [foo]");

ClusterSettings.getEnabledProcesses(settings);
}

@Test
public void getEnabledProcesses_returns_configured_processes_in_cluster_mode() {
public void getEnabledProcesses_returns_configured_processes_in_cluster_mode() throws Exception {
TestAppSettings settings = newSettingsForAppNode();
assertThat(ClusterSettings.getEnabledProcesses(settings)).containsOnly(COMPUTE_ENGINE, WEB_SERVER);

@@ -94,9 +92,9 @@ public class ClusterSettingsTest {
settings.set(CLUSTER_ENABLED, "true");

expectedException.expect(MessageException.class);
expectedException.expectMessage("Property [sonar.cluster.node.type] is mandatory");
expectedException.expectMessage("Property sonar.cluster.node.type is mandatory");

new ClusterSettings().accept(settings.getProps());
new ClusterSettings(network).accept(settings.getProps());
}

@Test
@@ -106,32 +104,20 @@ public class ClusterSettingsTest {
settings.set(CLUSTER_NODE_TYPE, "bla");

expectedException.expect(MessageException.class);
expectedException.expectMessage("Invalid value for property [sonar.cluster.node.type]: [bla], only [application, search] are allowed");
expectedException.expectMessage("Invalid value for property sonar.cluster.node.type: [bla], only [application, search] are allowed");

new ClusterSettings().accept(settings.getProps());
new ClusterSettings(network).accept(settings.getProps());
}

@Test
public void accept_throws_MessageException_if_internal_property_for_web_leader_is_configured() {
public void accept_throws_MessageException_if_internal_property_for_startup_leader_is_configured() throws Exception {
TestAppSettings settings = newSettingsForAppNode();
settings.set("sonar.cluster.web.startupLeader", "true");

expectedException.expect(MessageException.class);
expectedException.expectMessage("Property [sonar.cluster.web.startupLeader] is forbidden");

new ClusterSettings().accept(settings.getProps());
}

@Test
public void accept_throws_MessageException_if_search_enabled_with_loopback() {
TestAppSettings settings = newSettingsForSearchNode();
settings.set(CLUSTER_SEARCH_HOSTS, "192.168.1.1,192.168.1.2");
settings.set(SEARCH_HOST, "::1");

expectedException.expect(MessageException.class);
expectedException.expectMessage("The interface address [::1] of [sonar.search.host] must not be a loopback address");

new ClusterSettings().accept(settings.getProps());
new ClusterSettings(network).accept(settings.getProps());
}

@Test
@@ -141,38 +127,38 @@ public class ClusterSettingsTest {
// this property is supposed to fail if cluster is enabled
settings.set("sonar.cluster.web.startupLeader", "true");

new ClusterSettings().accept(settings.getProps());
new ClusterSettings(network).accept(settings.getProps());
}

@Test
public void accept_throws_MessageException_if_h2_on_application_node() {
public void accept_throws_MessageException_if_h2_on_application_node() throws Exception {
TestAppSettings settings = newSettingsForAppNode();
settings.set("sonar.jdbc.url", "jdbc:h2:mem");

expectedException.expect(MessageException.class);
expectedException.expectMessage("Embedded database is not supported in cluster mode");

new ClusterSettings().accept(settings.getProps());
new ClusterSettings(network).accept(settings.getProps());
}

@Test
public void accept_does_not_verify_h2_on_search_node() {
public void accept_does_not_verify_h2_on_search_node() throws Exception {
TestAppSettings settings = newSettingsForSearchNode();
settings.set("sonar.jdbc.url", "jdbc:h2:mem");

// do not fail
new ClusterSettings().accept(settings.getProps());
new ClusterSettings(network).accept(settings.getProps());
}

@Test
public void accept_throws_MessageException_on_application_node_if_default_jdbc_url() {
public void accept_throws_MessageException_on_application_node_if_default_jdbc_url() throws Exception {
TestAppSettings settings = newSettingsForAppNode();
settings.clearProperty(JDBC_URL);

expectedException.expect(MessageException.class);
expectedException.expectMessage("Embedded database is not supported in cluster mode");

new ClusterSettings().accept(settings.getProps());
new ClusterSettings(network).accept(settings.getProps());
}

@Test
@@ -182,63 +168,56 @@ public class ClusterSettingsTest {
}

@Test
public void isLocalElasticsearchEnabled_returns_true_on_search_node() {
public void isLocalElasticsearchEnabled_returns_true_on_search_node() throws Exception {
TestAppSettings settings = newSettingsForSearchNode();

assertThat(ClusterSettings.isLocalElasticsearchEnabled(settings)).isTrue();
}

@Test
public void isLocalElasticsearchEnabled_returns_true_for_a_application_node() {
public void isLocalElasticsearchEnabled_returns_true_for_a_application_node() throws Exception {
TestAppSettings settings = newSettingsForAppNode();

assertThat(ClusterSettings.isLocalElasticsearchEnabled(settings)).isFalse();
}

@Test
public void accept_throws_MessageException_if_searchHost_is_missing() {
public void accept_throws_MessageException_if_searchHost_is_missing() throws Exception {
TestAppSettings settings = newSettingsForSearchNode();
settings.clearProperty(SEARCH_HOST);
assertThatPropertyIsMandatory(settings, SEARCH_HOST);
}

@Test
public void accept_throws_MessageException_if_searchHost_is_blank() {
public void accept_throws_MessageException_if_searchHost_is_empty() throws Exception {
TestAppSettings settings = newSettingsForSearchNode();
settings.set(SEARCH_HOST, " ");
settings.set(SEARCH_HOST, "");
assertThatPropertyIsMandatory(settings, SEARCH_HOST);
}

@Test
public void accept_throws_MessageException_if_clusterHosts_is_missing() {
public void accept_throws_MessageException_if_clusterHosts_is_missing() throws Exception {
TestAppSettings settings = newSettingsForSearchNode();
settings.clearProperty(CLUSTER_HOSTS);
assertThatPropertyIsMandatory(settings, CLUSTER_HOSTS);
}

@Test
public void accept_throws_MessageException_if_clusterHosts_is_blank() {
TestAppSettings settings = newSettingsForSearchNode();
settings.set(CLUSTER_HOSTS, " ");
assertThatPropertyIsMandatory(settings, CLUSTER_HOSTS);
}

@Test
public void accept_throws_MessageException_if_clusterSearchHosts_is_missing() {
public void accept_throws_MessageException_if_clusterSearchHosts_is_missing() throws Exception {
TestAppSettings settings = newSettingsForSearchNode();
settings.clearProperty(CLUSTER_SEARCH_HOSTS);
assertThatPropertyIsMandatory(settings, CLUSTER_SEARCH_HOSTS);
}

@Test
public void accept_throws_MessageException_if_clusterSearchHosts_is_blank() {
public void accept_throws_MessageException_if_clusterSearchHosts_is_empty() throws Exception {
TestAppSettings settings = newSettingsForSearchNode();
settings.set(CLUSTER_SEARCH_HOSTS, " ");
settings.set(CLUSTER_SEARCH_HOSTS, "");
assertThatPropertyIsMandatory(settings, CLUSTER_SEARCH_HOSTS);
}

@Test
public void accept_throws_MessageException_if_jwt_token_is_not_set_on_application_nodes() {
public void accept_throws_MessageException_if_jwt_token_is_not_set_on_application_nodes() throws Exception {
TestAppSettings settings = newSettingsForAppNode();
settings.clearProperty("sonar.auth.jwtBase64Hs256Secret");
assertThatPropertyIsMandatory(settings, "sonar.auth.jwtBase64Hs256Secret");
@@ -246,47 +225,29 @@ public class ClusterSettingsTest {

private void assertThatPropertyIsMandatory(TestAppSettings settings, String key) {
expectedException.expect(MessageException.class);
expectedException.expectMessage(format("Property [%s] is mandatory", key));
expectedException.expectMessage(format("Property %s is mandatory", key));

new ClusterSettings().accept(settings.getProps());
new ClusterSettings(network).accept(settings.getProps());
}

private static TestAppSettings newSettingsForAppNode() {
private TestAppSettings newSettingsForAppNode() throws SocketException {
return new TestAppSettings()
.set(CLUSTER_ENABLED, "true")
.set(CLUSTER_NODE_TYPE, "application")
.set(CLUSTER_SEARCH_HOSTS, "localhost")
.set(CLUSTER_HOSTS, "192.168.233.1, 192.168.233.2,192.168.233.3")
.set(CLUSTER_NODE_HOST, nonLoopbackLocal.getHostAddress())
.set(CLUSTER_HOSTS, nonLoopbackLocal.getHostAddress())
.set(CLUSTER_SEARCH_HOSTS, nonLoopbackLocal.getHostAddress())
.set("sonar.auth.jwtBase64Hs256Secret", "abcde")
.set(JDBC_URL, "jdbc:mysql://localhost:3306/sonar");
}

private static TestAppSettings newSettingsForSearchNode() {
private TestAppSettings newSettingsForSearchNode() throws SocketException {
return new TestAppSettings()
.set(CLUSTER_ENABLED, "true")
.set(CLUSTER_NODE_TYPE, "search")
.set(CLUSTER_SEARCH_HOSTS, "192.168.233.1")
.set(CLUSTER_HOSTS, "192.168.233.1, 192.168.233.2,192.168.233.3")
.set(SEARCH_HOST, getNonLoopbackIpv4Address().getHostName());
}

private static InetAddress getNonLoopbackIpv4Address() {
try {
Enumeration<NetworkInterface> nets = NetworkInterface.getNetworkInterfaces();
for (NetworkInterface networkInterface : Collections.list(nets)) {
if (!networkInterface.isLoopback() && networkInterface.isUp()) {
Enumeration<InetAddress> inetAddresses = networkInterface.getInetAddresses();
while (inetAddresses.hasMoreElements()) {
InetAddress inetAddress = inetAddresses.nextElement();
if (inetAddress instanceof Inet4Address) {
return inetAddress;
}
}
}
}
} catch (SocketException se) {
throw new RuntimeException("Cannot find a non loopback card required for tests", se);
}
throw new RuntimeException("Cannot find a non loopback card required for tests");
.set(CLUSTER_NODE_HOST, nonLoopbackLocal.getHostAddress())
.set(CLUSTER_HOSTS, nonLoopbackLocal.getHostAddress())
.set(CLUSTER_SEARCH_HOSTS, nonLoopbackLocal.getHostAddress())
.set(SEARCH_HOST, nonLoopbackLocal.getHostAddress());
}
}

+ 10
- 11
server/sonar-main/src/test/java/org/sonar/application/es/EsSettingsTest.java Parādīt failu

@@ -31,7 +31,6 @@ import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import org.sonar.application.logging.ListAppender;
import org.sonar.process.cluster.ClusterProperties;
import org.sonar.process.ProcessProperties;
import org.sonar.process.Props;
import org.sonar.process.System2;
@@ -40,8 +39,8 @@ import static org.apache.commons.lang.RandomStringUtils.randomAlphanumeric;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_SEARCH_HOSTS;
import static org.sonar.process.ProcessProperties.CLUSTER_NAME;
import static org.sonar.process.ProcessProperties.CLUSTER_SEARCH_HOSTS;

public class EsSettingsTest {

@@ -150,9 +149,9 @@ public class EsSettingsTest {
props.set(ProcessProperties.PATH_HOME, homeDir.getAbsolutePath());
props.set(ProcessProperties.PATH_TEMP, temp.newFolder().getAbsolutePath());
props.set(ProcessProperties.PATH_LOGS, temp.newFolder().getAbsolutePath());
props.set(ClusterProperties.CLUSTER_NAME, "sonarqube-1");
props.set(ClusterProperties.CLUSTER_ENABLED, "true");
props.set(ClusterProperties.CLUSTER_NODE_NAME, "node-1");
props.set(ProcessProperties.CLUSTER_NAME, "sonarqube-1");
props.set(ProcessProperties.CLUSTER_ENABLED, "true");
props.set(ProcessProperties.CLUSTER_NODE_NAME, "node-1");

EsSettings esSettings = new EsSettings(props, new EsFileSystem(props), System2.INSTANCE);

@@ -165,8 +164,8 @@ public class EsSettingsTest {
public void test_node_name_default_for_cluster_mode() throws Exception {
File homeDir = temp.newFolder();
Props props = new Props(new Properties());
props.set(ClusterProperties.CLUSTER_NAME, "sonarqube");
props.set(ClusterProperties.CLUSTER_ENABLED, "true");
props.set(ProcessProperties.CLUSTER_NAME, "sonarqube");
props.set(ProcessProperties.CLUSTER_ENABLED, "true");
props.set(ProcessProperties.SEARCH_PORT, "1234");
props.set(ProcessProperties.SEARCH_HOST, "127.0.0.1");
props.set(ProcessProperties.PATH_HOME, homeDir.getAbsolutePath());
@@ -181,8 +180,8 @@ public class EsSettingsTest {
public void test_node_name_default_for_standalone_mode() throws Exception {
File homeDir = temp.newFolder();
Props props = new Props(new Properties());
props.set(ClusterProperties.CLUSTER_NAME, "sonarqube");
props.set(ClusterProperties.CLUSTER_ENABLED, "false");
props.set(ProcessProperties.CLUSTER_NAME, "sonarqube");
props.set(ProcessProperties.CLUSTER_ENABLED, "false");
props.set(ProcessProperties.SEARCH_PORT, "1234");
props.set(ProcessProperties.SEARCH_HOST, "127.0.0.1");
props.set(ProcessProperties.PATH_HOME, homeDir.getAbsolutePath());
@@ -296,7 +295,7 @@ public class EsSettingsTest {
Props props = new Props(new Properties());
ProcessProperties.completeDefaults(props);
props.set(ProcessProperties.PATH_HOME, homeDir.getAbsolutePath());
props.set(ClusterProperties.CLUSTER_ENABLED, Boolean.toString(cluster));
props.set(ProcessProperties.CLUSTER_ENABLED, Boolean.toString(cluster));
return props;
}
}

+ 4
- 0
server/sonar-process/pom.xml Parādīt failu

@@ -16,6 +16,10 @@
</properties>

<dependencies>
<dependency>
<groupId>com.hazelcast</groupId>
<artifactId>hazelcast</artifactId>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>

+ 33
- 0
server/sonar-process/src/main/java/org/sonar/process/NetworkUtils.java Parādīt failu

@@ -19,7 +19,12 @@
*/
package org.sonar.process;

import java.net.Inet4Address;
import java.net.InetAddress;
import java.net.SocketException;
import java.net.UnknownHostException;
import java.util.Optional;
import java.util.function.Predicate;

public interface NetworkUtils {
NetworkUtils INSTANCE = new NetworkUtilsImpl();
@@ -40,4 +45,32 @@ public interface NetworkUtils {
* @return "ipv4_1, ipv4_2"
*/
String getIPAddresses();

/**
* Converts a text representation of an IP address or host name to
* a {@link InetAddress}.
* If text value references an IPv4 or IPv6 address, then DNS is
* not used.
*/
InetAddress toInetAddress(String hostOrAddress) throws UnknownHostException;

boolean isLocalInetAddress(InetAddress address) throws SocketException;

boolean isLoopbackInetAddress(InetAddress address);

/**
* Returns the machine {@link InetAddress} that matches the specified
* predicate. If multiple addresses match then a single one
* is picked in a non deterministic way.
*/
Optional<InetAddress> getLocalInetAddress(Predicate<InetAddress> predicate);

/**
* Returns a local {@link InetAddress} that is IPv4 and not
* loopback. If multiple addresses match then a single one
* is picked in a non deterministic way.
*/
default Optional<InetAddress> getLocalNonLoopbackIpv4Address() {
return getLocalInetAddress(a -> !a.isLoopbackAddress() && a instanceof Inet4Address);
}
}

+ 41
- 6
server/sonar-process/src/main/java/org/sonar/process/NetworkUtilsImpl.java Parādīt failu

@@ -20,6 +20,7 @@
package org.sonar.process;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.net.InetAddresses;
import java.io.IOException;
import java.net.Inet4Address;
import java.net.InetAddress;
@@ -27,17 +28,20 @@ import java.net.NetworkInterface;
import java.net.ServerSocket;
import java.net.SocketException;
import java.net.UnknownHostException;
import java.util.Collections;
import java.util.HashSet;
import java.util.Optional;
import java.util.Set;
import java.util.function.Predicate;
import java.util.stream.Collectors;

import static java.util.Collections.list;
import static org.apache.commons.lang.StringUtils.isBlank;

public final class NetworkUtilsImpl implements NetworkUtils {
public class NetworkUtilsImpl implements NetworkUtils {

private static final Set<Integer> ALREADY_ALLOCATED = new HashSet<>();
private static final int MAX_TRIES = 50;
private static final Set<Integer> PORTS_ALREADY_ALLOCATED = new HashSet<>();
private static final int PORT_MAX_TRIES = 50;

NetworkUtilsImpl() {
// prevent instantiation
@@ -55,10 +59,10 @@ public final class NetworkUtilsImpl implements NetworkUtils {
*/
@VisibleForTesting
static int getNextAvailablePort(InetAddress address, PortAllocator portAllocator) {
for (int i = 0; i < MAX_TRIES; i++) {
for (int i = 0; i < PORT_MAX_TRIES; i++) {
int port = portAllocator.getAvailable(address);
if (isValidPort(port)) {
ALREADY_ALLOCATED.add(port);
PORTS_ALREADY_ALLOCATED.add(port);
return port;
}
}
@@ -66,7 +70,7 @@ public final class NetworkUtilsImpl implements NetworkUtils {
}

private static boolean isValidPort(int port) {
return port > 1023 && !ALREADY_ALLOCATED.contains(port);
return port > 1023 && !PORTS_ALREADY_ALLOCATED.contains(port);
}

static class PortAllocator {
@@ -118,4 +122,35 @@ public final class NetworkUtilsImpl implements NetworkUtils {

return ips;
}

@Override
public InetAddress toInetAddress(String hostOrAddress) throws UnknownHostException {
if (InetAddresses.isInetAddress(hostOrAddress)) {
return InetAddresses.forString(hostOrAddress);
}
return InetAddress.getByName(hostOrAddress);
}

@Override
public boolean isLocalInetAddress(InetAddress address) throws SocketException {
return NetworkInterface.getByInetAddress(address) != null ;
}

@Override
public boolean isLoopbackInetAddress(InetAddress address) {
return address.isLoopbackAddress();
}

@Override
public Optional<InetAddress> getLocalInetAddress(Predicate<InetAddress> predicate) {
try {
return Collections.list(NetworkInterface.getNetworkInterfaces()).stream()
.flatMap(ni -> Collections.list(ni.getInetAddresses()).stream())
.filter(a -> a.getHostAddress() != null)
.filter(predicate)
.findFirst();
} catch (SocketException e) {
throw new IllegalStateException("Can not retrieve network interfaces", e);
}
}
}

+ 28
- 15
server/sonar-process/src/main/java/org/sonar/process/ProcessProperties.java Parādīt failu

@@ -23,8 +23,7 @@ import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.Map;
import java.util.Properties;

import static org.sonar.process.cluster.ClusterProperties.putClusterDefaults;
import java.util.UUID;

/**
* Constants shared by search, web server and app processes.
@@ -76,6 +75,17 @@ public class ProcessProperties {
public static final String HTTP_PROXY_USER = "http.proxyUser";
public static final String HTTP_PROXY_PASSWORD = "http.proxyPassword";

public static final String CLUSTER_ENABLED = "sonar.cluster.enabled";
public static final String CLUSTER_NODE_TYPE = "sonar.cluster.node.type";
public static final String CLUSTER_SEARCH_HOSTS = "sonar.cluster.search.hosts";
public static final String CLUSTER_HOSTS = "sonar.cluster.hosts";
public static final String CLUSTER_NODE_PORT = "sonar.cluster.node.port";
public static final int CLUSTER_NODE_PORT_DEFAULT_VALUE = 9003;
public static final String CLUSTER_NODE_HOST = "sonar.cluster.node.host";
public static final String CLUSTER_NODE_NAME = "sonar.cluster.node.name";
public static final String CLUSTER_NAME = "sonar.cluster.name";
public static final String CLUSTER_WEB_STARTUP_LEADER = "sonar.cluster.web.startupLeader";

private ProcessProperties() {
// only static stuff
}
@@ -89,18 +99,6 @@ public class ProcessProperties {
fixPortIfZero(props, SEARCH_HOST, SEARCH_PORT);
}

private static void fixPortIfZero(Props props, String addressPropertyKey, String portPropertyKey) {
String port = props.value(portPropertyKey);
if ("0".equals(port)) {
String address = props.nonNullValue(addressPropertyKey);
try {
props.set(portPropertyKey, String.valueOf(NetworkUtils.INSTANCE.getNextAvailablePort(InetAddress.getByName(address))));
} catch (UnknownHostException e) {
throw new IllegalStateException("Cannot resolve address [" + address + "] set by property [" + addressPropertyKey + "]", e);
}
}
}

public static Properties defaults() {
Properties defaults = new Properties();
defaults.put(SEARCH_HOST, InetAddress.getLoopbackAddress().getHostAddress());
@@ -124,8 +122,23 @@ public class ProcessProperties {
defaults.put(JDBC_MIN_EVICTABLE_IDLE_TIME_MILLIS, "600000");
defaults.put(JDBC_TIME_BETWEEN_EVICTION_RUNS_MILLIS, "30000");

putClusterDefaults(defaults);
defaults.put(CLUSTER_ENABLED, "false");
defaults.put(CLUSTER_NAME, "sonarqube");
defaults.put(CLUSTER_NODE_PORT, Integer.toString(CLUSTER_NODE_PORT_DEFAULT_VALUE));
defaults.put(CLUSTER_NODE_NAME, "sonarqube-" + UUID.randomUUID().toString());

return defaults;
}

private static void fixPortIfZero(Props props, String addressPropertyKey, String portPropertyKey) {
String port = props.value(portPropertyKey);
if ("0".equals(port)) {
String address = props.nonNullValue(addressPropertyKey);
try {
props.set(portPropertyKey, String.valueOf(NetworkUtils.INSTANCE.getNextAvailablePort(InetAddress.getByName(address))));
} catch (UnknownHostException e) {
throw new IllegalStateException("Cannot resolve address [" + address + "] set by property [" + addressPropertyKey + "]", e);
}
}
}
}

+ 0
- 56
server/sonar-process/src/main/java/org/sonar/process/cluster/ClusterProperties.java Parādīt failu

@@ -1,56 +0,0 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.process.cluster;

import java.util.Properties;
import java.util.UUID;

import static java.lang.String.valueOf;

public final class ClusterProperties {
public static final String CLUSTER_ENABLED = "sonar.cluster.enabled";
public static final String CLUSTER_NODE_TYPE = "sonar.cluster.node.type";
public static final String CLUSTER_SEARCH_HOSTS = "sonar.cluster.search.hosts";
public static final String CLUSTER_HOSTS = "sonar.cluster.hosts";
public static final String CLUSTER_NODE_PORT = "sonar.cluster.node.port";
public static final String CLUSTER_NODE_HOST = "sonar.cluster.node.host";
public static final String CLUSTER_NODE_NAME = "sonar.cluster.node.name";
public static final String CLUSTER_NAME = "sonar.cluster.name";
public static final String HAZELCAST_LOG_LEVEL = "sonar.log.level.app.hazelcast";
public static final String CLUSTER_WEB_LEADER = "sonar.cluster.web.startupLeader";
// Internal property used by sonar-application to share the local endpoint of Hazelcast
public static final String CLUSTER_LOCALENDPOINT = "sonar.cluster.hazelcast.localEndPoint";
// Internal property used by sonar-application to share the local UUID of the Hazelcast member
public static final String CLUSTER_MEMBERUUID = "sonar.cluster.hazelcast.memberUUID";

private ClusterProperties() {
// prevents instantiation
}

public static void putClusterDefaults(Properties properties) {
properties.put(CLUSTER_ENABLED, valueOf(false));
properties.put(CLUSTER_NAME, "sonarqube");
properties.put(CLUSTER_NODE_HOST, "");
properties.put(CLUSTER_HOSTS, "");
properties.put(CLUSTER_NODE_PORT, "9003");
properties.put(CLUSTER_NODE_NAME, "sonarqube-" + UUID.randomUUID().toString());
properties.put(HAZELCAST_LOG_LEVEL, "WARN");
}
}

+ 1
- 2
server/sonar-process/src/main/java/org/sonar/process/cluster/NodeType.java Parādīt failu

@@ -20,7 +20,6 @@
package org.sonar.process.cluster;

import static java.util.Arrays.stream;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_TYPE;

public enum NodeType {
APPLICATION("application"), SEARCH("search");
@@ -39,7 +38,7 @@ public enum NodeType {
return stream(values())
.filter(t -> nodeType.equals(t.value))
.findFirst()
.orElseThrow(() -> new IllegalArgumentException("Invalid value for [" + CLUSTER_NODE_TYPE + "]: [" + nodeType + "]"));
.orElseThrow(() -> new IllegalArgumentException("Invalid value: " + nodeType));
}

public static boolean isValid(String nodeType) {

+ 1
- 1
server/sonar-process/src/main/java/org/sonar/process/cluster/health/NodeHealthProvider.java Parādīt failu

@@ -21,7 +21,7 @@ package org.sonar.process.cluster.health;

public interface NodeHealthProvider {
/**
* Returns the {@link NodeHealth} for the current SonarQube instance.
* Returns the {@link NodeHealth} for the current SonarQube node.
*
* <p>Implementation must support being called very frequently and from concurrent threads</p>
*/

+ 10
- 10
server/sonar-process/src/main/java/org/sonar/process/cluster/health/SharedHealthStateImpl.java Parādīt failu

@@ -27,8 +27,8 @@ import java.util.function.Predicate;
import java.util.stream.Collectors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.sonar.process.cluster.ClusterObjectKeys;
import org.sonar.process.cluster.HazelcastClient;
import org.sonar.process.cluster.hz.HazelcastMember;
import org.sonar.process.cluster.hz.HazelcastObjects;

import static java.util.Objects.requireNonNull;

@@ -36,10 +36,10 @@ public class SharedHealthStateImpl implements SharedHealthState {
private static final Logger LOG = LoggerFactory.getLogger(SharedHealthStateImpl.class);
private static final int TIMEOUT_30_SECONDS = 30 * 1000;

private final HazelcastClient hazelcastClient;
private final HazelcastMember hzMember;

public SharedHealthStateImpl(HazelcastClient hazelcastClient) {
this.hazelcastClient = hazelcastClient;
public SharedHealthStateImpl(HazelcastMember hzMember) {
this.hzMember = hzMember;
}

@Override
@@ -50,13 +50,13 @@ public class SharedHealthStateImpl implements SharedHealthState {
if (LOG.isTraceEnabled()) {
LOG.trace("Reading {} and adding {}", new HashMap<>(sqHealthState), nodeHealth);
}
sqHealthState.put(hazelcastClient.getUUID(), new TimestampedNodeHealth(nodeHealth, hazelcastClient.getClusterTime()));
sqHealthState.put(hzMember.getUuid(), new TimestampedNodeHealth(nodeHealth, hzMember.getClusterTime()));
}

@Override
public void clearMine() {
Map<String, TimestampedNodeHealth> sqHealthState = readReplicatedMap();
String clientUUID = hazelcastClient.getUUID();
String clientUUID = hzMember.getUuid();
if (LOG.isTraceEnabled()) {
LOG.trace("Reading {} and clearing for {}", new HashMap<>(sqHealthState), clientUUID);
}
@@ -65,10 +65,10 @@ public class SharedHealthStateImpl implements SharedHealthState {

@Override
public Set<NodeHealth> readAll() {
long clusterTime = hazelcastClient.getClusterTime();
long clusterTime = hzMember.getClusterTime();
long timeout = clusterTime - TIMEOUT_30_SECONDS;
Map<String, TimestampedNodeHealth> sqHealthState = readReplicatedMap();
Set<String> hzMemberUUIDs = hazelcastClient.getMemberUuids();
Set<String> hzMemberUUIDs = hzMember.getMemberUuids();
Set<NodeHealth> existingNodeHealths = sqHealthState.entrySet().stream()
.filter(outOfDate(timeout))
.filter(ofNonExistentMember(hzMemberUUIDs))
@@ -101,7 +101,7 @@ public class SharedHealthStateImpl implements SharedHealthState {
}

private Map<String, TimestampedNodeHealth> readReplicatedMap() {
return hazelcastClient.getReplicatedMap(ClusterObjectKeys.SQ_HEALTH_STATE);
return hzMember.getReplicatedMap(HazelcastObjects.SQ_HEALTH_STATE);
}

}

server/sonar-process/src/main/java/org/sonar/process/cluster/HazelcastClient.java → server/sonar-process/src/main/java/org/sonar/process/cluster/hz/HazelcastMember.java Parādīt failu

@@ -17,18 +17,45 @@
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.process.cluster.hz;

package org.sonar.process.cluster;
import com.hazelcast.core.Cluster;
import com.hazelcast.core.IAtomicReference;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.locks.Lock;
import org.sonar.process.ProcessId;
import org.sonar.process.cluster.NodeType;

public interface HazelcastMember extends AutoCloseable {

interface Attribute {
/**
* The key of the hostname attribute of a member
*/
String HOSTNAME = "HOSTNAME";
/**
* The key of the ips list attribute of a member
*/
String IP_ADDRESSES = "IP_ADDRESSES";
/**
* The key of the node name attribute of a member
*/
String NODE_NAME = "NODE_NAME";
/**
* The role of the sonar-application inside the SonarQube cluster
* {@link NodeType}
*/
String NODE_TYPE = "NODE_TYPE";
/**
* Key of process as defined by {@link ProcessId#getKey()}
*/
String PROCESS_KEY = "PROCESS_KEY";
}

<E> IAtomicReference<E> getAtomicReference(String name);

/**
* The interface Hazelcast client wrapper.
*/
public interface HazelcastClient {
/**
* Gets the set shared by the cluster and identified by name
*/
@@ -45,17 +72,13 @@ public interface HazelcastClient {
<K, V> Map<K, V> getMap(String name);

/**
* Gets the replicated map shared by the cluster and identified by name
* Gets the replicated map shared by the cluster and identified by name.
* Result can be casted to {@link com.hazelcast.core.ReplicatedMap} if needed to
* benefit from listeners.
*/
<K,V> Map<K,V> getReplicatedMap(String name);
<K, V> Map<K, V> getReplicatedMap(String name);

/**
* The UUID of the Hazelcast client.
*
* <p>The uuid of the member of the current client is a member, otherwise the UUID of the client if the
* member is a local client of one of the members.</p>
*/
String getUUID();
String getUuid();

/**
* The UUIDs of all the members (both members and local clients of these members) currently connected to the
@@ -69,7 +92,12 @@ public interface HazelcastClient {
Lock getLock(String name);

/**
* Retrieves the cluster time which is (alsmost) identical on all members of the cluster.
* Retrieves the cluster time which is (almost) identical on all members of the cluster.
*/
long getClusterTime();

Cluster getCluster();

@Override
void close();
}

+ 142
- 0
server/sonar-process/src/main/java/org/sonar/process/cluster/hz/HazelcastMemberBuilder.java Parādīt failu

@@ -0,0 +1,142 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.process.cluster.hz;

import com.hazelcast.config.Config;
import com.hazelcast.config.JoinConfig;
import com.hazelcast.config.MemberAttributeConfig;
import com.hazelcast.config.NetworkConfig;
import com.hazelcast.core.Hazelcast;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.stream.Collectors;
import javax.annotation.CheckForNull;
import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessId;
import org.sonar.process.ProcessProperties;
import org.sonar.process.cluster.NodeType;
import org.sonar.process.cluster.hz.HazelcastMember.Attribute;

import static java.lang.String.format;
import static java.util.Collections.singletonList;
import static java.util.Objects.requireNonNull;

public class HazelcastMemberBuilder {

private String clusterName;
private String nodeName;
private int port;
private NodeType nodeType;
private ProcessId processId;
private String networkInterface;
private List<String> members = new ArrayList<>();

public HazelcastMemberBuilder setClusterName(String s) {
this.clusterName = s;
return this;
}

public HazelcastMemberBuilder setNodeName(String s) {
this.nodeName = s;
return this;
}

public HazelcastMemberBuilder setNodeType(NodeType t) {
this.nodeType = t;
return this;
}

public HazelcastMemberBuilder setProcessId(ProcessId p) {
this.processId = p;
return this;
}

public HazelcastMemberBuilder setPort(int i) {
this.port = i;
return this;
}

public HazelcastMemberBuilder setNetworkInterface(String s) {
this.networkInterface = s;
return this;
}

@CheckForNull
List<String> getMembers() {
return members;
}

/**
* Adds references to cluster members. If port is missing, then default
* port is automatically added.
*/
public HazelcastMemberBuilder setMembers(Collection<String> c) {
this.members = c.stream()
.map(host -> host.contains(":") ? host : format("%s:%d", host, ProcessProperties.CLUSTER_NODE_PORT_DEFAULT_VALUE))
.collect(Collectors.toList());
return this;
}

public HazelcastMember build() {
Config config = new Config();
config.getGroupConfig().setName(requireNonNull(clusterName, "Cluster name is missing"));

// Configure network
NetworkConfig netConfig = config.getNetworkConfig();
netConfig
.setPort(port)
.setPortAutoIncrement(false)
.setReuseAddress(true);
netConfig.getInterfaces()
.setEnabled(true)
.setInterfaces(singletonList(requireNonNull(networkInterface, "Network interface is missing")));

// Only allowing TCP/IP configuration
JoinConfig joinConfig = netConfig.getJoin();
joinConfig.getAwsConfig().setEnabled(false);
joinConfig.getMulticastConfig().setEnabled(false);
joinConfig.getTcpIpConfig().setEnabled(true);
joinConfig.getTcpIpConfig().setMembers(requireNonNull(members, "Members are missing"));
// We are not using the partition group of Hazelcast, so disabling it
config.getPartitionGroupConfig().setEnabled(false);

// Tweak HazelCast configuration
config
// Increase the number of tries
.setProperty("hazelcast.tcp.join.port.try.count", "10")
// Don't bind on all interfaces
.setProperty("hazelcast.socket.bind.any", "false")
// Don't phone home
.setProperty("hazelcast.phone.home.enabled", "false")
// Use slf4j for logging
.setProperty("hazelcast.logging.type", "slf4j");

MemberAttributeConfig attributes = config.getMemberAttributeConfig();
attributes.setStringAttribute(Attribute.HOSTNAME, NetworkUtils.INSTANCE.getHostname());
attributes.setStringAttribute(Attribute.IP_ADDRESSES, NetworkUtils.INSTANCE.getIPAddresses());
attributes.setStringAttribute(Attribute.NODE_NAME, requireNonNull(nodeName, "Node name is missing"));
attributes.setStringAttribute(Attribute.NODE_TYPE, requireNonNull(nodeType, "Node type is missing").getValue());
attributes.setStringAttribute(Attribute.PROCESS_KEY, requireNonNull(processId, "Process key is missing").getKey());

return new HazelcastMemberImpl(Hazelcast.newHazelcastInstance(config));
}

}

+ 100
- 0
server/sonar-process/src/main/java/org/sonar/process/cluster/hz/HazelcastMemberImpl.java Parādīt failu

@@ -0,0 +1,100 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.process.cluster.hz;

import com.hazelcast.core.Cluster;
import com.hazelcast.core.HazelcastInstance;
import com.hazelcast.core.HazelcastInstanceNotActiveException;
import com.hazelcast.core.IAtomicReference;
import com.hazelcast.core.Member;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.locks.Lock;
import java.util.stream.Collectors;
import org.slf4j.LoggerFactory;

class HazelcastMemberImpl implements HazelcastMember {

private final HazelcastInstance hzInstance;

HazelcastMemberImpl(HazelcastInstance hzInstance) {
this.hzInstance = hzInstance;
}

@Override
public <E> IAtomicReference<E> getAtomicReference(String name) {
return hzInstance.getAtomicReference(name);
}

@Override
public <E> Set<E> getSet(String s) {
return hzInstance.getSet(s);
}

@Override
public <E> List<E> getList(String s) {
return hzInstance.getList(s);
}

@Override
public <K, V> Map<K, V> getMap(String s) {
return hzInstance.getMap(s);
}

@Override
public <K, V> Map<K, V> getReplicatedMap(String s) {
return hzInstance.getReplicatedMap(s);
}

@Override
public String getUuid() {
return hzInstance.getLocalEndpoint().getUuid();
}

@Override
public Set<String> getMemberUuids() {
return hzInstance.getCluster().getMembers().stream().map(Member::getUuid).collect(Collectors.toSet());
}

@Override
public Lock getLock(String s) {
return hzInstance.getLock(s);
}

@Override
public long getClusterTime() {
return hzInstance.getCluster().getClusterTime();
}

@Override
public Cluster getCluster() {
return hzInstance.getCluster();
}

@Override
public void close() {
try {
hzInstance.shutdown();
} catch (HazelcastInstanceNotActiveException e) {
LoggerFactory.getLogger(getClass()).debug("Unable to shutdown Hazelcast member", e);
}
}
}

server/sonar-process/src/main/java/org/sonar/process/cluster/ClusterObjectKeys.java → server/sonar-process/src/main/java/org/sonar/process/cluster/hz/HazelcastObjects.java Parādīt failu

@@ -18,14 +18,17 @@
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/

package org.sonar.process.cluster;
package org.sonar.process.cluster.hz;

import org.sonar.process.ProcessId;
import org.sonar.process.cluster.NodeType;

/**
* This class holds all object keys accessible via Hazelcast
*/
public final class ClusterObjectKeys {
public final class HazelcastObjects {

private ClusterObjectKeys() {
private HazelcastObjects() {
// Holder for clustered objects
}

@@ -37,23 +40,6 @@ public final class ClusterObjectKeys {
* The key of atomic reference holding the leader UUID
*/
public static final String LEADER = "LEADER";
/**
* The key of the hostname attribute of a member
*/
public static final String HOSTNAME = "HOSTNAME";
/**
* The key of the ips list attribute of a member
*/
public static final String IP_ADDRESSES = "IP_ADDRESSES";
/**
* The key of the node name attribute of a member
*/
public static final String NODE_NAME = "NODE_NAME";
/**
* The role of the sonar-application inside the SonarQube cluster
* {@link NodeType}
*/
public static final String NODE_TYPE = "NODE_TYPE";
/**
* The key of atomic reference holding the SonarQube version of the cluster
*/
@@ -62,10 +48,6 @@ public final class ClusterObjectKeys {
* The key of atomic reference holding the name of the cluster (used for precondition checks)
*/
public static final String CLUSTER_NAME = "CLUSTER_NAME";
/**
* The key of the Set holding the UUIDs of clients
*/
public static final String LOCAL_MEMBER_UUIDS = "LOCAL_MEMBER_UUIDS";
/**
* The key of replicated map holding the CeWorker UUIDs
*/

+ 23
- 0
server/sonar-process/src/main/java/org/sonar/process/cluster/hz/package-info.java Parādīt failu

@@ -0,0 +1,23 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
@ParametersAreNonnullByDefault
package org.sonar.process.cluster.hz;

import javax.annotation.ParametersAreNonnullByDefault;

+ 100
- 0
server/sonar-process/src/test/java/org/sonar/process/NetworkUtilsImplTest.java Parādīt failu

@@ -0,0 +1,100 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.process;

import java.net.Inet4Address;
import java.net.InetAddress;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Optional;
import java.util.Set;
import org.hamcrest.CoreMatchers;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;

import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.Assume.assumeThat;

public class NetworkUtilsImplTest {

@Rule
public ExpectedException expectedException = ExpectedException.none();

private NetworkUtilsImpl underTest = new NetworkUtilsImpl();

@Test
public void getNextAvailablePort_returns_a_port() throws Exception {
int port = underTest.getNextAvailablePort(InetAddress.getLocalHost());
assertThat(port)
.isGreaterThan(1_023)
.isLessThanOrEqualTo(65_535);
}

@Test
public void getNextAvailablePort_does_not_return_twice_the_same_port() throws Exception {
Set<Integer> ports = new HashSet<>(Arrays.asList(
underTest.getNextAvailablePort(InetAddress.getLocalHost()),
underTest.getNextAvailablePort(InetAddress.getLocalHost()),
underTest.getNextAvailablePort(InetAddress.getLocalHost())));
assertThat(ports).hasSize(3);
}

@Test
public void getLocalNonLoopbackIpv4Address_returns_a_valid_local_and_non_loopback_ipv4() {
Optional<InetAddress> address = underTest.getLocalNonLoopbackIpv4Address();

// address is empty on offline builds
assumeThat(address.isPresent(), CoreMatchers.is(true));

assertThat(address.get()).isInstanceOf(Inet4Address.class);
assertThat(address.get().isLoopbackAddress()).isFalse();
}

@Test
public void getLocalInetAddress_filters_local_addresses() {
InetAddress address = underTest.getLocalInetAddress(InetAddress::isLoopbackAddress).get();
assertThat(address.isLoopbackAddress()).isTrue();
}

@Test
public void getLocalInetAddress_returns_empty_if_no_local_addresses_match() {
Optional<InetAddress> address = underTest.getLocalInetAddress(a -> false);
assertThat(address).isEmpty();
}

@Test
public void toInetAddress_supports_host_names() throws Exception {
assertThat(underTest.toInetAddress("localhost")).isNotNull();
// do not test values that require DNS calls. Build must support offline mode.
}

@Test
public void toInetAddress_supports_ipv4() throws Exception {
assertThat(underTest.toInetAddress("1.2.3.4")).isNotNull();
}

@Test
public void toInetAddress_supports_ipv6() throws Exception {
assertThat(underTest.toInetAddress("2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccb")).isNotNull();
assertThat(underTest.toInetAddress("[2a01:e34:ef1f:dbb0:c2f6:a978:c5c0:9ccb]")).isNotNull();
}

}

+ 80
- 0
server/sonar-process/src/test/java/org/sonar/process/cluster/hz/HazelcastMemberBuilderTest.java Parādīt failu

@@ -0,0 +1,80 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.process.cluster.hz;

import java.net.InetAddress;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.DisableOnDebug;
import org.junit.rules.TestRule;
import org.junit.rules.Timeout;
import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessId;
import org.sonar.process.ProcessProperties;
import org.sonar.process.cluster.NodeType;

import static java.util.Arrays.asList;
import static org.assertj.core.api.Assertions.assertThat;

public class HazelcastMemberBuilderTest {

@Rule
public TestRule safeguardTimeout = new DisableOnDebug(Timeout.seconds(60));

// use loopback for support of offline builds
private InetAddress loopback = InetAddress.getLoopbackAddress();
private HazelcastMemberBuilder underTest = new HazelcastMemberBuilder();

@Test
public void build_member() {
HazelcastMember member = underTest
.setNodeType(NodeType.APPLICATION)
.setProcessId(ProcessId.COMPUTE_ENGINE)
.setClusterName("foo")
.setNodeName("bar")
.setPort(NetworkUtils.INSTANCE.getNextAvailablePort(loopback))
.setNetworkInterface(loopback.getHostAddress())
.build();

assertThat(member.getUuid()).isNotEmpty();
assertThat(member.getClusterTime()).isGreaterThan(0);
assertThat(member.getCluster().getMembers()).hasSize(1);
assertThat(member.getMemberUuids()).containsOnlyOnce(member.getUuid());

assertThat(member.getAtomicReference("baz")).isNotNull();
assertThat(member.getLock("baz")).isNotNull();
assertThat(member.getReplicatedMap("baz")).isNotNull();
assertThat(member.getList("baz")).isNotNull();
assertThat(member.getMap("baz")).isNotNull();
assertThat(member.getSet("baz")).isNotNull();

member.close();
}

@Test
public void default_port_is_added_when_missing() {
underTest.setMembers(asList("foo", "bar:9100", "1.2.3.4"));

assertThat(underTest.getMembers()).containsExactly(
"foo:" + ProcessProperties.CLUSTER_NODE_PORT_DEFAULT_VALUE,
"bar:9100",
"1.2.3.4:" + ProcessProperties.CLUSTER_NODE_PORT_DEFAULT_VALUE);
}
}

+ 149
- 0
server/sonar-server/src/main/java/org/sonar/server/cluster/StartableHazelcastMember.java Parādīt failu

@@ -0,0 +1,149 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package org.sonar.server.cluster;

import com.hazelcast.core.Cluster;
import com.hazelcast.core.IAtomicReference;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.locks.Lock;
import org.sonar.api.Startable;
import org.sonar.api.config.Configuration;
import org.sonar.process.NetworkUtils;
import org.sonar.process.ProcessId;
import org.sonar.process.ProcessProperties;
import org.sonar.process.cluster.NodeType;
import org.sonar.process.cluster.hz.HazelcastMember;
import org.sonar.process.cluster.hz.HazelcastMemberBuilder;

import static java.lang.String.format;
import static java.util.Arrays.asList;
import static java.util.Objects.requireNonNull;
import static org.sonar.process.ProcessEntryPoint.PROPERTY_PROCESS_KEY;
import static org.sonar.process.ProcessProperties.CLUSTER_HOSTS;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_TYPE;

/**
* Implementation of {@link HazelcastMember} as used by Compute Engine and
* Web Server processes. It is configured by {@link Configuration}
* and its lifecycle is managed by picocontainer.
*/
public class StartableHazelcastMember implements HazelcastMember, Startable {

private final Configuration config;
private final NetworkUtils networkUtils;
private HazelcastMember member = null;

public StartableHazelcastMember(Configuration config, NetworkUtils networkUtils) {
this.config = config;
this.networkUtils = networkUtils;
}

@Override
public <E> IAtomicReference<E> getAtomicReference(String name) {
return nonNullMember().getAtomicReference(name);
}

@Override
public <E> Set<E> getSet(String name) {
return nonNullMember().getSet(name);
}

@Override
public <E> List<E> getList(String name) {
return nonNullMember().getList(name);
}

@Override
public <K, V> Map<K, V> getMap(String name) {
return nonNullMember().getMap(name);
}

@Override
public <K, V> Map<K, V> getReplicatedMap(String name) {
return nonNullMember().getReplicatedMap(name);
}

@Override
public String getUuid() {
return nonNullMember().getUuid();
}

@Override
public Set<String> getMemberUuids() {
return nonNullMember().getMemberUuids();
}

@Override
public Lock getLock(String name) {
return nonNullMember().getLock(name);
}

@Override
public long getClusterTime() {
return nonNullMember().getClusterTime();
}

@Override
public Cluster getCluster() {
return nonNullMember().getCluster();
}

private HazelcastMember nonNullMember() {
return requireNonNull(member, "Hazelcast member not started");
}

@Override
public void close() {
if (member != null) {
member.close();
member = null;
}
}

@Override
public void start() {
String networkAddress = config.get(CLUSTER_NODE_HOST).orElseThrow(() -> new IllegalStateException("Missing node host"));
int freePort;
try {
freePort = networkUtils.getNextAvailablePort(InetAddress.getByName(networkAddress));
} catch (UnknownHostException e) {
throw new IllegalStateException(format("Can not resolve address %s", networkAddress), e);
}
this.member = new HazelcastMemberBuilder()
.setClusterName(config.get(ProcessProperties.CLUSTER_NAME).orElseThrow(() -> new IllegalStateException("Missing cluster name")))
.setNodeName(config.get(ProcessProperties.CLUSTER_NODE_NAME).orElseThrow(() -> new IllegalStateException("Missing node name")))
.setNodeType(NodeType.parse(config.get(CLUSTER_NODE_TYPE).orElseThrow(() -> new IllegalStateException("Missing node type"))))
.setPort(freePort)
.setProcessId(ProcessId.fromKey(config.get(PROPERTY_PROCESS_KEY).orElseThrow(() -> new IllegalStateException("Missing process key"))))
.setMembers(asList(config.getStringArray(CLUSTER_HOSTS)))
.setNetworkInterface(networkAddress)
.build();
}

@Override
public void stop() {
close();
}
}

server/sonar-server/src/main/java/org/sonar/server/hz/package-info.java → server/sonar-server/src/main/java/org/sonar/server/cluster/package-info.java Parādīt failu

@@ -18,6 +18,6 @@
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
@ParametersAreNonnullByDefault
package org.sonar.server.hz;
package org.sonar.server.cluster;

import javax.annotation.ParametersAreNonnullByDefault;

+ 4
- 4
server/sonar-server/src/main/java/org/sonar/server/es/EsClientProvider.java Parādīt failu

@@ -46,10 +46,10 @@ import org.sonar.process.cluster.NodeType;
import org.sonar.process.ProcessProperties;

import static java.util.Collections.unmodifiableList;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_TYPE;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_SEARCH_HOSTS;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_NAME;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_TYPE;
import static org.sonar.process.ProcessProperties.CLUSTER_SEARCH_HOSTS;
import static org.sonar.process.cluster.NodeType.SEARCH;

@ComputeEngineSide

+ 1
- 1
server/sonar-server/src/main/java/org/sonar/server/es/NewIndex.java Parādīt failu

@@ -39,7 +39,7 @@ import static com.google.common.base.Preconditions.checkArgument;
import static java.lang.String.format;
import static java.lang.String.valueOf;
import static java.util.Objects.requireNonNull;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.server.es.DefaultIndexSettings.ANALYZER;
import static org.sonar.server.es.DefaultIndexSettings.FIELDDATA_ENABLED;
import static org.sonar.server.es.DefaultIndexSettings.FIELD_FIELDDATA;

+ 3
- 3
server/sonar-server/src/main/java/org/sonar/server/health/NodeHealthProviderImpl.java Parādīt failu

@@ -28,9 +28,9 @@ import org.sonar.process.cluster.health.NodeHealth;
import org.sonar.process.cluster.health.NodeHealthProvider;

import static java.lang.String.format;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_PORT;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_PORT;
import static org.sonar.process.cluster.health.NodeDetails.newNodeDetailsBuilder;
import static org.sonar.process.cluster.health.NodeHealth.newNodeHealthBuilder;


+ 0
- 127
server/sonar-server/src/main/java/org/sonar/server/hz/HazelcastLocalClient.java Parādīt failu

@@ -1,127 +0,0 @@
/*
* SonarQube
* Copyright (C) 2009-2017 SonarSource SA
* mailto:info AT sonarsource DOT com
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU Lesser General Public
* License as published by the Free Software Foundation; either
* version 3 of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
* Lesser General Public License for more details.
*
* You should have received a copy of the GNU Lesser General Public License
* along with this program; if not, write to the Free Software Foundation,
* Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/

package org.sonar.server.hz;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
import com.hazelcast.client.config.ClientConfig;
import com.hazelcast.core.HazelcastInstance;
import com.hazelcast.core.Member;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.locks.Lock;
import org.picocontainer.Startable;
import org.sonar.api.config.Configuration;
import org.sonar.process.cluster.ClusterObjectKeys;
import org.sonar.process.cluster.HazelcastClient;

import static org.apache.commons.lang.StringUtils.isNotEmpty;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_LOCALENDPOINT;

/**
* This class will connect as a Hazelcast client to the local instance of Hazelcluster
*/
public class HazelcastLocalClient implements Startable, HazelcastClient {

private static final String HAZELCAST_CLUSTER_NAME = "sonarqube";
private final ClientConfig hzConfig;

@VisibleForTesting
HazelcastInstance hzInstance;

public HazelcastLocalClient(Configuration config) {
boolean clusterEnabled = config.getBoolean(CLUSTER_ENABLED).orElse(false);
String clusterLocalEndPoint = config.get(CLUSTER_LOCALENDPOINT).orElse(null);

Preconditions.checkState(clusterEnabled, "Cluster is not enabled");
Preconditions.checkState(isNotEmpty(clusterLocalEndPoint), "LocalEndPoint have not been set");

hzConfig = new ClientConfig();
hzConfig.getGroupConfig().setName(HAZELCAST_CLUSTER_NAME);
hzConfig.getNetworkConfig().addAddress(clusterLocalEndPoint);

// Tweak HazelCast configuration
hzConfig
// Increase the number of tries
.setProperty("hazelcast.tcp.join.port.try.count", "10")
// Don't phone home
.setProperty("hazelcast.phone.home.enabled", "false")
// Use slf4j for logging
.setProperty("hazelcast.logging.type", "slf4j");
}

@Override
public <E> Set<E> getSet(String name) {
return hzInstance.getSet(name);
}

@Override
public <E> List<E> getList(String name) {
return hzInstance.getList(name);
}

@Override
public <K, V> Map<K, V> getMap(String name) {
return hzInstance.getMap(name);
}

@Override
public <K, V> Map<K, V> getReplicatedMap(String name) {
return hzInstance.getReplicatedMap(name);
}

@Override
public String getUUID() {
return hzInstance.getLocalEndpoint().getUuid();
}

@Override
public Set<String> getMemberUuids() {
ImmutableSet.Builder<String> builder = ImmutableSet.builder();
builder.addAll(hzInstance.getSet(ClusterObjectKeys.LOCAL_MEMBER_UUIDS));
hzInstance.getCluster().getMembers().stream().map(Member::getUuid).forEach(builder::add);
return builder.build();
}

@Override
public Lock getLock(String name) {
return hzInstance.getLock(name);
}

@Override
public long getClusterTime() {
return hzInstance.getCluster().getClusterTime();
}

@Override
public void start() {
this.hzInstance = com.hazelcast.client.HazelcastClient.newHazelcastClient(hzConfig);
}

@Override
public void stop() {
// Shutdown Hazelcast properly
hzInstance.shutdown();
}
}

+ 1
- 3
server/sonar-server/src/main/java/org/sonar/server/platform/WebServer.java Parādīt failu

@@ -19,12 +19,10 @@
*/
package org.sonar.server.platform;

import org.sonar.process.cluster.ClusterProperties;

public interface WebServer {

/**
* WebServer is standalone when property {@link ClusterProperties#CLUSTER_ENABLED} is {@code false} or
* WebServer is standalone when property {@link org.sonar.process.ProcessProperties#CLUSTER_ENABLED} is {@code false} or
* undefined.
*/
boolean isStandalone();

+ 3
- 3
server/sonar-server/src/main/java/org/sonar/server/platform/WebServerImpl.java Parādīt failu

@@ -22,8 +22,8 @@ package org.sonar.server.platform;
import org.sonar.api.config.Configuration;
import org.sonar.api.utils.log.Loggers;

import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_WEB_LEADER;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_WEB_STARTUP_LEADER;

public class WebServerImpl implements WebServer {

@@ -33,7 +33,7 @@ public class WebServerImpl implements WebServer {
public WebServerImpl(Configuration config) {
this.clusterEnabled = config.getBoolean(CLUSTER_ENABLED).orElse(false);
if (this.clusterEnabled) {
this.startupLeader = config.getBoolean(CLUSTER_WEB_LEADER).orElse(false);
this.startupLeader = config.getBoolean(CLUSTER_WEB_STARTUP_LEADER).orElse(false);
Loggers.get(WebServerImpl.class).info("Cluster enabled (startup {})", startupLeader ? "leader" : "follower");
} else {
this.startupLeader = true;

+ 2
- 2
server/sonar-server/src/main/java/org/sonar/server/platform/platformlevel/PlatformLevel4.java Parādīt failu

@@ -37,6 +37,7 @@ import org.sonar.server.authentication.AuthenticationModule;
import org.sonar.server.batch.BatchWsModule;
import org.sonar.server.branch.BranchFeatureProxyImpl;
import org.sonar.server.ce.ws.CeWsModule;
import org.sonar.server.cluster.StartableHazelcastMember;
import org.sonar.server.component.ComponentCleanerService;
import org.sonar.server.component.ComponentFinder;
import org.sonar.server.component.ComponentService;
@@ -62,7 +63,6 @@ import org.sonar.server.es.metadata.MetadataIndexDefinition;
import org.sonar.server.event.NewAlerts;
import org.sonar.server.favorite.FavoriteModule;
import org.sonar.server.health.NodeHealthModule;
import org.sonar.server.hz.HazelcastLocalClient;
import org.sonar.server.issue.AddTagsAction;
import org.sonar.server.issue.AssignAction;
import org.sonar.server.issue.CommentAction;
@@ -247,7 +247,7 @@ public class PlatformLevel4 extends PlatformLevel {
EsDbCompatibilityImpl.class);

addIfCluster(
HazelcastLocalClient.class,
StartableHazelcastMember.class,
NodeHealthModule.class);

add(

+ 4
- 4
server/sonar-server/src/test/java/org/sonar/server/es/EsClientProviderTest.java Parādīt failu

@@ -34,10 +34,10 @@ import org.sonar.process.ProcessProperties;

import static java.lang.String.format;
import static org.assertj.core.api.Assertions.assertThat;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_TYPE;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_SEARCH_HOSTS;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_NAME;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_TYPE;
import static org.sonar.process.ProcessProperties.CLUSTER_SEARCH_HOSTS;

public class EsClientProviderTest {


+ 1
- 1
server/sonar-server/src/test/java/org/sonar/server/es/NewIndexTest.java Parādīt failu

@@ -32,7 +32,7 @@ import org.sonar.process.ProcessProperties;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.data.MapEntry.entry;
import static org.junit.Assert.fail;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.server.es.NewIndex.SettingsConfiguration.newBuilder;

public class NewIndexTest {

+ 2
- 2
server/sonar-server/src/test/java/org/sonar/server/health/NodeHealthModuleTest.java Parādīt failu

@@ -31,8 +31,8 @@ import org.sonar.api.platform.Server;
import org.sonar.api.utils.System2;
import org.sonar.core.platform.ComponentContainer;
import org.sonar.process.NetworkUtils;
import org.sonar.process.cluster.HazelcastClient;
import org.sonar.process.cluster.health.SharedHealthStateImpl;
import org.sonar.process.cluster.hz.HazelcastMember;

import static java.lang.String.valueOf;
import static org.apache.commons.lang.RandomStringUtils.randomAlphanumeric;
@@ -61,7 +61,7 @@ public class NodeHealthModuleTest {
mapSettings.asConfig(),
server,
networkUtils,
mock(HazelcastClient.class));
mock(HazelcastMember.class));
// HealthAction dependencies
container.add(mock(HealthChecker.class));


+ 3
- 3
server/sonar-server/src/test/java/org/sonar/server/health/NodeHealthProviderImplTest.java Parādīt failu

@@ -37,9 +37,9 @@ import static org.apache.commons.lang.RandomStringUtils.randomAlphanumeric;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NODE_PORT;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_HOST;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_NAME;
import static org.sonar.process.ProcessProperties.CLUSTER_NODE_PORT;

public class NodeHealthProviderImplTest {
@Rule

+ 1
- 1
server/sonar-server/src/test/java/org/sonar/server/rule/index/RuleIndexDefinitionTest.java Parādīt failu

@@ -32,7 +32,7 @@ import org.sonar.server.es.NewIndex;

import static org.assertj.core.api.Assertions.assertThat;
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_ENABLED;
import static org.sonar.process.ProcessProperties.CLUSTER_ENABLED;
import static org.sonar.server.es.DefaultIndexSettingsElement.ENGLISH_HTML_ANALYZER;
import static org.sonar.server.rule.index.RuleIndexDefinition.FIELD_RULE_HTML_DESCRIPTION;
import static org.sonar.server.rule.index.RuleIndexDefinition.FIELD_RULE_KEY;

+ 2
- 2
sonar-application/src/main/java/org/sonar/application/App.java Parādīt failu

@@ -33,7 +33,7 @@ import org.sonar.application.command.CommandFactory;
import org.sonar.application.command.CommandFactoryImpl;

import static org.sonar.application.config.SonarQubeVersionHelper.getSonarqubeVersion;
import static org.sonar.process.cluster.ClusterProperties.CLUSTER_NAME;
import static org.sonar.process.ProcessProperties.CLUSTER_NAME;

public class App {

@@ -50,7 +50,7 @@ public class App {

try (AppState appState = new AppStateFactory(settings).create()) {
appState.registerSonarQubeVersion(getSonarqubeVersion());
appState.registerClusterName(settings.getProps().value(CLUSTER_NAME, "sonarqube"));
appState.registerClusterName(settings.getProps().nonNullValue(CLUSTER_NAME));
AppReloader appReloader = new AppReloaderImpl(settingsLoader, fileSystem, appState, logging);
fileSystem.reset();
CommandFactory commandFactory = new CommandFactoryImpl(settings.getProps(), fileSystem.getTempDir(), System2.INSTANCE);

+ 19
- 0
tests/src/test/java/org/sonarqube/tests/cluster/ClusterTest.java Parādīt failu

@@ -38,6 +38,7 @@ import org.junit.rules.TemporaryFolder;
import org.junit.rules.TestRule;
import org.junit.rules.Timeout;
import org.sonarqube.ws.WsSystem;
import org.sonarqube.ws.client.GetRequest;

import static com.google.common.base.Preconditions.checkState;
import static org.assertj.core.api.Assertions.assertThat;
@@ -68,6 +69,24 @@ public class ClusterTest {
db.stop();
}

/**
* TODO WIP
*/
@Test
public void wip() throws Exception {
try (Cluster cluster = newCluster(3, 2)) {
cluster.getNodes().forEach(Node::start);

Node app = cluster.getAppNode(0);
app.waitForHealthGreen();

System.out.println("-----------------------------------------------------------------------");
String json = app.wsClient().wsConnector().call(new GetRequest("api/system/cluster_info")).content();
System.out.println(json);
System.out.println("-----------------------------------------------------------------------");
}
}

@Test
public void test_high_availability_topology() throws Exception {
try (Cluster cluster = newCluster(3, 2)) {

Notiek ielāde…
Atcelt
Saglabāt