Wenjun Ruan
5 months ago
committed by
GitHub
90 changed files with 3152 additions and 2457 deletions
@ -0,0 +1,118 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.common.utils; |
||||||
|
|
||||||
|
import org.apache.commons.collections4.MapUtils; |
||||||
|
import org.apache.commons.collections4.SetUtils; |
||||||
|
|
||||||
|
import java.util.Collections; |
||||||
|
import java.util.HashSet; |
||||||
|
import java.util.List; |
||||||
|
import java.util.Map; |
||||||
|
import java.util.Set; |
||||||
|
import java.util.stream.Collectors; |
||||||
|
|
||||||
|
import lombok.Getter; |
||||||
|
import lombok.ToString; |
||||||
|
|
||||||
|
@Getter |
||||||
|
@ToString |
||||||
|
public class MapComparator<K, V> { |
||||||
|
|
||||||
|
private final Map<K, V> oldMap; |
||||||
|
private final Map<K, V> newMap; |
||||||
|
|
||||||
|
public MapComparator(Map<K, V> oldMap, Map<K, V> newMap) { |
||||||
|
this.oldMap = oldMap; |
||||||
|
this.newMap = newMap; |
||||||
|
} |
||||||
|
|
||||||
|
/** |
||||||
|
* Get keys that are in the new map but not in the old map |
||||||
|
*/ |
||||||
|
public Set<K> getKeysToAdd() { |
||||||
|
if (MapUtils.isEmpty(newMap)) { |
||||||
|
return SetUtils.emptySet(); |
||||||
|
} |
||||||
|
if (MapUtils.isEmpty(oldMap)) { |
||||||
|
return new HashSet<>(newMap.keySet()); |
||||||
|
} |
||||||
|
Set<K> keysToAdd = new HashSet<>(newMap.keySet()); |
||||||
|
keysToAdd.removeAll(oldMap.keySet()); |
||||||
|
return keysToAdd; |
||||||
|
} |
||||||
|
|
||||||
|
/** |
||||||
|
* Get values which key in the new map but not in the old map |
||||||
|
*/ |
||||||
|
public List<V> getValuesToAdd() { |
||||||
|
if (MapUtils.isEmpty(newMap)) { |
||||||
|
return Collections.emptyList(); |
||||||
|
} |
||||||
|
return getKeysToAdd().stream().map(newMap::get).collect(Collectors.toList()); |
||||||
|
} |
||||||
|
|
||||||
|
/** |
||||||
|
* Get keys which in the old map but not in the new map |
||||||
|
*/ |
||||||
|
public Set<K> getKeysToRemove() { |
||||||
|
if (MapUtils.isEmpty(oldMap)) { |
||||||
|
return SetUtils.emptySet(); |
||||||
|
} |
||||||
|
if (MapUtils.isEmpty(newMap)) { |
||||||
|
return new HashSet<>(oldMap.keySet()); |
||||||
|
} |
||||||
|
Set<K> keysToRemove = new HashSet<>(oldMap.keySet()); |
||||||
|
keysToRemove.removeAll(newMap.keySet()); |
||||||
|
return keysToRemove; |
||||||
|
} |
||||||
|
|
||||||
|
/** |
||||||
|
* Get values which key in the old map but not in the new map |
||||||
|
*/ |
||||||
|
public List<V> getValuesToRemove() { |
||||||
|
if (MapUtils.isEmpty(oldMap)) { |
||||||
|
return Collections.emptyList(); |
||||||
|
} |
||||||
|
return getKeysToRemove().stream().map(oldMap::get).collect(Collectors.toList()); |
||||||
|
} |
||||||
|
|
||||||
|
/** |
||||||
|
* Get keys which in both the old map and the new map, but the value is different |
||||||
|
*/ |
||||||
|
public Set<K> getKeysToUpdate() { |
||||||
|
if (MapUtils.isEmpty(oldMap) || MapUtils.isEmpty(newMap)) { |
||||||
|
return SetUtils.emptySet(); |
||||||
|
} |
||||||
|
Set<K> keysToUpdate = new HashSet<>(newMap.keySet()); |
||||||
|
keysToUpdate.retainAll(oldMap.keySet()); |
||||||
|
keysToUpdate.removeIf(key -> newMap.get(key).equals(oldMap.get(key))); |
||||||
|
|
||||||
|
return keysToUpdate; |
||||||
|
} |
||||||
|
|
||||||
|
/** |
||||||
|
* Get new values which key in both the old map and the new map, but the value is different |
||||||
|
*/ |
||||||
|
public List<V> getNewValuesToUpdate() { |
||||||
|
if (MapUtils.isEmpty(oldMap) || MapUtils.isEmpty(newMap)) { |
||||||
|
return Collections.emptyList(); |
||||||
|
} |
||||||
|
return getKeysToUpdate().stream().map(newMap::get).collect(Collectors.toList()); |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,170 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
package org.apache.dolphinscheduler.common.utils; |
||||||
|
|
||||||
|
import static com.google.common.truth.Truth.assertThat; |
||||||
|
|
||||||
|
import java.util.HashMap; |
||||||
|
import java.util.Map; |
||||||
|
|
||||||
|
import org.junit.jupiter.api.Test; |
||||||
|
|
||||||
|
class MapComparatorTest { |
||||||
|
|
||||||
|
@Test |
||||||
|
void getKeysToAdd() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparator(); |
||||||
|
assertThat(mapComparator.getKeysToAdd()).containsExactly("key5"); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getKeysToAdd_newMapIsNull() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparatorWithNewMapIsNull(); |
||||||
|
assertThat(mapComparator.getKeysToAdd()).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getKeysToAdd_oldMapIsNull() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparatorWithOldMapIsNull(); |
||||||
|
assertThat(mapComparator.getKeysToAdd()).containsExactly("key1", "key2", "key3", "key5"); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getValuesToAdd() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparator(); |
||||||
|
assertThat(mapComparator.getValuesToAdd()).containsExactly("map2_value5"); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getValuesToAdd_newMapIsNull() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparatorWithNewMapIsNull(); |
||||||
|
assertThat(mapComparator.getValuesToAdd()).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getValuesToAdd_oldMapIsNull() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparatorWithOldMapIsNull(); |
||||||
|
assertThat(mapComparator.getValuesToAdd()) |
||||||
|
.containsExactly("map2_value1", "map2_value2", "map2_value3", "map2_value5"); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getKeysToRemove() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparator(); |
||||||
|
assertThat(mapComparator.getKeysToRemove()).containsExactly("key4"); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getKeysToRemove_newMapIsNull() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparatorWithNewMapIsNull(); |
||||||
|
assertThat(mapComparator.getKeysToRemove()).containsExactly("key1", "key2", "key3", "key4"); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getKeysToRemove_oldMapIsNull() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparatorWithOldMapIsNull(); |
||||||
|
assertThat(mapComparator.getKeysToRemove()).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getValuesToRemove() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparator(); |
||||||
|
assertThat(mapComparator.getValuesToRemove()).containsExactly("map1_value4"); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getValuesToRemove_newMapIsNull() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparatorWithNewMapIsNull(); |
||||||
|
assertThat(mapComparator.getValuesToRemove()) |
||||||
|
.containsExactly("map1_value1", "map1_value2", "map1_value3", "map1_value4"); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getValuesToRemove_oldMapIsNull() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparatorWithOldMapIsNull(); |
||||||
|
assertThat(mapComparator.getValuesToRemove()).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getKeysToUpdate() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparator(); |
||||||
|
assertThat(mapComparator.getKeysToUpdate()).containsExactly("key1", "key2", "key3"); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getKeysToUpdate_newMapIsNull() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparatorWithNewMapIsNull(); |
||||||
|
assertThat(mapComparator.getKeysToUpdate()).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getKeysToUpdate_oldMapIsNull() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparatorWithOldMapIsNull(); |
||||||
|
assertThat(mapComparator.getKeysToUpdate()).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getNewValuesToUpdate() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparator(); |
||||||
|
assertThat(mapComparator.getNewValuesToUpdate()) |
||||||
|
.containsExactly("map2_value1", "map2_value2", "map2_value3"); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getNewValuesToUpdate_newMapIsEmpty() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparatorWithNewMapIsNull(); |
||||||
|
assertThat(mapComparator.getNewValuesToUpdate()).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getNewValuesToUpdate_oldMapIsEmpty() { |
||||||
|
MapComparator<String, String> mapComparator = getMapComparatorWithOldMapIsNull(); |
||||||
|
assertThat(mapComparator.getNewValuesToUpdate()).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
private MapComparator<String, String> getMapComparator() { |
||||||
|
Map<String, String> map1 = new HashMap<>(); |
||||||
|
map1.put("key1", "map1_value1"); |
||||||
|
map1.put("key2", "map1_value2"); |
||||||
|
map1.put("key3", "map1_value3"); |
||||||
|
map1.put("key4", "map1_value4"); |
||||||
|
Map<String, String> map2 = new HashMap<>(); |
||||||
|
map2.put("key1", "map2_value1"); |
||||||
|
map2.put("key2", "map2_value2"); |
||||||
|
map2.put("key3", "map2_value3"); |
||||||
|
map2.put("key5", "map2_value5"); |
||||||
|
return new MapComparator<>(map1, map2); |
||||||
|
} |
||||||
|
|
||||||
|
private MapComparator<String, String> getMapComparatorWithNewMapIsNull() { |
||||||
|
Map<String, String> map1 = new HashMap<>(); |
||||||
|
map1.put("key1", "map1_value1"); |
||||||
|
map1.put("key2", "map1_value2"); |
||||||
|
map1.put("key3", "map1_value3"); |
||||||
|
map1.put("key4", "map1_value4"); |
||||||
|
return new MapComparator<>(map1, null); |
||||||
|
} |
||||||
|
|
||||||
|
private MapComparator<String, String> getMapComparatorWithOldMapIsNull() { |
||||||
|
Map<String, String> map2 = new HashMap<>(); |
||||||
|
map2.put("key1", "map2_value1"); |
||||||
|
map2.put("key2", "map2_value2"); |
||||||
|
map2.put("key3", "map2_value3"); |
||||||
|
map2.put("key5", "map2_value5"); |
||||||
|
return new MapComparator<>(null, map2); |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,36 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.dao.repository.impl; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.dao.entity.WorkerGroup; |
||||||
|
import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; |
||||||
|
import org.apache.dolphinscheduler.dao.repository.BaseDao; |
||||||
|
import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; |
||||||
|
|
||||||
|
import lombok.NonNull; |
||||||
|
|
||||||
|
import org.springframework.stereotype.Repository; |
||||||
|
|
||||||
|
@Repository |
||||||
|
public class WorkerGroupDaoImpl extends BaseDao<WorkerGroup, WorkerGroupMapper> implements WorkerGroupDao { |
||||||
|
|
||||||
|
public WorkerGroupDaoImpl(@NonNull WorkerGroupMapper workerGroupMapper) { |
||||||
|
super(workerGroupMapper); |
||||||
|
} |
||||||
|
|
||||||
|
} |
@ -0,0 +1,69 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.registry.api.Event; |
||||||
|
import org.apache.dolphinscheduler.registry.api.SubscribeListener; |
||||||
|
|
||||||
|
import lombok.extern.slf4j.Slf4j; |
||||||
|
|
||||||
|
@Slf4j |
||||||
|
public abstract class AbstractClusterSubscribeListener<T extends BaseServerMetadata> implements SubscribeListener { |
||||||
|
|
||||||
|
@Override |
||||||
|
public void notify(Event event) { |
||||||
|
try { |
||||||
|
// make sure the event is processed in order
|
||||||
|
synchronized (this) { |
||||||
|
Event.Type type = event.type(); |
||||||
|
T server = parseServerFromHeartbeat(event.data()); |
||||||
|
if (server == null) { |
||||||
|
log.error("Unknown cluster change event: {}", event); |
||||||
|
return; |
||||||
|
} |
||||||
|
switch (type) { |
||||||
|
case ADD: |
||||||
|
log.info("Server {} added", server); |
||||||
|
onServerAdded(server); |
||||||
|
break; |
||||||
|
case REMOVE: |
||||||
|
log.warn("Server {} removed", server); |
||||||
|
onServerRemove(server); |
||||||
|
break; |
||||||
|
case UPDATE: |
||||||
|
log.debug("Server {} updated", server); |
||||||
|
onServerUpdate(server); |
||||||
|
break; |
||||||
|
default: |
||||||
|
break; |
||||||
|
} |
||||||
|
} |
||||||
|
} catch (Exception ex) { |
||||||
|
log.error("Notify cluster change event: {} failed", event, ex); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
abstract T parseServerFromHeartbeat(String serverHeartBeatJson); |
||||||
|
|
||||||
|
public abstract void onServerAdded(T serverHeartBeat); |
||||||
|
|
||||||
|
public abstract void onServerRemove(T serverHeartBeat); |
||||||
|
|
||||||
|
public abstract void onServerUpdate(T serverHeartBeat); |
||||||
|
|
||||||
|
} |
@ -0,0 +1,57 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.registry.api.RegistryClient; |
||||||
|
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; |
||||||
|
|
||||||
|
import lombok.Getter; |
||||||
|
import lombok.extern.slf4j.Slf4j; |
||||||
|
|
||||||
|
import org.springframework.beans.factory.annotation.Autowired; |
||||||
|
import org.springframework.stereotype.Component; |
||||||
|
|
||||||
|
@Slf4j |
||||||
|
@Component |
||||||
|
public class ClusterManager { |
||||||
|
|
||||||
|
@Getter |
||||||
|
private final MasterClusters masterClusters; |
||||||
|
|
||||||
|
@Getter |
||||||
|
private final WorkerClusters workerClusters; |
||||||
|
|
||||||
|
@Autowired |
||||||
|
private WorkerGroupChangeNotifier workerGroupChangeNotifier; |
||||||
|
|
||||||
|
@Autowired |
||||||
|
private RegistryClient registryClient; |
||||||
|
|
||||||
|
public ClusterManager() { |
||||||
|
this.masterClusters = new MasterClusters(); |
||||||
|
this.workerClusters = new WorkerClusters(); |
||||||
|
} |
||||||
|
|
||||||
|
public void start() { |
||||||
|
this.registryClient.subscribe(RegistryNodeType.MASTER.getRegistryPath(), masterClusters); |
||||||
|
this.registryClient.subscribe(RegistryNodeType.WORKER.getRegistryPath(), workerClusters); |
||||||
|
this.workerGroupChangeNotifier.subscribeWorkerGroupsChange(workerClusters); |
||||||
|
log.info("ClusterManager started..."); |
||||||
|
} |
||||||
|
|
||||||
|
} |
@ -0,0 +1,60 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.dao.AlertDao; |
||||||
|
import org.apache.dolphinscheduler.service.alert.ListenerEventAlertManager; |
||||||
|
|
||||||
|
import lombok.extern.slf4j.Slf4j; |
||||||
|
|
||||||
|
import org.springframework.beans.factory.annotation.Autowired; |
||||||
|
import org.springframework.stereotype.Component; |
||||||
|
|
||||||
|
@Slf4j |
||||||
|
@Component |
||||||
|
public class ClusterStateMonitors { |
||||||
|
|
||||||
|
@Autowired |
||||||
|
private ClusterManager clusterManager; |
||||||
|
|
||||||
|
@Autowired |
||||||
|
private ListenerEventAlertManager listenerEventAlertManager; |
||||||
|
|
||||||
|
@Autowired |
||||||
|
private AlertDao alertDao; |
||||||
|
|
||||||
|
public void start() { |
||||||
|
this.clusterManager.getMasterClusters() |
||||||
|
.registerListener((IClusters.ServerRemovedListener<MasterServerMetadata>) this::masterRemoved); |
||||||
|
this.clusterManager.getWorkerClusters() |
||||||
|
.registerListener((IClusters.ServerRemovedListener<WorkerServerMetadata>) this::workerRemoved); |
||||||
|
log.info("ClusterStateMonitors started..."); |
||||||
|
} |
||||||
|
|
||||||
|
void masterRemoved(MasterServerMetadata masterServer) { |
||||||
|
// todo: unify the alert message
|
||||||
|
alertDao.sendServerStoppedAlert(masterServer.getAddress(), "MASTER"); |
||||||
|
listenerEventAlertManager.publishServerDownListenerEvent(masterServer.getAddress(), "MASTER"); |
||||||
|
} |
||||||
|
|
||||||
|
void workerRemoved(WorkerServerMetadata workerServer) { |
||||||
|
alertDao.sendServerStoppedAlert(workerServer.getAddress(), "WORKER"); |
||||||
|
listenerEventAlertManager.publishServerDownListenerEvent(workerServer.getAddress(), "WORKER"); |
||||||
|
} |
||||||
|
|
||||||
|
} |
@ -0,0 +1,79 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
||||||
|
|
||||||
|
import java.util.List; |
||||||
|
|
||||||
|
public interface IClusters<S extends IClusters.IServerMetadata> { |
||||||
|
|
||||||
|
List<S> getServers(); |
||||||
|
|
||||||
|
void registerListener(IClustersChangeListener<S> listener); |
||||||
|
|
||||||
|
interface IServerMetadata { |
||||||
|
|
||||||
|
String getAddress(); |
||||||
|
|
||||||
|
ServerStatus getServerStatus(); |
||||||
|
|
||||||
|
} |
||||||
|
|
||||||
|
interface IClustersChangeListener<S extends IServerMetadata> { |
||||||
|
|
||||||
|
void onServerAdded(S server); |
||||||
|
|
||||||
|
void onServerRemove(S server); |
||||||
|
|
||||||
|
void onServerUpdate(S server); |
||||||
|
|
||||||
|
} |
||||||
|
|
||||||
|
interface ServerAddedListener<S extends IServerMetadata> extends IClustersChangeListener<S> { |
||||||
|
|
||||||
|
@Override |
||||||
|
default void onServerRemove(S server) { |
||||||
|
// only care about server added
|
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
default void onServerUpdate(S server) { |
||||||
|
// only care about server added
|
||||||
|
} |
||||||
|
|
||||||
|
} |
||||||
|
|
||||||
|
interface ServerRemovedListener<S extends IServerMetadata> extends IClustersChangeListener<S> { |
||||||
|
|
||||||
|
@Override |
||||||
|
default void onServerAdded(S server) { |
||||||
|
// only care about server removed
|
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
void onServerRemove(S server); |
||||||
|
|
||||||
|
@Override |
||||||
|
default void onServerUpdate(S server) { |
||||||
|
// only care about server added
|
||||||
|
} |
||||||
|
|
||||||
|
} |
||||||
|
|
||||||
|
} |
@ -0,0 +1,25 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import java.util.List; |
||||||
|
|
||||||
|
public interface IMasterSlotReBalancer { |
||||||
|
|
||||||
|
void doReBalance(List<MasterServerMetadata> masterServerList); |
||||||
|
} |
@ -0,0 +1,99 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
||||||
|
import org.apache.dolphinscheduler.common.model.MasterHeartBeat; |
||||||
|
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||||
|
|
||||||
|
import org.apache.commons.collections4.list.UnmodifiableList; |
||||||
|
|
||||||
|
import java.util.ArrayList; |
||||||
|
import java.util.List; |
||||||
|
import java.util.Map; |
||||||
|
import java.util.concurrent.ConcurrentHashMap; |
||||||
|
import java.util.concurrent.CopyOnWriteArrayList; |
||||||
|
import java.util.stream.Collectors; |
||||||
|
|
||||||
|
import lombok.extern.slf4j.Slf4j; |
||||||
|
|
||||||
|
@Slf4j |
||||||
|
public class MasterClusters extends AbstractClusterSubscribeListener<MasterServerMetadata> |
||||||
|
implements |
||||||
|
IClusters<MasterServerMetadata> { |
||||||
|
|
||||||
|
/** |
||||||
|
* Master address -> MasterServer |
||||||
|
**/ |
||||||
|
private final Map<String, MasterServerMetadata> masterServerMap = new ConcurrentHashMap<>(); |
||||||
|
|
||||||
|
private final List<IClustersChangeListener<MasterServerMetadata>> masterClusterChangeListeners = |
||||||
|
new CopyOnWriteArrayList<>(); |
||||||
|
|
||||||
|
@Override |
||||||
|
public List<MasterServerMetadata> getServers() { |
||||||
|
return UnmodifiableList.unmodifiableList(new ArrayList<>(masterServerMap.values())); |
||||||
|
} |
||||||
|
|
||||||
|
public List<MasterServerMetadata> getNormalServers() { |
||||||
|
List<MasterServerMetadata> normalMasterServers = masterServerMap.values() |
||||||
|
.stream() |
||||||
|
.filter(masterServer -> masterServer.getServerStatus() == ServerStatus.NORMAL) |
||||||
|
.collect(Collectors.toList()); |
||||||
|
return UnmodifiableList.unmodifiableList(normalMasterServers); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void registerListener(IClustersChangeListener<MasterServerMetadata> listener) { |
||||||
|
masterClusterChangeListeners.add(listener); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
MasterServerMetadata parseServerFromHeartbeat(String masterHeartBeatJson) { |
||||||
|
MasterHeartBeat masterHeartBeat = JSONUtils.parseObject(masterHeartBeatJson, MasterHeartBeat.class); |
||||||
|
if (masterHeartBeat == null) { |
||||||
|
return null; |
||||||
|
} |
||||||
|
return MasterServerMetadata.parseFromHeartBeat(masterHeartBeat); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerAdded(MasterServerMetadata masterServer) { |
||||||
|
masterServerMap.put(masterServer.getAddress(), masterServer); |
||||||
|
for (IClustersChangeListener<MasterServerMetadata> listener : masterClusterChangeListeners) { |
||||||
|
listener.onServerAdded(masterServer); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerRemove(MasterServerMetadata masterServer) { |
||||||
|
masterServerMap.remove(masterServer.getAddress()); |
||||||
|
for (IClustersChangeListener<MasterServerMetadata> listener : masterClusterChangeListeners) { |
||||||
|
listener.onServerRemove(masterServer); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerUpdate(MasterServerMetadata masterServer) { |
||||||
|
masterServerMap.put(masterServer.getAddress(), masterServer); |
||||||
|
for (IClustersChangeListener<MasterServerMetadata> listener : masterClusterChangeListeners) { |
||||||
|
listener.onServerUpdate(masterServer); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
} |
@ -0,0 +1,46 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.model.MasterHeartBeat; |
||||||
|
|
||||||
|
import lombok.Data; |
||||||
|
import lombok.EqualsAndHashCode; |
||||||
|
import lombok.experimental.SuperBuilder; |
||||||
|
|
||||||
|
@Data |
||||||
|
@SuperBuilder |
||||||
|
@EqualsAndHashCode(callSuper = true) |
||||||
|
public class MasterServerMetadata extends BaseServerMetadata implements Comparable<MasterServerMetadata> { |
||||||
|
|
||||||
|
public static MasterServerMetadata parseFromHeartBeat(MasterHeartBeat masterHeartBeat) { |
||||||
|
return MasterServerMetadata.builder() |
||||||
|
.address(masterHeartBeat.getHost() + ":" + masterHeartBeat.getPort()) |
||||||
|
.cpuUsage(masterHeartBeat.getCpuUsage()) |
||||||
|
.memoryUsage(masterHeartBeat.getMemoryUsage()) |
||||||
|
.serverStatus(masterHeartBeat.getServerStatus()) |
||||||
|
.build(); |
||||||
|
} |
||||||
|
|
||||||
|
// Use the master address to sort the master server
|
||||||
|
@Override |
||||||
|
public int compareTo(MasterServerMetadata o) { |
||||||
|
return this.getAddress().compareTo(o.getAddress()); |
||||||
|
} |
||||||
|
|
||||||
|
} |
@ -0,0 +1,101 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||||
|
|
||||||
|
import java.util.List; |
||||||
|
|
||||||
|
import lombok.extern.slf4j.Slf4j; |
||||||
|
|
||||||
|
import org.springframework.stereotype.Component; |
||||||
|
|
||||||
|
@Slf4j |
||||||
|
@Component |
||||||
|
public class MasterSlotManager implements IMasterSlotReBalancer { |
||||||
|
|
||||||
|
private final MasterClusters masterClusters; |
||||||
|
|
||||||
|
private final MasterConfig masterConfig; |
||||||
|
|
||||||
|
private volatile int currentSlot = -1; |
||||||
|
|
||||||
|
private volatile int totalSlots = 0; |
||||||
|
|
||||||
|
public MasterSlotManager(ClusterManager clusterManager, MasterConfig masterConfig) { |
||||||
|
this.masterConfig = masterConfig; |
||||||
|
this.masterClusters = clusterManager.getMasterClusters(); |
||||||
|
this.masterClusters.registerListener(new IClusters.IClustersChangeListener<MasterServerMetadata>() { |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerAdded(MasterServerMetadata server) { |
||||||
|
doReBalance(masterClusters.getNormalServers()); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerRemove(MasterServerMetadata server) { |
||||||
|
doReBalance(masterClusters.getNormalServers()); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerUpdate(MasterServerMetadata server) { |
||||||
|
doReBalance(masterClusters.getNormalServers()); |
||||||
|
} |
||||||
|
}); |
||||||
|
} |
||||||
|
|
||||||
|
/** |
||||||
|
* Get the current master slot, if the slot is -1, it means the master slot is not available. |
||||||
|
*/ |
||||||
|
public int getCurrentMasterSlot() { |
||||||
|
return currentSlot; |
||||||
|
} |
||||||
|
|
||||||
|
/** |
||||||
|
* Get the total master slots. |
||||||
|
*/ |
||||||
|
public int getTotalMasterSlots() { |
||||||
|
return totalSlots; |
||||||
|
} |
||||||
|
|
||||||
|
public boolean checkSlotValid() { |
||||||
|
return totalSlots > 0 && currentSlot >= 0; |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void doReBalance(List<MasterServerMetadata> normalMasterServers) { |
||||||
|
|
||||||
|
int tmpCurrentSlot = -1; |
||||||
|
for (int i = 0; i < normalMasterServers.size(); i++) { |
||||||
|
if (normalMasterServers.get(i).getAddress().equals(masterConfig.getMasterAddress())) { |
||||||
|
tmpCurrentSlot = i; |
||||||
|
break; |
||||||
|
} |
||||||
|
} |
||||||
|
if (tmpCurrentSlot == -1) { |
||||||
|
log.warn( |
||||||
|
"Do re balance failed, cannot found the current master: {} in the normal master clusters: {}. Please check the current master server status", |
||||||
|
masterConfig.getMasterAddress(), normalMasterServers); |
||||||
|
currentSlot = -1; |
||||||
|
return; |
||||||
|
} |
||||||
|
|
||||||
|
totalSlots = normalMasterServers.size(); |
||||||
|
currentSlot = tmpCurrentSlot; |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,143 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
||||||
|
import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; |
||||||
|
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||||
|
import org.apache.dolphinscheduler.dao.entity.WorkerGroup; |
||||||
|
import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils; |
||||||
|
|
||||||
|
import org.apache.commons.collections4.list.UnmodifiableList; |
||||||
|
|
||||||
|
import java.util.ArrayList; |
||||||
|
import java.util.Collections; |
||||||
|
import java.util.List; |
||||||
|
import java.util.Map; |
||||||
|
import java.util.Objects; |
||||||
|
import java.util.concurrent.ConcurrentHashMap; |
||||||
|
import java.util.concurrent.CopyOnWriteArrayList; |
||||||
|
import java.util.stream.Collectors; |
||||||
|
|
||||||
|
public class WorkerClusters extends AbstractClusterSubscribeListener<WorkerServerMetadata> |
||||||
|
implements |
||||||
|
IClusters<WorkerServerMetadata>, |
||||||
|
WorkerGroupChangeNotifier.WorkerGroupListener { |
||||||
|
|
||||||
|
// WorkerIdentifier(workerAddress) -> worker
|
||||||
|
private final Map<String, WorkerServerMetadata> workerMapping = new ConcurrentHashMap<>(); |
||||||
|
|
||||||
|
// WorkerGroup -> WorkerIdentifier(workerAddress)
|
||||||
|
private final Map<String, List<String>> workerGroupMapping = new ConcurrentHashMap<>(); |
||||||
|
|
||||||
|
private final List<IClustersChangeListener<WorkerServerMetadata>> workerClusterChangeListeners = |
||||||
|
new CopyOnWriteArrayList<>(); |
||||||
|
|
||||||
|
@Override |
||||||
|
public List<WorkerServerMetadata> getServers() { |
||||||
|
return UnmodifiableList.unmodifiableList(new ArrayList<>(workerMapping.values())); |
||||||
|
} |
||||||
|
|
||||||
|
public List<String> getWorkerServerAddressByGroup(String workerGroup) { |
||||||
|
if (WorkerGroupUtils.getDefaultWorkerGroup().equals(workerGroup)) { |
||||||
|
return UnmodifiableList.unmodifiableList(new ArrayList<>(workerMapping.keySet())); |
||||||
|
} |
||||||
|
return workerGroupMapping.getOrDefault(workerGroup, Collections.emptyList()); |
||||||
|
} |
||||||
|
|
||||||
|
public List<String> getNormalWorkerServerAddressByGroup(String workerGroup) { |
||||||
|
List<String> normalWorkerAddresses = getWorkerServerAddressByGroup(workerGroup) |
||||||
|
.stream() |
||||||
|
.map(workerMapping::get) |
||||||
|
.filter(Objects::nonNull) |
||||||
|
.filter(workerServer -> workerServer.getServerStatus() == ServerStatus.NORMAL) |
||||||
|
.map(WorkerServerMetadata::getAddress) |
||||||
|
.collect(Collectors.toList()); |
||||||
|
return UnmodifiableList.unmodifiableList(normalWorkerAddresses); |
||||||
|
} |
||||||
|
|
||||||
|
public boolean containsWorkerGroup(String workerGroup) { |
||||||
|
return WorkerGroupUtils.getDefaultWorkerGroup().equals(workerGroup) |
||||||
|
|| workerGroupMapping.containsKey(workerGroup); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void registerListener(IClustersChangeListener<WorkerServerMetadata> listener) { |
||||||
|
workerClusterChangeListeners.add(listener); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onWorkerGroupDelete(List<WorkerGroup> workerGroups) { |
||||||
|
for (WorkerGroup workerGroup : workerGroups) { |
||||||
|
workerGroupMapping.remove(workerGroup.getName()); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onWorkerGroupAdd(List<WorkerGroup> workerGroups) { |
||||||
|
// The logic of adding WorkerGroup is the same as updating WorkerGroup
|
||||||
|
// Both need to change the WorkerGroup mapping to the latest
|
||||||
|
onWorkerGroupChange(workerGroups); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onWorkerGroupChange(List<WorkerGroup> workerGroups) { |
||||||
|
for (WorkerGroup workerGroup : workerGroups) { |
||||||
|
List<String> activeWorkers = WorkerGroupUtils.getWorkerAddressListFromWorkerGroup(workerGroup) |
||||||
|
.stream() |
||||||
|
.map(workerMapping::get) |
||||||
|
.filter(Objects::nonNull) |
||||||
|
.map(WorkerServerMetadata::getAddress) |
||||||
|
.collect(Collectors.toList()); |
||||||
|
workerGroupMapping.put(workerGroup.getName(), activeWorkers); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
WorkerServerMetadata parseServerFromHeartbeat(String serverHeartBeatJson) { |
||||||
|
WorkerHeartBeat workerHeartBeat = JSONUtils.parseObject(serverHeartBeatJson, WorkerHeartBeat.class); |
||||||
|
if (workerHeartBeat == null) { |
||||||
|
return null; |
||||||
|
} |
||||||
|
return WorkerServerMetadata.parseFromHeartBeat(workerHeartBeat); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerAdded(WorkerServerMetadata workerServer) { |
||||||
|
workerMapping.put(workerServer.getAddress(), workerServer); |
||||||
|
for (IClustersChangeListener<WorkerServerMetadata> listener : workerClusterChangeListeners) { |
||||||
|
listener.onServerAdded(workerServer); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerRemove(WorkerServerMetadata workerServer) { |
||||||
|
workerMapping.remove(workerServer.getAddress(), workerServer); |
||||||
|
for (IClustersChangeListener<WorkerServerMetadata> listener : workerClusterChangeListeners) { |
||||||
|
listener.onServerRemove(workerServer); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerUpdate(WorkerServerMetadata workerServer) { |
||||||
|
workerMapping.put(workerServer.getAddress(), workerServer); |
||||||
|
for (IClustersChangeListener<WorkerServerMetadata> listener : workerClusterChangeListeners) { |
||||||
|
listener.onServerUpdate(workerServer); |
||||||
|
} |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,116 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.utils.MapComparator; |
||||||
|
import org.apache.dolphinscheduler.dao.entity.WorkerGroup; |
||||||
|
import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; |
||||||
|
import org.apache.dolphinscheduler.server.master.utils.MasterThreadFactory; |
||||||
|
|
||||||
|
import org.apache.commons.collections4.CollectionUtils; |
||||||
|
|
||||||
|
import java.time.Duration; |
||||||
|
import java.util.HashMap; |
||||||
|
import java.util.List; |
||||||
|
import java.util.Map; |
||||||
|
import java.util.concurrent.CopyOnWriteArrayList; |
||||||
|
import java.util.concurrent.TimeUnit; |
||||||
|
import java.util.stream.Collectors; |
||||||
|
|
||||||
|
import lombok.extern.slf4j.Slf4j; |
||||||
|
|
||||||
|
import org.springframework.stereotype.Component; |
||||||
|
|
||||||
|
/** |
||||||
|
* Use to watch the worker group from database and notify the change. |
||||||
|
*/ |
||||||
|
@Slf4j |
||||||
|
@Component |
||||||
|
public class WorkerGroupChangeNotifier { |
||||||
|
|
||||||
|
private static final long DEFAULT_REFRESH_WORKER_INTERVAL = Duration.ofMinutes(1).toMillis(); |
||||||
|
|
||||||
|
private final WorkerGroupDao workerGroupDao; |
||||||
|
private final List<WorkerGroupListener> listeners = new CopyOnWriteArrayList<>(); |
||||||
|
|
||||||
|
private Map<String, WorkerGroup> workerGroupMap = new HashMap<>(); |
||||||
|
|
||||||
|
public WorkerGroupChangeNotifier(WorkerGroupDao workerGroupDao) { |
||||||
|
this.workerGroupDao = workerGroupDao; |
||||||
|
detectWorkerGroupChanges(); |
||||||
|
MasterThreadFactory.getDefaultSchedulerThreadExecutor().scheduleWithFixedDelay( |
||||||
|
this::detectWorkerGroupChanges, |
||||||
|
DEFAULT_REFRESH_WORKER_INTERVAL, |
||||||
|
DEFAULT_REFRESH_WORKER_INTERVAL, |
||||||
|
TimeUnit.SECONDS); |
||||||
|
} |
||||||
|
|
||||||
|
public void subscribeWorkerGroupsChange(WorkerGroupListener listener) { |
||||||
|
listeners.add(listener); |
||||||
|
} |
||||||
|
|
||||||
|
void detectWorkerGroupChanges() { |
||||||
|
try { |
||||||
|
MapComparator<String, WorkerGroup> mapComparator = detectChangedWorkerGroups(); |
||||||
|
triggerListeners(mapComparator); |
||||||
|
workerGroupMap = mapComparator.getNewMap(); |
||||||
|
} catch (Exception ex) { |
||||||
|
log.error("Detect WorkerGroup changes failed", ex); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
Map<String, WorkerGroup> getWorkerGroupMap() { |
||||||
|
return workerGroupMap; |
||||||
|
} |
||||||
|
|
||||||
|
private MapComparator<String, WorkerGroup> detectChangedWorkerGroups() { |
||||||
|
final Map<String, WorkerGroup> tmpWorkerGroupMap = workerGroupDao.queryAll() |
||||||
|
.stream() |
||||||
|
.collect(Collectors.toMap(WorkerGroup::getName, workerGroup -> workerGroup)); |
||||||
|
return new MapComparator<>(workerGroupMap, tmpWorkerGroupMap); |
||||||
|
} |
||||||
|
|
||||||
|
private void triggerListeners(MapComparator<String, WorkerGroup> mapComparator) { |
||||||
|
if (CollectionUtils.isEmpty(listeners)) { |
||||||
|
return; |
||||||
|
} |
||||||
|
final List<WorkerGroup> workerGroupsAdded = mapComparator.getValuesToAdd(); |
||||||
|
if (CollectionUtils.isNotEmpty(workerGroupsAdded)) { |
||||||
|
listeners.forEach(listener -> listener.onWorkerGroupAdd(workerGroupsAdded)); |
||||||
|
} |
||||||
|
|
||||||
|
final List<WorkerGroup> workerGroupsRemoved = mapComparator.getValuesToRemove(); |
||||||
|
if (CollectionUtils.isNotEmpty(workerGroupsRemoved)) { |
||||||
|
listeners.forEach(listener -> listener.onWorkerGroupDelete(workerGroupsRemoved)); |
||||||
|
} |
||||||
|
|
||||||
|
final List<WorkerGroup> workerGroupsUpdated = mapComparator.getNewValuesToUpdate(); |
||||||
|
if (CollectionUtils.isNotEmpty(workerGroupsUpdated)) { |
||||||
|
listeners.forEach(listener -> listener.onWorkerGroupChange(workerGroupsUpdated)); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
public interface WorkerGroupListener { |
||||||
|
|
||||||
|
void onWorkerGroupDelete(List<WorkerGroup> workerGroups); |
||||||
|
|
||||||
|
void onWorkerGroupAdd(List<WorkerGroup> workerGroups); |
||||||
|
|
||||||
|
void onWorkerGroupChange(List<WorkerGroup> workerGroups); |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,52 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; |
||||||
|
|
||||||
|
import lombok.Builder; |
||||||
|
import lombok.Data; |
||||||
|
import lombok.EqualsAndHashCode; |
||||||
|
import lombok.experimental.SuperBuilder; |
||||||
|
|
||||||
|
@Data |
||||||
|
@SuperBuilder |
||||||
|
@EqualsAndHashCode(callSuper = true) |
||||||
|
public class WorkerServerMetadata extends BaseServerMetadata { |
||||||
|
|
||||||
|
@Builder.Default |
||||||
|
private final String workerGroup = "default"; |
||||||
|
|
||||||
|
// Only used in FixedWeightedRoundRobinWorkerLoadBalancer
|
||||||
|
@Builder.Default |
||||||
|
private final double workerWeight = 1; |
||||||
|
|
||||||
|
private final double taskThreadPoolUsage; |
||||||
|
|
||||||
|
public static WorkerServerMetadata parseFromHeartBeat(WorkerHeartBeat workerHeartBeat) { |
||||||
|
return WorkerServerMetadata.builder() |
||||||
|
.address(workerHeartBeat.getHost() + ":" + workerHeartBeat.getPort()) |
||||||
|
.cpuUsage(workerHeartBeat.getCpuUsage()) |
||||||
|
.memoryUsage(workerHeartBeat.getMemoryUsage()) |
||||||
|
.serverStatus(workerHeartBeat.getServerStatus()) |
||||||
|
.workerWeight(workerHeartBeat.getWorkerHostWeight()) |
||||||
|
.taskThreadPoolUsage(workerHeartBeat.getThreadPoolUsage()) |
||||||
|
.build(); |
||||||
|
} |
||||||
|
|
||||||
|
} |
@ -0,0 +1,115 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster.loadbalancer; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.IClusters; |
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters; |
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerServerMetadata; |
||||||
|
|
||||||
|
import org.apache.commons.collections4.CollectionUtils; |
||||||
|
|
||||||
|
import java.util.List; |
||||||
|
import java.util.Map; |
||||||
|
import java.util.Objects; |
||||||
|
import java.util.Optional; |
||||||
|
import java.util.concurrent.ConcurrentHashMap; |
||||||
|
import java.util.concurrent.atomic.AtomicInteger; |
||||||
|
import java.util.stream.Collectors; |
||||||
|
|
||||||
|
import org.jetbrains.annotations.NotNull; |
||||||
|
|
||||||
|
/** |
||||||
|
* This load balancer is used to select a worker from {@link WorkerClusters} by dynamic weights. |
||||||
|
* </p> |
||||||
|
* The dynamic weights are calculated by the worker's load. e.g. cpu/memory/disk usage/thread usage etc. |
||||||
|
* You can config the weight calculation strategy in {@link WorkerLoadBalancerConfigurationProperties.DynamicWeightConfigProperties}. |
||||||
|
*/ |
||||||
|
public class DynamicWeightedRoundRobinWorkerLoadBalancer implements IWorkerLoadBalancer { |
||||||
|
|
||||||
|
private final WorkerClusters workerClusters; |
||||||
|
|
||||||
|
private final AtomicInteger robinIndex = new AtomicInteger(0); |
||||||
|
|
||||||
|
private Map<String, WeightedServer<WorkerServerMetadata>> weightedServerMap = new ConcurrentHashMap<>(); |
||||||
|
|
||||||
|
public DynamicWeightedRoundRobinWorkerLoadBalancer(WorkerClusters workerClusters, |
||||||
|
WorkerLoadBalancerConfigurationProperties.DynamicWeightConfigProperties dynamicWeightConfigProperties) { |
||||||
|
this.workerClusters = workerClusters; |
||||||
|
this.workerClusters.registerListener(new IClusters.IClustersChangeListener<WorkerServerMetadata>() { |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerAdded(WorkerServerMetadata server) { |
||||||
|
weightedServerMap.put(server.getAddress(), new WeightedServer<>(server, calculateWeight(server))); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerRemove(WorkerServerMetadata server) { |
||||||
|
weightedServerMap.remove(server.getAddress()); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerUpdate(WorkerServerMetadata server) { |
||||||
|
weightedServerMap.put(server.getAddress(), new WeightedServer<>(server, calculateWeight(server))); |
||||||
|
} |
||||||
|
|
||||||
|
private double calculateWeight(WorkerServerMetadata server) { |
||||||
|
return 100 - (dynamicWeightConfigProperties.getCpuUsageWeight() * server.getCpuUsage() |
||||||
|
+ dynamicWeightConfigProperties.getMemoryUsageWeight() * server.getMemoryUsage() |
||||||
|
+ dynamicWeightConfigProperties.getTaskThreadPoolUsageWeight() |
||||||
|
* server.getTaskThreadPoolUsage()) |
||||||
|
/ 3; |
||||||
|
} |
||||||
|
}); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public Optional<String> select(@NotNull String workerGroup) { |
||||||
|
List<WeightedServer<WorkerServerMetadata>> weightedServers = |
||||||
|
workerClusters.getNormalWorkerServerAddressByGroup(workerGroup) |
||||||
|
.stream() |
||||||
|
.map(weightedServerMap::get) |
||||||
|
.filter(Objects::nonNull) // filter non null here to avoid the two map changed between
|
||||||
|
// workerClusters
|
||||||
|
// and weightedServerMap is not atomic
|
||||||
|
.collect(Collectors.toList()); |
||||||
|
if (CollectionUtils.isEmpty(weightedServers)) { |
||||||
|
return Optional.empty(); |
||||||
|
} |
||||||
|
|
||||||
|
double totalWeight = weightedServers.stream().mapToDouble(WeightedServer::getWeight).sum(); |
||||||
|
|
||||||
|
WeightedServer<WorkerServerMetadata> selectedWorker = null; |
||||||
|
while (selectedWorker == null) { |
||||||
|
WeightedServer<WorkerServerMetadata> tmpWorker = |
||||||
|
weightedServers.get((robinIndex.incrementAndGet()) % weightedServers.size()); |
||||||
|
tmpWorker.setCurrentWeight(tmpWorker.getCurrentWeight() + tmpWorker.getWeight()); |
||||||
|
|
||||||
|
if (tmpWorker.getCurrentWeight() >= totalWeight) { |
||||||
|
tmpWorker.setCurrentWeight(tmpWorker.getCurrentWeight() - totalWeight); |
||||||
|
selectedWorker = tmpWorker; |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
return Optional.of(selectedWorker.getServer().getAddress()); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public WorkerLoadBalancerType getType() { |
||||||
|
return WorkerLoadBalancerType.DYNAMIC_WEIGHTED_ROUND_ROBIN; |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,109 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster.loadbalancer; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.IClusters; |
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters; |
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerServerMetadata; |
||||||
|
|
||||||
|
import org.apache.commons.collections4.CollectionUtils; |
||||||
|
|
||||||
|
import java.util.List; |
||||||
|
import java.util.Map; |
||||||
|
import java.util.Objects; |
||||||
|
import java.util.Optional; |
||||||
|
import java.util.concurrent.ConcurrentHashMap; |
||||||
|
import java.util.concurrent.atomic.AtomicInteger; |
||||||
|
import java.util.stream.Collectors; |
||||||
|
|
||||||
|
import org.jetbrains.annotations.NotNull; |
||||||
|
|
||||||
|
/** |
||||||
|
* This load balancer uses a fixed weighted round robin algorithm to select a worker from {@link WorkerClusters}. |
||||||
|
* <p> |
||||||
|
* e.g. if there are 3 worker servers with weight 1, 2, 3, the selection will be like: |
||||||
|
* 1, 2, 3, 1, 2, 3, 3, 1, 2, 3, 3, 1, 2, 3, ... |
||||||
|
* <p> |
||||||
|
* The weight of each worker server is determined by the worker server itself. |
||||||
|
*/ |
||||||
|
public class FixedWeightedRoundRobinWorkerLoadBalancer implements IWorkerLoadBalancer { |
||||||
|
|
||||||
|
private final WorkerClusters workerClusters; |
||||||
|
|
||||||
|
private final AtomicInteger robinIndex = new AtomicInteger(0); |
||||||
|
|
||||||
|
private final Map<String, WeightedServer<WorkerServerMetadata>> weightedServerMap = new ConcurrentHashMap<>(); |
||||||
|
|
||||||
|
public FixedWeightedRoundRobinWorkerLoadBalancer(WorkerClusters workerClusters) { |
||||||
|
this.workerClusters = workerClusters; |
||||||
|
this.workerClusters.registerListener(new IClusters.IClustersChangeListener<WorkerServerMetadata>() { |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerAdded(WorkerServerMetadata server) { |
||||||
|
weightedServerMap.put(server.getAddress(), new WeightedServer<>(server, server.getWorkerWeight())); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerRemove(WorkerServerMetadata server) { |
||||||
|
weightedServerMap.remove(server.getAddress(), new WeightedServer<>(server, server.getWorkerWeight())); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onServerUpdate(WorkerServerMetadata server) { |
||||||
|
// don't care the update event, since this will not affect the weight
|
||||||
|
} |
||||||
|
}); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public Optional<String> select(@NotNull String workerGroup) { |
||||||
|
List<WeightedServer<WorkerServerMetadata>> weightedServers = |
||||||
|
workerClusters.getNormalWorkerServerAddressByGroup(workerGroup) |
||||||
|
.stream() |
||||||
|
.map(weightedServerMap::get) |
||||||
|
// filter non null here to avoid the two map changed between
|
||||||
|
// workerClusters and weightedServerMap is not atomic
|
||||||
|
.filter(Objects::nonNull) |
||||||
|
.collect(Collectors.toList()); |
||||||
|
if (CollectionUtils.isEmpty(weightedServers)) { |
||||||
|
return Optional.empty(); |
||||||
|
} |
||||||
|
|
||||||
|
double totalWeight = weightedServers.stream().mapToDouble(WeightedServer::getWeight).sum(); |
||||||
|
|
||||||
|
WeightedServer<WorkerServerMetadata> selectedWorker = null; |
||||||
|
while (selectedWorker == null) { |
||||||
|
WeightedServer<WorkerServerMetadata> tmpWorker = |
||||||
|
weightedServers.get((robinIndex.incrementAndGet()) % weightedServers.size()); |
||||||
|
tmpWorker.setCurrentWeight(tmpWorker.getCurrentWeight() + tmpWorker.getWeight()); |
||||||
|
|
||||||
|
if (tmpWorker.getCurrentWeight() >= totalWeight) { |
||||||
|
tmpWorker.setCurrentWeight(tmpWorker.getCurrentWeight() - totalWeight); |
||||||
|
selectedWorker = tmpWorker; |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
return Optional.of(selectedWorker.getServer().getAddress()); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public WorkerLoadBalancerType getType() { |
||||||
|
return WorkerLoadBalancerType.FIXED_WEIGHTED_ROUND_ROBIN; |
||||||
|
} |
||||||
|
|
||||||
|
} |
@ -0,0 +1,55 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster.loadbalancer; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters; |
||||||
|
|
||||||
|
import org.apache.commons.collections4.CollectionUtils; |
||||||
|
|
||||||
|
import java.util.List; |
||||||
|
import java.util.Optional; |
||||||
|
import java.util.concurrent.ThreadLocalRandom; |
||||||
|
|
||||||
|
import org.jetbrains.annotations.NotNull; |
||||||
|
|
||||||
|
/** |
||||||
|
* The worker load balancer used to select a worker from the {@link WorkerClusters} by random algorithm. |
||||||
|
*/ |
||||||
|
public class RandomWorkerLoadBalancer implements IWorkerLoadBalancer { |
||||||
|
|
||||||
|
private final WorkerClusters workerClusters; |
||||||
|
|
||||||
|
public RandomWorkerLoadBalancer(WorkerClusters workerClusters) { |
||||||
|
this.workerClusters = workerClusters; |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public Optional<String> select(@NotNull String workerGroup) { |
||||||
|
List<String> workerServerAddresses = workerClusters.getNormalWorkerServerAddressByGroup(workerGroup); |
||||||
|
if (CollectionUtils.isEmpty(workerServerAddresses)) { |
||||||
|
return Optional.empty(); |
||||||
|
} |
||||||
|
int index = ThreadLocalRandom.current().nextInt(workerServerAddresses.size()); |
||||||
|
return Optional.of(workerServerAddresses.get(index)); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public WorkerLoadBalancerType getType() { |
||||||
|
return WorkerLoadBalancerType.RANDOM; |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,56 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster.loadbalancer; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters; |
||||||
|
|
||||||
|
import org.apache.commons.collections4.CollectionUtils; |
||||||
|
|
||||||
|
import java.util.List; |
||||||
|
import java.util.Optional; |
||||||
|
import java.util.concurrent.atomic.AtomicInteger; |
||||||
|
|
||||||
|
import org.jetbrains.annotations.NotNull; |
||||||
|
|
||||||
|
/** |
||||||
|
* The worker load balancer used to select a worker from the {@link WorkerClusters} by round-robin algorithm. |
||||||
|
*/ |
||||||
|
public class RoundRobinWorkerLoadBalancer implements IWorkerLoadBalancer { |
||||||
|
|
||||||
|
private final WorkerClusters workerClusters; |
||||||
|
|
||||||
|
private final AtomicInteger robinIndex = new AtomicInteger(0); |
||||||
|
|
||||||
|
public RoundRobinWorkerLoadBalancer(WorkerClusters workerClusters) { |
||||||
|
this.workerClusters = workerClusters; |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public Optional<String> select(@NotNull String workerGroup) { |
||||||
|
List<String> workerServerAddresses = workerClusters.getNormalWorkerServerAddressByGroup(workerGroup); |
||||||
|
if (CollectionUtils.isEmpty(workerServerAddresses)) { |
||||||
|
return Optional.empty(); |
||||||
|
} |
||||||
|
return Optional.of(workerServerAddresses.get(robinIndex.getAndIncrement() % workerServerAddresses.size())); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public WorkerLoadBalancerType getType() { |
||||||
|
return WorkerLoadBalancerType.ROUND_ROBIN; |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,45 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster.loadbalancer; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.IClusters; |
||||||
|
|
||||||
|
import lombok.AllArgsConstructor; |
||||||
|
import lombok.Builder; |
||||||
|
import lombok.Data; |
||||||
|
import lombok.NoArgsConstructor; |
||||||
|
|
||||||
|
@Data |
||||||
|
@Builder |
||||||
|
@AllArgsConstructor |
||||||
|
@NoArgsConstructor |
||||||
|
public class WeightedServer<T extends IClusters.IServerMetadata> { |
||||||
|
|
||||||
|
private T server; |
||||||
|
|
||||||
|
private double weight; |
||||||
|
|
||||||
|
private double currentWeight; |
||||||
|
|
||||||
|
public WeightedServer(T server, double weight) { |
||||||
|
this.server = server; |
||||||
|
this.weight = weight; |
||||||
|
this.currentWeight = 0; // Initialize currentWeight is 0
|
||||||
|
} |
||||||
|
|
||||||
|
} |
@ -0,0 +1,50 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster.loadbalancer; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.ClusterManager; |
||||||
|
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||||
|
|
||||||
|
import org.springframework.context.annotation.Bean; |
||||||
|
import org.springframework.context.annotation.Configuration; |
||||||
|
|
||||||
|
@Configuration |
||||||
|
public class WorkerLoadBalancerConfiguration { |
||||||
|
|
||||||
|
@Bean |
||||||
|
public IWorkerLoadBalancer randomWorkerLoadBalancer(MasterConfig masterConfig, ClusterManager clusterManager) { |
||||||
|
WorkerLoadBalancerConfigurationProperties workerLoadBalancerConfigurationProperties = |
||||||
|
masterConfig.getWorkerLoadBalancerConfigurationProperties(); |
||||||
|
switch (workerLoadBalancerConfigurationProperties.getType()) { |
||||||
|
case RANDOM: |
||||||
|
return new RandomWorkerLoadBalancer(clusterManager.getWorkerClusters()); |
||||||
|
case ROUND_ROBIN: |
||||||
|
return new RoundRobinWorkerLoadBalancer(clusterManager.getWorkerClusters()); |
||||||
|
case FIXED_WEIGHTED_ROUND_ROBIN: |
||||||
|
return new FixedWeightedRoundRobinWorkerLoadBalancer(clusterManager.getWorkerClusters()); |
||||||
|
case DYNAMIC_WEIGHTED_ROUND_ROBIN: |
||||||
|
return new DynamicWeightedRoundRobinWorkerLoadBalancer( |
||||||
|
clusterManager.getWorkerClusters(), |
||||||
|
workerLoadBalancerConfigurationProperties.getDynamicWeightConfigProperties()); |
||||||
|
default: |
||||||
|
throw new IllegalArgumentException( |
||||||
|
"unSupport worker load balancer type " + workerLoadBalancerConfigurationProperties.getType()); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
} |
@ -0,0 +1,61 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster.loadbalancer; |
||||||
|
|
||||||
|
import lombok.Data; |
||||||
|
|
||||||
|
import org.springframework.validation.Errors; |
||||||
|
|
||||||
|
@Data |
||||||
|
public class WorkerLoadBalancerConfigurationProperties { |
||||||
|
|
||||||
|
private WorkerLoadBalancerType type = WorkerLoadBalancerType.ROUND_ROBIN; |
||||||
|
|
||||||
|
private DynamicWeightConfigProperties dynamicWeightConfigProperties = new DynamicWeightConfigProperties(); |
||||||
|
|
||||||
|
public void validate(Errors errors) { |
||||||
|
dynamicWeightConfigProperties.validated(errors); |
||||||
|
} |
||||||
|
|
||||||
|
@Data |
||||||
|
public static class DynamicWeightConfigProperties { |
||||||
|
|
||||||
|
private int cpuUsageWeight = 30; |
||||||
|
|
||||||
|
private int memoryUsageWeight = 30; |
||||||
|
|
||||||
|
private int taskThreadPoolUsageWeight = 40; |
||||||
|
|
||||||
|
public void validated(Errors errors) { |
||||||
|
if (cpuUsageWeight < 0) { |
||||||
|
errors.rejectValue("cpuUsageWeight", "cpuUsageWeight", "cpuUsageWeight must >= 0"); |
||||||
|
} |
||||||
|
if (memoryUsageWeight < 0) { |
||||||
|
errors.rejectValue("memoryUsageWeight", "memoryUsageWeight", "memoryUsageWeight must >= 0"); |
||||||
|
} |
||||||
|
if (taskThreadPoolUsageWeight < 0) { |
||||||
|
errors.rejectValue("threadUsageWeight", "threadUsageWeight", "threadUsageWeight must >= 0"); |
||||||
|
} |
||||||
|
if (cpuUsageWeight + memoryUsageWeight + taskThreadPoolUsageWeight != 100) { |
||||||
|
errors.rejectValue("cpuUsageWeight", "cpuUsageWeight", |
||||||
|
"cpuUsageWeight + memoryUsageWeight + threadUsageWeight must be 100"); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
} |
||||||
|
} |
@ -1,47 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.exceptions; |
|
||||||
|
|
||||||
/** |
|
||||||
* execute exception |
|
||||||
*/ |
|
||||||
public class ExecuteException extends Exception { |
|
||||||
|
|
||||||
public ExecuteException() { |
|
||||||
super(); |
|
||||||
} |
|
||||||
|
|
||||||
public ExecuteException(String message) { |
|
||||||
super(message); |
|
||||||
} |
|
||||||
|
|
||||||
public ExecuteException(String message, Throwable cause) { |
|
||||||
super(message, cause); |
|
||||||
} |
|
||||||
|
|
||||||
public ExecuteException(Throwable cause) { |
|
||||||
super(cause); |
|
||||||
} |
|
||||||
|
|
||||||
protected ExecuteException(String message, Throwable cause, |
|
||||||
boolean enableSuppression, |
|
||||||
boolean writableStackTrace) { |
|
||||||
super(message, cause, enableSuppression, writableStackTrace); |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -1,69 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.extract.base.utils.Host; |
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; |
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWorker; |
|
||||||
import org.apache.dolphinscheduler.server.master.registry.ServerNodeManager; |
|
||||||
|
|
||||||
import org.apache.commons.collections4.CollectionUtils; |
|
||||||
|
|
||||||
import java.util.ArrayList; |
|
||||||
import java.util.Collection; |
|
||||||
import java.util.List; |
|
||||||
import java.util.Optional; |
|
||||||
import java.util.Set; |
|
||||||
|
|
||||||
import org.springframework.beans.factory.annotation.Autowired; |
|
||||||
|
|
||||||
/** |
|
||||||
* common host manager |
|
||||||
*/ |
|
||||||
public abstract class CommonHostManager implements HostManager { |
|
||||||
|
|
||||||
/** |
|
||||||
* server node manager |
|
||||||
*/ |
|
||||||
@Autowired |
|
||||||
protected ServerNodeManager serverNodeManager; |
|
||||||
|
|
||||||
@Override |
|
||||||
public Optional<Host> select(String workerGroup) throws WorkerGroupNotFoundException { |
|
||||||
List<HostWorker> candidates = getWorkerCandidates(workerGroup); |
|
||||||
if (CollectionUtils.isEmpty(candidates)) { |
|
||||||
return Optional.empty(); |
|
||||||
} |
|
||||||
return Optional.ofNullable(select(candidates)); |
|
||||||
} |
|
||||||
|
|
||||||
protected abstract HostWorker select(Collection<HostWorker> nodes); |
|
||||||
|
|
||||||
protected List<HostWorker> getWorkerCandidates(String workerGroup) throws WorkerGroupNotFoundException { |
|
||||||
List<HostWorker> hostWorkers = new ArrayList<>(); |
|
||||||
Set<String> nodes = serverNodeManager.getWorkerGroupNodes(workerGroup); |
|
||||||
if (CollectionUtils.isNotEmpty(nodes)) { |
|
||||||
for (String node : nodes) { |
|
||||||
serverNodeManager.getWorkerNodeInfo(node).ifPresent( |
|
||||||
workerNodeInfo -> hostWorkers |
|
||||||
.add(HostWorker.of(node, workerNodeInfo.getWorkerHostWeight(), workerGroup))); |
|
||||||
} |
|
||||||
} |
|
||||||
return hostWorkers; |
|
||||||
} |
|
||||||
} |
|
@ -1,64 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostSelector; |
|
||||||
|
|
||||||
import org.springframework.beans.factory.annotation.Autowired; |
|
||||||
import org.springframework.beans.factory.config.AutowireCapableBeanFactory; |
|
||||||
import org.springframework.context.annotation.Bean; |
|
||||||
import org.springframework.context.annotation.Configuration; |
|
||||||
|
|
||||||
/** |
|
||||||
* host manager config |
|
||||||
*/ |
|
||||||
@Configuration |
|
||||||
public class HostManagerConfig { |
|
||||||
|
|
||||||
private AutowireCapableBeanFactory beanFactory; |
|
||||||
|
|
||||||
@Autowired |
|
||||||
private MasterConfig masterConfig; |
|
||||||
|
|
||||||
@Autowired |
|
||||||
public HostManagerConfig(AutowireCapableBeanFactory beanFactory) { |
|
||||||
this.beanFactory = beanFactory; |
|
||||||
} |
|
||||||
|
|
||||||
@Bean |
|
||||||
public HostManager hostManager() { |
|
||||||
HostSelector selector = masterConfig.getHostSelector(); |
|
||||||
HostManager hostManager; |
|
||||||
switch (selector) { |
|
||||||
case RANDOM: |
|
||||||
hostManager = new RandomHostManager(); |
|
||||||
break; |
|
||||||
case ROUND_ROBIN: |
|
||||||
hostManager = new RoundRobinHostManager(); |
|
||||||
break; |
|
||||||
case LOWER_WEIGHT: |
|
||||||
hostManager = new LowerWeightHostManager(); |
|
||||||
break; |
|
||||||
default: |
|
||||||
throw new IllegalArgumentException("unSupport selector " + selector); |
|
||||||
} |
|
||||||
beanFactory.autowireBean(hostManager); |
|
||||||
return hostManager; |
|
||||||
} |
|
||||||
} |
|
@ -1,165 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
|
||||||
import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; |
|
||||||
import org.apache.dolphinscheduler.extract.base.utils.Host; |
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; |
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWeight; |
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWorker; |
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.host.assign.LowerWeightRoundRobin; |
|
||||||
import org.apache.dolphinscheduler.server.master.registry.WorkerInfoChangeListener; |
|
||||||
|
|
||||||
import org.apache.commons.collections4.CollectionUtils; |
|
||||||
|
|
||||||
import java.util.Collection; |
|
||||||
import java.util.HashMap; |
|
||||||
import java.util.HashSet; |
|
||||||
import java.util.Map; |
|
||||||
import java.util.Optional; |
|
||||||
import java.util.Set; |
|
||||||
import java.util.concurrent.ConcurrentHashMap; |
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock; |
|
||||||
|
|
||||||
import javax.annotation.PostConstruct; |
|
||||||
|
|
||||||
import lombok.extern.slf4j.Slf4j; |
|
||||||
|
|
||||||
/** |
|
||||||
* lower weight host manager |
|
||||||
*/ |
|
||||||
@Slf4j |
|
||||||
public class LowerWeightHostManager extends CommonHostManager { |
|
||||||
|
|
||||||
/** |
|
||||||
* selector |
|
||||||
*/ |
|
||||||
private LowerWeightRoundRobin selector; |
|
||||||
|
|
||||||
/** |
|
||||||
* worker host weights |
|
||||||
*/ |
|
||||||
private ConcurrentHashMap<String, Set<HostWeight>> workerHostWeightsMap; |
|
||||||
|
|
||||||
private final ReentrantReadWriteLock workerGroupLock = new ReentrantReadWriteLock(); |
|
||||||
|
|
||||||
private final ReentrantReadWriteLock.ReadLock workerGroupReadLock = workerGroupLock.readLock(); |
|
||||||
private final ReentrantReadWriteLock.WriteLock workerGroupWriteLock = workerGroupLock.writeLock(); |
|
||||||
|
|
||||||
@PostConstruct |
|
||||||
public void init() { |
|
||||||
this.selector = new LowerWeightRoundRobin(); |
|
||||||
this.workerHostWeightsMap = new ConcurrentHashMap<>(); |
|
||||||
serverNodeManager.addWorkerInfoChangeListener(new WorkerWeightListener()); |
|
||||||
} |
|
||||||
|
|
||||||
@Override |
|
||||||
public Optional<Host> select(String workerGroup) throws WorkerGroupNotFoundException { |
|
||||||
Set<HostWeight> workerHostWeights = getWorkerHostWeights(workerGroup); |
|
||||||
if (CollectionUtils.isNotEmpty(workerHostWeights)) { |
|
||||||
return Optional.ofNullable(selector.select(workerHostWeights).getHost()); |
|
||||||
} |
|
||||||
return Optional.empty(); |
|
||||||
} |
|
||||||
|
|
||||||
@Override |
|
||||||
public HostWorker select(Collection<HostWorker> nodes) { |
|
||||||
throw new UnsupportedOperationException("not support"); |
|
||||||
} |
|
||||||
|
|
||||||
private class WorkerWeightListener implements WorkerInfoChangeListener { |
|
||||||
|
|
||||||
@Override |
|
||||||
public void notify(Map<String, Set<String>> workerGroups, Map<String, WorkerHeartBeat> workerNodeInfo) { |
|
||||||
syncWorkerResources(workerGroups, workerNodeInfo); |
|
||||||
} |
|
||||||
|
|
||||||
/** |
|
||||||
* Sync worker resource. |
|
||||||
* |
|
||||||
* @param workerGroupNodes worker group nodes, key is worker group, value is worker group nodes. |
|
||||||
* @param workerNodeInfoMap worker node info map, key is worker node, value is worker info. |
|
||||||
*/ |
|
||||||
private void syncWorkerResources(final Map<String, Set<String>> workerGroupNodes, |
|
||||||
final Map<String, WorkerHeartBeat> workerNodeInfoMap) { |
|
||||||
try { |
|
||||||
Map<String, Set<HostWeight>> workerHostWeights = new HashMap<>(); |
|
||||||
for (Map.Entry<String, Set<String>> entry : workerGroupNodes.entrySet()) { |
|
||||||
String workerGroup = entry.getKey(); |
|
||||||
Set<String> nodes = entry.getValue(); |
|
||||||
Set<HostWeight> hostWeights = new HashSet<>(nodes.size()); |
|
||||||
for (String node : nodes) { |
|
||||||
WorkerHeartBeat heartbeat = workerNodeInfoMap.getOrDefault(node, null); |
|
||||||
Optional<HostWeight> hostWeightOpt = getHostWeight(node, workerGroup, heartbeat); |
|
||||||
hostWeightOpt.ifPresent(hostWeights::add); |
|
||||||
} |
|
||||||
if (!hostWeights.isEmpty()) { |
|
||||||
workerHostWeights.put(workerGroup, hostWeights); |
|
||||||
} |
|
||||||
} |
|
||||||
syncWorkerHostWeight(workerHostWeights); |
|
||||||
} catch (Throwable ex) { |
|
||||||
log.error("Sync worker resource error", ex); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
private void syncWorkerHostWeight(Map<String, Set<HostWeight>> workerHostWeights) { |
|
||||||
workerGroupWriteLock.lock(); |
|
||||||
try { |
|
||||||
workerHostWeightsMap.clear(); |
|
||||||
workerHostWeightsMap.putAll(workerHostWeights); |
|
||||||
} finally { |
|
||||||
workerGroupWriteLock.unlock(); |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
public Optional<HostWeight> getHostWeight(String workerAddress, String workerGroup, WorkerHeartBeat heartBeat) { |
|
||||||
if (heartBeat == null) { |
|
||||||
log.warn("Worker {} in WorkerGroup {} have not received the heartbeat", workerAddress, workerGroup); |
|
||||||
return Optional.empty(); |
|
||||||
} |
|
||||||
if (ServerStatus.BUSY == heartBeat.getServerStatus()) { |
|
||||||
log.warn("Worker {} in workerGroup {} is Busy, heartbeat is {}", workerAddress, workerGroup, heartBeat); |
|
||||||
return Optional.empty(); |
|
||||||
} |
|
||||||
return Optional.of( |
|
||||||
new HostWeight( |
|
||||||
HostWorker.of(workerAddress, heartBeat.getWorkerHostWeight(), workerGroup), |
|
||||||
heartBeat.getCpuUsage(), |
|
||||||
heartBeat.getMemoryUsage(), |
|
||||||
heartBeat.getDiskUsage(), |
|
||||||
heartBeat.getThreadPoolUsage(), |
|
||||||
heartBeat.getStartupTime())); |
|
||||||
} |
|
||||||
|
|
||||||
private Set<HostWeight> getWorkerHostWeights(String workerGroup) throws WorkerGroupNotFoundException { |
|
||||||
workerGroupReadLock.lock(); |
|
||||||
try { |
|
||||||
Set<HostWeight> hostWeights = workerHostWeightsMap.get(workerGroup); |
|
||||||
if (hostWeights == null) { |
|
||||||
throw new WorkerGroupNotFoundException("Can not find worker group " + workerGroup); |
|
||||||
} |
|
||||||
return hostWeights; |
|
||||||
} finally { |
|
||||||
workerGroupReadLock.unlock(); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -1,47 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWorker; |
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.host.assign.RandomSelector; |
|
||||||
|
|
||||||
import java.util.Collection; |
|
||||||
|
|
||||||
/** |
|
||||||
* random host manager |
|
||||||
*/ |
|
||||||
public class RandomHostManager extends CommonHostManager { |
|
||||||
|
|
||||||
/** |
|
||||||
* selector |
|
||||||
*/ |
|
||||||
private final RandomSelector selector; |
|
||||||
|
|
||||||
/** |
|
||||||
* set round robin |
|
||||||
*/ |
|
||||||
public RandomHostManager() { |
|
||||||
this.selector = new RandomSelector(); |
|
||||||
} |
|
||||||
|
|
||||||
@Override |
|
||||||
public HostWorker select(Collection<HostWorker> nodes) { |
|
||||||
return selector.select(nodes); |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -1,47 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWorker; |
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.host.assign.RoundRobinSelector; |
|
||||||
|
|
||||||
import java.util.Collection; |
|
||||||
|
|
||||||
/** |
|
||||||
* round robin host manager |
|
||||||
*/ |
|
||||||
public class RoundRobinHostManager extends CommonHostManager { |
|
||||||
|
|
||||||
/** |
|
||||||
* selector |
|
||||||
*/ |
|
||||||
private final RoundRobinSelector selector; |
|
||||||
|
|
||||||
/** |
|
||||||
* set round robin |
|
||||||
*/ |
|
||||||
public RoundRobinHostManager() { |
|
||||||
this.selector = new RoundRobinSelector(); |
|
||||||
} |
|
||||||
|
|
||||||
@Override |
|
||||||
public HostWorker select(Collection<HostWorker> nodes) { |
|
||||||
return selector.select(nodes); |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -1,47 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host.assign; |
|
||||||
|
|
||||||
import org.apache.commons.collections4.CollectionUtils; |
|
||||||
|
|
||||||
import java.util.Collection; |
|
||||||
|
|
||||||
/** |
|
||||||
* AbstractSelector |
|
||||||
*/ |
|
||||||
public abstract class AbstractSelector<T> implements Selector<T> { |
|
||||||
|
|
||||||
@Override |
|
||||||
public T select(Collection<T> source) { |
|
||||||
|
|
||||||
if (CollectionUtils.isEmpty(source)) { |
|
||||||
throw new IllegalArgumentException("Empty source."); |
|
||||||
} |
|
||||||
|
|
||||||
/** |
|
||||||
* if only one , return directly |
|
||||||
*/ |
|
||||||
if (source.size() == 1) { |
|
||||||
return (T) source.toArray()[0]; |
|
||||||
} |
|
||||||
return doSelect(source); |
|
||||||
} |
|
||||||
|
|
||||||
protected abstract T doSelect(Collection<T> source); |
|
||||||
|
|
||||||
} |
|
@ -1,69 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host.assign; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.extract.base.utils.Constants; |
|
||||||
import org.apache.dolphinscheduler.extract.base.utils.Host; |
|
||||||
|
|
||||||
import lombok.Data; |
|
||||||
|
|
||||||
@Data |
|
||||||
public class HostWeight { |
|
||||||
|
|
||||||
private final int THREAD_USAGE_FACTOR = 10; |
|
||||||
|
|
||||||
private final int CPU_USAGE_FACTOR = 20; |
|
||||||
|
|
||||||
private final int MEMORY_USAGE_FACTOR = 20; |
|
||||||
|
|
||||||
private final int DISK_USAGE_FACTOR = 50; |
|
||||||
|
|
||||||
private final Host host; |
|
||||||
|
|
||||||
private final double weight; |
|
||||||
|
|
||||||
// if the weight is small, then is will be chosen first
|
|
||||||
private double currentWeight; |
|
||||||
|
|
||||||
public HostWeight(HostWorker hostWorker, |
|
||||||
double cpuUsage, |
|
||||||
double memoryUsage, |
|
||||||
double diskUsage, |
|
||||||
double threadPoolUsage, |
|
||||||
long startTime) { |
|
||||||
this.host = hostWorker; |
|
||||||
this.weight = calculateWeight(cpuUsage, memoryUsage, diskUsage, threadPoolUsage, startTime); |
|
||||||
this.currentWeight = this.weight; |
|
||||||
} |
|
||||||
|
|
||||||
private double calculateWeight(double cpuUsage, |
|
||||||
double memoryUsage, |
|
||||||
double diskUsage, |
|
||||||
double threadPoolUsage, |
|
||||||
long startTime) { |
|
||||||
double calculatedWeight = 100 - (cpuUsage * CPU_USAGE_FACTOR + memoryUsage * MEMORY_USAGE_FACTOR |
|
||||||
+ diskUsage * DISK_USAGE_FACTOR + threadPoolUsage * THREAD_USAGE_FACTOR); |
|
||||||
long uptime = System.currentTimeMillis() - startTime; |
|
||||||
if (uptime > 0 && uptime < Constants.WARM_UP_TIME) { |
|
||||||
// If the warm-up is not over, add the weight
|
|
||||||
return calculatedWeight * Constants.WARM_UP_TIME / uptime; |
|
||||||
} |
|
||||||
return calculatedWeight; |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -1,77 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host.assign; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.extract.base.utils.Host; |
|
||||||
|
|
||||||
/** |
|
||||||
* host worker |
|
||||||
*/ |
|
||||||
public class HostWorker extends Host { |
|
||||||
|
|
||||||
/** |
|
||||||
* host weight |
|
||||||
*/ |
|
||||||
private int hostWeight; |
|
||||||
|
|
||||||
/** |
|
||||||
* worker group |
|
||||||
*/ |
|
||||||
private String workerGroup; |
|
||||||
|
|
||||||
public HostWorker(String ip, int port, int hostWeight, String workerGroup) { |
|
||||||
super(ip, port); |
|
||||||
this.hostWeight = hostWeight; |
|
||||||
this.workerGroup = workerGroup; |
|
||||||
} |
|
||||||
|
|
||||||
public HostWorker(String address, int hostWeight, String workerGroup) { |
|
||||||
super(address); |
|
||||||
this.hostWeight = hostWeight; |
|
||||||
this.workerGroup = workerGroup; |
|
||||||
} |
|
||||||
|
|
||||||
public int getHostWeight() { |
|
||||||
return hostWeight; |
|
||||||
} |
|
||||||
|
|
||||||
public void setHostWeight(int hostWeight) { |
|
||||||
this.hostWeight = hostWeight; |
|
||||||
} |
|
||||||
|
|
||||||
public String getWorkerGroup() { |
|
||||||
return workerGroup; |
|
||||||
} |
|
||||||
|
|
||||||
public void setWorkerGroup(String workerGroup) { |
|
||||||
this.workerGroup = workerGroup; |
|
||||||
} |
|
||||||
|
|
||||||
public static HostWorker of(String address, int hostWeight, String workerGroup) { |
|
||||||
return new HostWorker(address, hostWeight, workerGroup); |
|
||||||
} |
|
||||||
|
|
||||||
@Override |
|
||||||
public String toString() { |
|
||||||
return "Host{" |
|
||||||
+ "hostWeight=" + hostWeight |
|
||||||
+ ", workerGroup='" + workerGroup + '\'' |
|
||||||
+ '}'; |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -1,54 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host.assign; |
|
||||||
|
|
||||||
import java.util.Collection; |
|
||||||
|
|
||||||
/** |
|
||||||
* lower weight round robin |
|
||||||
*/ |
|
||||||
public class LowerWeightRoundRobin extends AbstractSelector<HostWeight> { |
|
||||||
|
|
||||||
/** |
|
||||||
* Selects a HostWeight from a collection of HostWeight objects. |
|
||||||
* The selection is based on the current weight of each HostWeight. |
|
||||||
* The HostWeight with the smallest current weight is selected. |
|
||||||
* |
|
||||||
* @param sources A collection of HostWeight objects to select from. |
|
||||||
* @return The selected HostWeight with the smallest current weight. |
|
||||||
*/ |
|
||||||
@Override |
|
||||||
public HostWeight doSelect(Collection<HostWeight> sources) { |
|
||||||
double totalWeight = 0; |
|
||||||
double lowWeight = 0; |
|
||||||
HostWeight lowerNode = null; |
|
||||||
for (HostWeight hostWeight : sources) { |
|
||||||
totalWeight += hostWeight.getWeight(); |
|
||||||
hostWeight.setCurrentWeight(hostWeight.getCurrentWeight() + hostWeight.getWeight()); |
|
||||||
if (lowerNode == null || lowWeight > hostWeight.getCurrentWeight()) { |
|
||||||
lowerNode = hostWeight; |
|
||||||
lowWeight = hostWeight.getCurrentWeight(); |
|
||||||
} |
|
||||||
} |
|
||||||
if (lowerNode != null) { |
|
||||||
lowerNode.setCurrentWeight(lowerNode.getCurrentWeight() + totalWeight); |
|
||||||
} |
|
||||||
return lowerNode; |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -1,66 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host.assign; |
|
||||||
|
|
||||||
import java.util.ArrayList; |
|
||||||
import java.util.Collection; |
|
||||||
import java.util.List; |
|
||||||
import java.util.concurrent.ThreadLocalRandom; |
|
||||||
|
|
||||||
/** |
|
||||||
* random selector |
|
||||||
*/ |
|
||||||
public class RandomSelector extends AbstractSelector<HostWorker> { |
|
||||||
|
|
||||||
/** |
|
||||||
* This method selects a HostWorker from a collection of HostWorker objects using a weighted random algorithm. |
|
||||||
* The selection is based on the weight of each HostWorker. |
|
||||||
* A random number is generated and the HostWorker whose weight spans this random number is selected. |
|
||||||
* |
|
||||||
* @param source A collection of HostWorker objects to select from. |
|
||||||
* @return The selected HostWorker based on the weighted random algorithm. |
|
||||||
*/ |
|
||||||
@Override |
|
||||||
public HostWorker doSelect(final Collection<HostWorker> source) { |
|
||||||
|
|
||||||
List<HostWorker> hosts = new ArrayList<>(source); |
|
||||||
int size = hosts.size(); |
|
||||||
int[] weights = new int[size]; |
|
||||||
int totalWeight = 0; |
|
||||||
int index = 0; |
|
||||||
|
|
||||||
for (HostWorker host : hosts) { |
|
||||||
totalWeight += host.getHostWeight(); |
|
||||||
weights[index] = host.getHostWeight(); |
|
||||||
index++; |
|
||||||
} |
|
||||||
|
|
||||||
if (totalWeight > 0) { |
|
||||||
int offset = ThreadLocalRandom.current().nextInt(totalWeight); |
|
||||||
|
|
||||||
for (int i = 0; i < size; i++) { |
|
||||||
offset -= weights[i]; |
|
||||||
if (offset < 0) { |
|
||||||
return hosts.get(i); |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
return hosts.get(ThreadLocalRandom.current().nextInt(size)); |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -1,148 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host.assign; |
|
||||||
|
|
||||||
import java.util.ArrayList; |
|
||||||
import java.util.Collection; |
|
||||||
import java.util.List; |
|
||||||
import java.util.concurrent.ConcurrentHashMap; |
|
||||||
import java.util.concurrent.ConcurrentMap; |
|
||||||
import java.util.concurrent.atomic.AtomicBoolean; |
|
||||||
import java.util.concurrent.atomic.AtomicLong; |
|
||||||
|
|
||||||
import org.springframework.stereotype.Service; |
|
||||||
|
|
||||||
/** |
|
||||||
* Smooth Weight Round Robin |
|
||||||
*/ |
|
||||||
@Service |
|
||||||
public class RoundRobinSelector extends AbstractSelector<HostWorker> { |
|
||||||
|
|
||||||
private ConcurrentMap<String, ConcurrentMap<String, WeightedRoundRobin>> workGroupWeightMap = |
|
||||||
new ConcurrentHashMap<>(); |
|
||||||
|
|
||||||
private static final int RECYCLE_PERIOD = 100000; |
|
||||||
|
|
||||||
private AtomicBoolean updateLock = new AtomicBoolean(); |
|
||||||
|
|
||||||
protected static class WeightedRoundRobin { |
|
||||||
|
|
||||||
private int weight; |
|
||||||
private AtomicLong current = new AtomicLong(0); |
|
||||||
private long lastUpdate; |
|
||||||
|
|
||||||
int getWeight() { |
|
||||||
return weight; |
|
||||||
} |
|
||||||
|
|
||||||
void setWeight(int weight) { |
|
||||||
this.weight = weight; |
|
||||||
current.set(0); |
|
||||||
} |
|
||||||
|
|
||||||
long increaseCurrent() { |
|
||||||
return current.addAndGet(weight); |
|
||||||
} |
|
||||||
|
|
||||||
void sel(int total) { |
|
||||||
current.addAndGet(-1L * total); |
|
||||||
} |
|
||||||
|
|
||||||
long getLastUpdate() { |
|
||||||
return lastUpdate; |
|
||||||
} |
|
||||||
|
|
||||||
void setLastUpdate(long lastUpdate) { |
|
||||||
this.lastUpdate = lastUpdate; |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
||||||
|
|
||||||
/** |
|
||||||
* This method selects a HostWorker from a collection of HostWorker objects using a weighted round-robin algorithm. |
|
||||||
* The selection is based on the current weight of each HostWorker. |
|
||||||
* The HostWorker with the highest current weight is selected. |
|
||||||
* |
|
||||||
* @param source A collection of HostWorker objects to select from. |
|
||||||
* @return The selected HostWorker with the highest current weight. |
|
||||||
*/ |
|
||||||
@Override |
|
||||||
public HostWorker doSelect(Collection<HostWorker> source) { |
|
||||||
|
|
||||||
List<HostWorker> hosts = new ArrayList<>(source); |
|
||||||
String key = hosts.get(0).getWorkerGroup(); |
|
||||||
ConcurrentMap<String, WeightedRoundRobin> map = workGroupWeightMap.get(key); |
|
||||||
if (map == null) { |
|
||||||
workGroupWeightMap.putIfAbsent(key, new ConcurrentHashMap<>()); |
|
||||||
map = workGroupWeightMap.get(key); |
|
||||||
} |
|
||||||
|
|
||||||
int totalWeight = 0; |
|
||||||
long maxCurrent = Long.MIN_VALUE; |
|
||||||
long now = System.currentTimeMillis(); |
|
||||||
HostWorker selectedHost = null; |
|
||||||
WeightedRoundRobin selectWeightRoundRobin = null; |
|
||||||
|
|
||||||
for (HostWorker host : hosts) { |
|
||||||
String workGroupHost = host.getWorkerGroup() + host.getAddress(); |
|
||||||
WeightedRoundRobin weightedRoundRobin = map.get(workGroupHost); |
|
||||||
int weight = host.getHostWeight(); |
|
||||||
if (weight < 0) { |
|
||||||
weight = 0; |
|
||||||
} |
|
||||||
|
|
||||||
if (weightedRoundRobin == null) { |
|
||||||
weightedRoundRobin = new WeightedRoundRobin(); |
|
||||||
// set weight
|
|
||||||
weightedRoundRobin.setWeight(weight); |
|
||||||
map.putIfAbsent(workGroupHost, weightedRoundRobin); |
|
||||||
weightedRoundRobin = map.get(workGroupHost); |
|
||||||
} |
|
||||||
if (weight != weightedRoundRobin.getWeight()) { |
|
||||||
weightedRoundRobin.setWeight(weight); |
|
||||||
} |
|
||||||
|
|
||||||
long cur = weightedRoundRobin.increaseCurrent(); |
|
||||||
weightedRoundRobin.setLastUpdate(now); |
|
||||||
if (cur > maxCurrent) { |
|
||||||
maxCurrent = cur; |
|
||||||
selectedHost = host; |
|
||||||
selectWeightRoundRobin = weightedRoundRobin; |
|
||||||
} |
|
||||||
|
|
||||||
totalWeight += weight; |
|
||||||
} |
|
||||||
|
|
||||||
if (!updateLock.get() && hosts.size() != map.size() && updateLock.compareAndSet(false, true)) { |
|
||||||
try { |
|
||||||
ConcurrentMap<String, WeightedRoundRobin> newMap = new ConcurrentHashMap<>(map); |
|
||||||
newMap.entrySet().removeIf(item -> now - item.getValue().getLastUpdate() > RECYCLE_PERIOD); |
|
||||||
workGroupWeightMap.put(key, newMap); |
|
||||||
} finally { |
|
||||||
updateLock.set(false); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
if (selectedHost != null) { |
|
||||||
selectWeightRoundRobin.sel(totalWeight); |
|
||||||
return selectedHost; |
|
||||||
} |
|
||||||
|
|
||||||
return hosts.get(0); |
|
||||||
} |
|
||||||
} |
|
@ -1,114 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.registry; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
|
||||||
import org.apache.dolphinscheduler.common.model.MasterHeartBeat; |
|
||||||
import org.apache.dolphinscheduler.common.model.Server; |
|
||||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
|
||||||
import org.apache.dolphinscheduler.service.queue.MasterPriorityQueue; |
|
||||||
|
|
||||||
import java.util.Date; |
|
||||||
import java.util.List; |
|
||||||
import java.util.Map; |
|
||||||
import java.util.concurrent.locks.Lock; |
|
||||||
import java.util.concurrent.locks.ReentrantLock; |
|
||||||
import java.util.stream.Collectors; |
|
||||||
|
|
||||||
import lombok.extern.slf4j.Slf4j; |
|
||||||
|
|
||||||
import org.springframework.beans.factory.annotation.Autowired; |
|
||||||
import org.springframework.stereotype.Service; |
|
||||||
|
|
||||||
@Service |
|
||||||
@Slf4j |
|
||||||
public class MasterSlotManager { |
|
||||||
|
|
||||||
@Autowired |
|
||||||
protected ServerNodeManager serverNodeManager; |
|
||||||
|
|
||||||
@Autowired |
|
||||||
protected MasterConfig masterConfig; |
|
||||||
|
|
||||||
private volatile int currentSlot = 0; |
|
||||||
private volatile int totalSlot = 0; |
|
||||||
|
|
||||||
public void start() { |
|
||||||
serverNodeManager.addMasterInfoChangeListener(new MasterSlotManager.SlotChangeListener()); |
|
||||||
} |
|
||||||
|
|
||||||
public int getSlot() { |
|
||||||
return currentSlot; |
|
||||||
} |
|
||||||
|
|
||||||
public int getMasterSize() { |
|
||||||
return totalSlot; |
|
||||||
} |
|
||||||
|
|
||||||
public class SlotChangeListener implements MasterInfoChangeListener { |
|
||||||
|
|
||||||
private final Lock slotLock = new ReentrantLock(); |
|
||||||
|
|
||||||
private final MasterPriorityQueue masterPriorityQueue = new MasterPriorityQueue(); |
|
||||||
|
|
||||||
@Override |
|
||||||
public void notify(Map<String, MasterHeartBeat> masterNodeInfo) { |
|
||||||
List<Server> serverList = masterNodeInfo.values().stream() |
|
||||||
.filter(heartBeat -> !heartBeat.getServerStatus().equals(ServerStatus.BUSY)) |
|
||||||
.map(this::convertHeartBeatToServer) |
|
||||||
.collect(Collectors.toList()); |
|
||||||
syncMasterNodes(serverList); |
|
||||||
} |
|
||||||
|
|
||||||
/** |
|
||||||
* sync master nodes |
|
||||||
*/ |
|
||||||
private void syncMasterNodes(List<Server> masterNodes) { |
|
||||||
slotLock.lock(); |
|
||||||
try { |
|
||||||
this.masterPriorityQueue.clear(); |
|
||||||
this.masterPriorityQueue.putAll(masterNodes); |
|
||||||
int tempCurrentSlot = masterPriorityQueue.getIndex(masterConfig.getMasterAddress()); |
|
||||||
int tempTotalSlot = masterNodes.size(); |
|
||||||
if (tempCurrentSlot < 0) { |
|
||||||
totalSlot = 0; |
|
||||||
currentSlot = 0; |
|
||||||
log.warn("Current master is not in active master list"); |
|
||||||
} else if (tempCurrentSlot != currentSlot || tempTotalSlot != totalSlot) { |
|
||||||
totalSlot = tempTotalSlot; |
|
||||||
currentSlot = tempCurrentSlot; |
|
||||||
log.info("Update master nodes, total master size: {}, current slot: {}", totalSlot, currentSlot); |
|
||||||
} |
|
||||||
} finally { |
|
||||||
slotLock.unlock(); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
private Server convertHeartBeatToServer(MasterHeartBeat masterHeartBeat) { |
|
||||||
Server server = new Server(); |
|
||||||
server.setCreateTime(new Date(masterHeartBeat.getStartupTime())); |
|
||||||
server.setLastHeartbeatTime(new Date(masterHeartBeat.getReportTime())); |
|
||||||
server.setId(masterHeartBeat.getProcessId()); |
|
||||||
server.setHost(masterHeartBeat.getHost()); |
|
||||||
server.setPort(masterHeartBeat.getPort()); |
|
||||||
|
|
||||||
return server; |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
||||||
} |
|
@ -1,374 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.registry; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.common.constants.Constants; |
|
||||||
import org.apache.dolphinscheduler.common.model.MasterHeartBeat; |
|
||||||
import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; |
|
||||||
import org.apache.dolphinscheduler.common.thread.ThreadUtils; |
|
||||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
|
||||||
import org.apache.dolphinscheduler.dao.AlertDao; |
|
||||||
import org.apache.dolphinscheduler.dao.entity.WorkerGroup; |
|
||||||
import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper; |
|
||||||
import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils; |
|
||||||
import org.apache.dolphinscheduler.registry.api.Event; |
|
||||||
import org.apache.dolphinscheduler.registry.api.Event.Type; |
|
||||||
import org.apache.dolphinscheduler.registry.api.RegistryClient; |
|
||||||
import org.apache.dolphinscheduler.registry.api.SubscribeListener; |
|
||||||
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; |
|
||||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; |
|
||||||
import org.apache.dolphinscheduler.service.alert.ListenerEventAlertManager; |
|
||||||
|
|
||||||
import org.apache.commons.collections4.CollectionUtils; |
|
||||||
import org.apache.commons.lang3.ArrayUtils; |
|
||||||
|
|
||||||
import java.util.ArrayList; |
|
||||||
import java.util.Arrays; |
|
||||||
import java.util.Collections; |
|
||||||
import java.util.HashMap; |
|
||||||
import java.util.List; |
|
||||||
import java.util.Map; |
|
||||||
import java.util.Optional; |
|
||||||
import java.util.Set; |
|
||||||
import java.util.concurrent.ConcurrentHashMap; |
|
||||||
import java.util.concurrent.Executors; |
|
||||||
import java.util.concurrent.ScheduledExecutorService; |
|
||||||
import java.util.concurrent.TimeUnit; |
|
||||||
import java.util.concurrent.locks.ReentrantLock; |
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock; |
|
||||||
import java.util.stream.Collectors; |
|
||||||
|
|
||||||
import javax.annotation.PreDestroy; |
|
||||||
|
|
||||||
import lombok.extern.slf4j.Slf4j; |
|
||||||
|
|
||||||
import org.springframework.beans.factory.InitializingBean; |
|
||||||
import org.springframework.beans.factory.annotation.Autowired; |
|
||||||
import org.springframework.stereotype.Service; |
|
||||||
|
|
||||||
@Service |
|
||||||
@Slf4j |
|
||||||
public class ServerNodeManager implements InitializingBean { |
|
||||||
|
|
||||||
private final ReentrantReadWriteLock workerGroupLock = new ReentrantReadWriteLock(); |
|
||||||
private final ReentrantReadWriteLock.ReadLock workerGroupReadLock = workerGroupLock.readLock(); |
|
||||||
private final ReentrantReadWriteLock.WriteLock workerGroupWriteLock = workerGroupLock.writeLock(); |
|
||||||
|
|
||||||
private final ReentrantReadWriteLock workerNodeInfoLock = new ReentrantReadWriteLock(); |
|
||||||
private final ReentrantReadWriteLock.ReadLock workerNodeInfoReadLock = workerNodeInfoLock.readLock(); |
|
||||||
private final ReentrantReadWriteLock.WriteLock workerNodeInfoWriteLock = workerNodeInfoLock.writeLock(); |
|
||||||
|
|
||||||
private final ReentrantLock masterNodeInfoLock = new ReentrantLock(); |
|
||||||
|
|
||||||
/** |
|
||||||
* worker group nodes, workerGroup -> ips, combining registryWorkerGroupNodes and dbWorkerGroupNodes |
|
||||||
*/ |
|
||||||
private final ConcurrentHashMap<String, Set<String>> workerGroupNodes = new ConcurrentHashMap<>(); |
|
||||||
|
|
||||||
private final Map<String, MasterHeartBeat> masterNodeInfo = new HashMap<>(); |
|
||||||
|
|
||||||
private final Map<String, WorkerHeartBeat> workerNodeInfo = new HashMap<>(); |
|
||||||
|
|
||||||
/** |
|
||||||
* executor service |
|
||||||
*/ |
|
||||||
private ScheduledExecutorService executorService; |
|
||||||
|
|
||||||
@Autowired |
|
||||||
private RegistryClient registryClient; |
|
||||||
|
|
||||||
@Autowired |
|
||||||
private WorkerGroupMapper workerGroupMapper; |
|
||||||
|
|
||||||
@Autowired |
|
||||||
private AlertDao alertDao; |
|
||||||
|
|
||||||
@Autowired |
|
||||||
private MasterConfig masterConfig; |
|
||||||
|
|
||||||
@Autowired |
|
||||||
private ListenerEventAlertManager listenerEventAlertManager; |
|
||||||
|
|
||||||
private final List<WorkerInfoChangeListener> workerInfoChangeListeners = new ArrayList<>(); |
|
||||||
|
|
||||||
private final List<MasterInfoChangeListener> masterInfoChangeListeners = new ArrayList<>(); |
|
||||||
|
|
||||||
@Override |
|
||||||
public void afterPropertiesSet() { |
|
||||||
|
|
||||||
// load nodes from zookeeper
|
|
||||||
refreshNodesAndGroupMappings(); |
|
||||||
|
|
||||||
// init executor service
|
|
||||||
executorService = Executors |
|
||||||
.newSingleThreadScheduledExecutor(ThreadUtils.newDaemonThreadFactory("ServerNodeManagerExecutor")); |
|
||||||
executorService.scheduleWithFixedDelay( |
|
||||||
new WorkerNodeInfoAndGroupDbSyncTask(), |
|
||||||
0, |
|
||||||
masterConfig.getWorkerGroupRefreshInterval().getSeconds(), |
|
||||||
TimeUnit.SECONDS); |
|
||||||
|
|
||||||
registryClient.subscribe(RegistryNodeType.MASTER.getRegistryPath(), new MasterDataListener()); |
|
||||||
registryClient.subscribe(RegistryNodeType.WORKER.getRegistryPath(), new WorkerDataListener()); |
|
||||||
} |
|
||||||
|
|
||||||
class WorkerNodeInfoAndGroupDbSyncTask implements Runnable { |
|
||||||
|
|
||||||
@Override |
|
||||||
public void run() { |
|
||||||
try { |
|
||||||
// sync worker node info
|
|
||||||
refreshNodesAndGroupMappings(); |
|
||||||
} catch (Exception e) { |
|
||||||
log.error("WorkerNodeInfoAndGroupDbSyncTask error:", e); |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
/** |
|
||||||
* Refresh master/worker nodes and worker group mapping information |
|
||||||
*/ |
|
||||||
private void refreshNodesAndGroupMappings() { |
|
||||||
updateWorkerNodes(); |
|
||||||
updateWorkerGroupMappings(); |
|
||||||
notifyWorkerInfoChangeListeners(); |
|
||||||
|
|
||||||
updateMasterNodes(); |
|
||||||
notifyMasterInfoChangeListeners(); |
|
||||||
} |
|
||||||
|
|
||||||
/** |
|
||||||
* worker group node listener |
|
||||||
*/ |
|
||||||
class WorkerDataListener implements SubscribeListener { |
|
||||||
|
|
||||||
@Override |
|
||||||
public void notify(Event event) { |
|
||||||
final String path = event.path(); |
|
||||||
final Type type = event.type(); |
|
||||||
final String data = event.data(); |
|
||||||
if (registryClient.isWorkerPath(path)) { |
|
||||||
try { |
|
||||||
String[] parts = path.split("/"); |
|
||||||
final String workerAddress = parts[parts.length - 1]; |
|
||||||
|
|
||||||
// todo: update workerNodeInfo
|
|
||||||
log.debug("received subscribe event : {}", event); |
|
||||||
if (type == Type.ADD) { |
|
||||||
log.info("Worker: {} added, currentNode : {}", path, workerAddress); |
|
||||||
} else if (type == Type.REMOVE) { |
|
||||||
log.info("Worker node : {} down.", path); |
|
||||||
removeSingleWorkerNode(workerAddress); |
|
||||||
alertDao.sendServerStoppedAlert(1, path, "WORKER"); |
|
||||||
listenerEventAlertManager.publishServerDownListenerEvent(path, "WORKER"); |
|
||||||
} else if (type == Type.UPDATE) { |
|
||||||
syncSingleWorkerNodeInfo(workerAddress, JSONUtils.parseObject(data, WorkerHeartBeat.class)); |
|
||||||
} |
|
||||||
} catch (Exception ex) { |
|
||||||
log.error("WorkerGroupListener capture data change and get data failed", ex); |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
private void syncSingleWorkerNodeInfo(String workerAddress, WorkerHeartBeat info) { |
|
||||||
workerNodeInfoWriteLock.lock(); |
|
||||||
try { |
|
||||||
workerNodeInfo.put(workerAddress, info); |
|
||||||
} finally { |
|
||||||
workerNodeInfoWriteLock.unlock(); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
private void removeSingleWorkerNode(String workerAddress) { |
|
||||||
workerNodeInfoWriteLock.lock(); |
|
||||||
try { |
|
||||||
workerNodeInfo.remove(workerAddress); |
|
||||||
log.info("remove worker node {} from workerNodeInfo when worker server down", workerAddress); |
|
||||||
} finally { |
|
||||||
workerNodeInfoWriteLock.unlock(); |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
class MasterDataListener implements SubscribeListener { |
|
||||||
|
|
||||||
@Override |
|
||||||
public void notify(Event event) { |
|
||||||
final String path = event.path(); |
|
||||||
final Type type = event.type(); |
|
||||||
if (registryClient.isMasterPath(path)) { |
|
||||||
try { |
|
||||||
if (type.equals(Type.ADD)) { |
|
||||||
log.info("master node : {} added.", path); |
|
||||||
} else if (type.equals(Type.REMOVE)) { |
|
||||||
log.info("master node : {} down.", path); |
|
||||||
alertDao.sendServerStoppedAlert(1, path, "MASTER"); |
|
||||||
listenerEventAlertManager.publishServerDownListenerEvent(path, "MASTER"); |
|
||||||
} |
|
||||||
} catch (Exception ex) { |
|
||||||
log.error("MasterNodeListener capture data change and get data failed.", ex); |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
private void updateMasterNodes() { |
|
||||||
masterNodeInfoLock.lock(); |
|
||||||
try { |
|
||||||
masterNodeInfo.clear(); |
|
||||||
Map<String, String> masterNodeMaps = registryClient.getServerMaps(RegistryNodeType.MASTER); |
|
||||||
for (Map.Entry<String, String> entry : masterNodeMaps.entrySet()) { |
|
||||||
masterNodeInfo.put(entry.getKey(), JSONUtils.parseObject(entry.getValue(), MasterHeartBeat.class)); |
|
||||||
} |
|
||||||
} catch (Exception e) { |
|
||||||
log.error("update master nodes error", e); |
|
||||||
} finally { |
|
||||||
masterNodeInfoLock.unlock(); |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
||||||
|
|
||||||
private void updateWorkerNodes() { |
|
||||||
workerNodeInfoWriteLock.lock(); |
|
||||||
try { |
|
||||||
Map<String, String> workerNodeMaps = registryClient.getServerMaps(RegistryNodeType.WORKER); |
|
||||||
for (Map.Entry<String, String> entry : workerNodeMaps.entrySet()) { |
|
||||||
String nodeAddress = entry.getKey(); |
|
||||||
WorkerHeartBeat workerHeartBeat = JSONUtils.parseObject(entry.getValue(), WorkerHeartBeat.class); |
|
||||||
workerNodeInfo.put(nodeAddress, workerHeartBeat); |
|
||||||
} |
|
||||||
} finally { |
|
||||||
workerNodeInfoWriteLock.unlock(); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
private void updateWorkerGroupMappings() { |
|
||||||
List<WorkerGroup> workerGroups = workerGroupMapper.queryAllWorkerGroup(); |
|
||||||
Map<String, Set<String>> tmpWorkerGroupMappings = new HashMap<>(); |
|
||||||
try { |
|
||||||
workerNodeInfoReadLock.lock(); |
|
||||||
for (WorkerGroup workerGroup : workerGroups) { |
|
||||||
String workerGroupName = workerGroup.getName(); |
|
||||||
String[] workerAddresses = workerGroup.getAddrList().split(Constants.COMMA); |
|
||||||
if (ArrayUtils.isEmpty(workerAddresses)) { |
|
||||||
continue; |
|
||||||
} |
|
||||||
Set<String> activeWorkerNodes = Arrays.stream(workerAddresses) |
|
||||||
.filter(workerNodeInfo::containsKey).collect(Collectors.toSet()); |
|
||||||
tmpWorkerGroupMappings.put(workerGroupName, activeWorkerNodes); |
|
||||||
} |
|
||||||
if (!tmpWorkerGroupMappings.containsKey(WorkerGroupUtils.getDefaultWorkerGroup())) { |
|
||||||
tmpWorkerGroupMappings.put(WorkerGroupUtils.getDefaultWorkerGroup(), workerNodeInfo.keySet()); |
|
||||||
} |
|
||||||
} finally { |
|
||||||
workerNodeInfoReadLock.unlock(); |
|
||||||
} |
|
||||||
|
|
||||||
workerGroupWriteLock.lock(); |
|
||||||
try { |
|
||||||
workerGroupNodes.clear(); |
|
||||||
workerGroupNodes.putAll(tmpWorkerGroupMappings); |
|
||||||
} finally { |
|
||||||
workerGroupWriteLock.unlock(); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
public Map<String, Set<String>> getWorkerGroupNodes() { |
|
||||||
workerGroupReadLock.lock(); |
|
||||||
try { |
|
||||||
return Collections.unmodifiableMap(workerGroupNodes); |
|
||||||
} finally { |
|
||||||
workerGroupReadLock.unlock(); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
/** |
|
||||||
* get worker group nodes |
|
||||||
* |
|
||||||
* @param workerGroup workerGroup |
|
||||||
* @return worker nodes |
|
||||||
*/ |
|
||||||
public Set<String> getWorkerGroupNodes(String workerGroup) throws WorkerGroupNotFoundException { |
|
||||||
workerGroupReadLock.lock(); |
|
||||||
try { |
|
||||||
workerGroup = WorkerGroupUtils.getWorkerGroupOrDefault(workerGroup); |
|
||||||
Set<String> nodes = workerGroupNodes.get(workerGroup); |
|
||||||
if (nodes == null) { |
|
||||||
throw new WorkerGroupNotFoundException(String.format("WorkerGroup: %s is invalidated", workerGroup)); |
|
||||||
} |
|
||||||
if (CollectionUtils.isEmpty(nodes)) { |
|
||||||
return Collections.emptySet(); |
|
||||||
} |
|
||||||
return Collections.unmodifiableSet(nodes); |
|
||||||
} finally { |
|
||||||
workerGroupReadLock.unlock(); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
public Map<String, WorkerHeartBeat> getWorkerNodeInfo() { |
|
||||||
return Collections.unmodifiableMap(workerNodeInfo); |
|
||||||
} |
|
||||||
|
|
||||||
public Optional<WorkerHeartBeat> getWorkerNodeInfo(String workerServerAddress) { |
|
||||||
workerNodeInfoReadLock.lock(); |
|
||||||
try { |
|
||||||
return Optional.ofNullable(workerNodeInfo.getOrDefault(workerServerAddress, null)); |
|
||||||
} finally { |
|
||||||
workerNodeInfoReadLock.unlock(); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
public Map<String, MasterHeartBeat> getMasterNodeInfo() { |
|
||||||
return Collections.unmodifiableMap(masterNodeInfo); |
|
||||||
} |
|
||||||
|
|
||||||
/** |
|
||||||
* Add the resource change listener, when the resource changed, the listener will be notified. |
|
||||||
* |
|
||||||
* @param listener will be trigger, when the worker node info changed. |
|
||||||
*/ |
|
||||||
public synchronized void addWorkerInfoChangeListener(WorkerInfoChangeListener listener) { |
|
||||||
workerInfoChangeListeners.add(listener); |
|
||||||
} |
|
||||||
|
|
||||||
private void notifyWorkerInfoChangeListeners() { |
|
||||||
Map<String, Set<String>> workerGroupNodeMap = getWorkerGroupNodes(); |
|
||||||
Map<String, WorkerHeartBeat> workerNodeInfoMap = getWorkerNodeInfo(); |
|
||||||
for (WorkerInfoChangeListener listener : workerInfoChangeListeners) { |
|
||||||
listener.notify(workerGroupNodeMap, workerNodeInfoMap); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
public synchronized void addMasterInfoChangeListener(MasterInfoChangeListener listener) { |
|
||||||
masterInfoChangeListeners.add(listener); |
|
||||||
} |
|
||||||
|
|
||||||
private void notifyMasterInfoChangeListeners() { |
|
||||||
Map<String, MasterHeartBeat> masterNodeInfoMap = getMasterNodeInfo(); |
|
||||||
for (MasterInfoChangeListener listener : masterInfoChangeListeners) { |
|
||||||
listener.notify(masterNodeInfoMap); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
@PreDestroy |
|
||||||
public void destroy() { |
|
||||||
executorService.shutdownNow(); |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -1,38 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.registry; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; |
|
||||||
|
|
||||||
import java.util.Map; |
|
||||||
import java.util.Set; |
|
||||||
|
|
||||||
/** |
|
||||||
* The listener used in {@link ServerNodeManager} to notify the change of worker info. |
|
||||||
*/ |
|
||||||
public interface WorkerInfoChangeListener { |
|
||||||
|
|
||||||
/** |
|
||||||
* Used to notify the change of worker info. |
|
||||||
* |
|
||||||
* @param workerGroups worker groups map, key is worker group name, value is worker address. |
|
||||||
* @param workerNodeInfo worker node info map, key is worker address, value is worker info. |
|
||||||
*/ |
|
||||||
void notify(Map<String, Set<String>> workerGroups, Map<String, WorkerHeartBeat> workerNodeInfo); |
|
||||||
|
|
||||||
} |
|
@ -1,102 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.runner; |
|
||||||
|
|
||||||
import static com.google.common.base.Preconditions.checkNotNull; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.common.enums.TaskEventType; |
|
||||||
import org.apache.dolphinscheduler.common.utils.DateUtils; |
|
||||||
import org.apache.dolphinscheduler.extract.base.utils.Host; |
|
||||||
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext; |
|
||||||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; |
|
||||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
|
||||||
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException; |
|
||||||
import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException; |
|
||||||
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent; |
|
||||||
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService; |
|
||||||
import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatcher; |
|
||||||
|
|
||||||
import java.util.Date; |
|
||||||
import java.util.Optional; |
|
||||||
|
|
||||||
import lombok.extern.slf4j.Slf4j; |
|
||||||
|
|
||||||
@Slf4j |
|
||||||
public abstract class BaseTaskDispatcher implements TaskDispatcher { |
|
||||||
|
|
||||||
protected final TaskEventService taskEventService; |
|
||||||
protected final MasterConfig masterConfig; |
|
||||||
|
|
||||||
protected BaseTaskDispatcher(TaskEventService taskEventService, |
|
||||||
MasterConfig masterConfig) { |
|
||||||
this.taskEventService = checkNotNull(taskEventService); |
|
||||||
this.masterConfig = checkNotNull(masterConfig); |
|
||||||
} |
|
||||||
|
|
||||||
@Override |
|
||||||
public void dispatchTask(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException { |
|
||||||
Host taskInstanceDispatchHost; |
|
||||||
try { |
|
||||||
taskInstanceDispatchHost = getTaskInstanceDispatchHost(taskExecuteRunnable) |
|
||||||
.orElseThrow(() -> new TaskDispatchException("Cannot find the host to execute task.")); |
|
||||||
} catch (WorkerGroupNotFoundException workerGroupNotFoundException) { |
|
||||||
// todo: this is a temporary solution, we should refactor the ServerNodeManager to make sure there won't
|
|
||||||
// throw WorkerGroupNotFoundException unless the worker group is not exist in database
|
|
||||||
throw new TaskDispatchException( |
|
||||||
"Dispatch task: " + taskExecuteRunnable.getTaskExecutionContext().getTaskName() + " failed", |
|
||||||
workerGroupNotFoundException); |
|
||||||
} |
|
||||||
taskExecuteRunnable.getTaskExecutionContext().setHost(taskInstanceDispatchHost.getAddress()); |
|
||||||
doDispatch(taskExecuteRunnable); |
|
||||||
taskExecuteRunnable.getTaskInstance().setHost(taskInstanceDispatchHost.getAddress()); |
|
||||||
log.info("Success dispatch task {} to {}.", taskExecuteRunnable.getTaskExecutionContext().getTaskName(), |
|
||||||
taskInstanceDispatchHost.getAddress()); |
|
||||||
addDispatchEvent(taskExecuteRunnable); |
|
||||||
} |
|
||||||
|
|
||||||
protected abstract void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException; |
|
||||||
|
|
||||||
protected abstract Optional<Host> getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecutionContext) throws TaskDispatchException, WorkerGroupNotFoundException; |
|
||||||
|
|
||||||
protected void addDispatchEvent(TaskExecuteRunnable taskExecuteRunnable) { |
|
||||||
TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); |
|
||||||
TaskEvent taskEvent = TaskEvent.newDispatchEvent( |
|
||||||
taskExecutionContext.getProcessInstanceId(), |
|
||||||
taskExecutionContext.getTaskInstanceId(), |
|
||||||
taskExecutionContext.getHost()); |
|
||||||
taskEventService.addEvent(taskEvent); |
|
||||||
} |
|
||||||
|
|
||||||
private void addDispatchFailedEvent(TaskExecuteRunnable taskExecuteRunnable) { |
|
||||||
TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext(); |
|
||||||
TaskEvent taskEvent = TaskEvent.builder() |
|
||||||
.processInstanceId(taskExecutionContext.getProcessInstanceId()) |
|
||||||
.taskInstanceId(taskExecutionContext.getTaskInstanceId()) |
|
||||||
.state(TaskExecutionStatus.FAILURE) |
|
||||||
.logPath(taskExecutionContext.getLogPath()) |
|
||||||
.executePath(taskExecutionContext.getExecutePath()) |
|
||||||
.appIds(taskExecutionContext.getAppIds()) |
|
||||||
.processId(taskExecutionContext.getProcessId()) |
|
||||||
.varPool(taskExecutionContext.getVarPool()) |
|
||||||
.startTime(DateUtils.timeStampToDate(taskExecutionContext.getStartTime())) |
|
||||||
.endTime(new Date()) |
|
||||||
.event(TaskEventType.RESULT) |
|
||||||
.build(); |
|
||||||
taskEventService.addEvent(taskEvent); |
|
||||||
} |
|
||||||
} |
|
@ -0,0 +1,56 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.runner.dispatcher; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.extract.base.utils.Host; |
||||||
|
import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException; |
||||||
|
import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException; |
||||||
|
import org.apache.dolphinscheduler.server.master.runner.TaskExecuteRunnable; |
||||||
|
|
||||||
|
import java.util.Optional; |
||||||
|
|
||||||
|
import lombok.extern.slf4j.Slf4j; |
||||||
|
|
||||||
|
@Slf4j |
||||||
|
public abstract class BaseTaskDispatcher implements TaskDispatcher { |
||||||
|
|
||||||
|
@Override |
||||||
|
public void dispatchTask(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException { |
||||||
|
String taskName = taskExecuteRunnable.getTaskExecutionContext().getTaskName(); |
||||||
|
String taskInstanceDispatchAddress; |
||||||
|
try { |
||||||
|
taskInstanceDispatchAddress = getTaskInstanceDispatchHost(taskExecuteRunnable) |
||||||
|
.map(Host::getAddress) |
||||||
|
.orElseThrow(() -> new TaskDispatchException("Cannot find the host to execute task: " + taskName)); |
||||||
|
} catch (WorkerGroupNotFoundException workerGroupNotFoundException) { |
||||||
|
// todo: this is a temporary solution, we should refactor the ServerNodeManager to make sure there won't
|
||||||
|
// throw WorkerGroupNotFoundException unless the worker group is not exist in database
|
||||||
|
throw new TaskDispatchException("Dispatch task: " + taskName + " failed", workerGroupNotFoundException); |
||||||
|
} |
||||||
|
taskExecuteRunnable.getTaskExecutionContext().setHost(taskInstanceDispatchAddress); |
||||||
|
// todo: add dispatch address here to avoid set host in TaskExecuteRunnable before
|
||||||
|
doDispatch(taskExecuteRunnable); |
||||||
|
taskExecuteRunnable.getTaskInstance().setHost(taskInstanceDispatchAddress); |
||||||
|
log.info("Success dispatch task {} to {}.", taskName, taskInstanceDispatchAddress); |
||||||
|
} |
||||||
|
|
||||||
|
protected abstract void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException; |
||||||
|
|
||||||
|
protected abstract Optional<Host> getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecutionContext) throws TaskDispatchException, WorkerGroupNotFoundException; |
||||||
|
|
||||||
|
} |
@ -0,0 +1,106 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import static com.google.common.truth.Truth.assertThat; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
||||||
|
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean; |
||||||
|
|
||||||
|
import org.junit.jupiter.api.Test; |
||||||
|
|
||||||
|
class MasterClustersTest { |
||||||
|
|
||||||
|
@Test |
||||||
|
void getServers() { |
||||||
|
MasterClusters masterClusters = new MasterClusters(); |
||||||
|
MasterServerMetadata normalMasterServerMetadata = getNormalMasterServerMetadata(); |
||||||
|
MasterServerMetadata busyMasterServerMetadata = getBusyMasterServerMetadata(); |
||||||
|
masterClusters.onServerAdded(normalMasterServerMetadata); |
||||||
|
masterClusters.onServerAdded(busyMasterServerMetadata); |
||||||
|
assertThat(masterClusters.getServers()).containsExactly(normalMasterServerMetadata, busyMasterServerMetadata); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getNormalServers() { |
||||||
|
MasterClusters masterClusters = new MasterClusters(); |
||||||
|
MasterServerMetadata normalMasterServerMetadata = getNormalMasterServerMetadata(); |
||||||
|
MasterServerMetadata busyMasterServerMetadata = getBusyMasterServerMetadata(); |
||||||
|
masterClusters.onServerAdded(normalMasterServerMetadata); |
||||||
|
masterClusters.onServerAdded(busyMasterServerMetadata); |
||||||
|
assertThat(masterClusters.getNormalServers()).containsExactly(normalMasterServerMetadata); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void registerListener() { |
||||||
|
MasterClusters masterClusters = new MasterClusters(); |
||||||
|
AtomicBoolean addServerFlag = new AtomicBoolean(false); |
||||||
|
masterClusters.registerListener( |
||||||
|
(IClusters.ServerAddedListener<MasterServerMetadata>) server -> addServerFlag.set(true)); |
||||||
|
masterClusters.onServerAdded(getNormalMasterServerMetadata()); |
||||||
|
assertThat(addServerFlag.get()).isTrue(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void onServerAdded() { |
||||||
|
MasterClusters masterClusters = new MasterClusters(); |
||||||
|
masterClusters.onServerAdded(getNormalMasterServerMetadata()); |
||||||
|
assertThat(masterClusters.getServers()).containsExactly(getNormalMasterServerMetadata()); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void onServerRemove() { |
||||||
|
MasterClusters masterClusters = new MasterClusters(); |
||||||
|
masterClusters.onServerAdded(getNormalMasterServerMetadata()); |
||||||
|
masterClusters.onServerRemove(getNormalMasterServerMetadata()); |
||||||
|
assertThat(masterClusters.getServers()).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void onServerUpdate() { |
||||||
|
MasterClusters masterClusters = new MasterClusters(); |
||||||
|
masterClusters.onServerAdded(getNormalMasterServerMetadata()); |
||||||
|
MasterServerMetadata updatedMasterServerMetadata = MasterServerMetadata.builder() |
||||||
|
.address("127.0.0.1:5679") |
||||||
|
.cpuUsage(0.2) |
||||||
|
.memoryUsage(0.4) |
||||||
|
.serverStatus(ServerStatus.BUSY) |
||||||
|
.build(); |
||||||
|
masterClusters.onServerUpdate(updatedMasterServerMetadata); |
||||||
|
assertThat(masterClusters.getServers()).containsExactly(updatedMasterServerMetadata); |
||||||
|
} |
||||||
|
|
||||||
|
private MasterServerMetadata getNormalMasterServerMetadata() { |
||||||
|
return MasterServerMetadata.builder() |
||||||
|
.address("127.0.0.1:5679") |
||||||
|
.cpuUsage(0.2) |
||||||
|
.memoryUsage(0.4) |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.build(); |
||||||
|
} |
||||||
|
|
||||||
|
private MasterServerMetadata getBusyMasterServerMetadata() { |
||||||
|
return MasterServerMetadata.builder() |
||||||
|
.address("127.0.0.2:1235") |
||||||
|
.cpuUsage(0.8) |
||||||
|
.memoryUsage(0.8) |
||||||
|
.serverStatus(ServerStatus.BUSY) |
||||||
|
.build(); |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,50 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
||||||
|
import org.apache.dolphinscheduler.common.model.MasterHeartBeat; |
||||||
|
|
||||||
|
import org.junit.jupiter.api.Test; |
||||||
|
|
||||||
|
import com.google.common.truth.Truth; |
||||||
|
|
||||||
|
class MasterServerMetadataTest { |
||||||
|
|
||||||
|
@Test |
||||||
|
void parseFromHeartBeat() { |
||||||
|
MasterHeartBeat masterHeartBeat = MasterHeartBeat.builder() |
||||||
|
.processId(30001) |
||||||
|
.startupTime(System.currentTimeMillis()) |
||||||
|
.reportTime(System.currentTimeMillis()) |
||||||
|
.jvmCpuUsage(0.1) |
||||||
|
.cpuUsage(0.2) |
||||||
|
.jvmMemoryUsage(0.3) |
||||||
|
.memoryUsage(0.4) |
||||||
|
.diskUsage(0.5) |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.host("localhost") |
||||||
|
.port(12345) |
||||||
|
.build(); |
||||||
|
MasterServerMetadata masterServer = MasterServerMetadata.parseFromHeartBeat(masterHeartBeat); |
||||||
|
Truth.assertThat(masterServer.getCpuUsage()).isEqualTo(0.2); |
||||||
|
Truth.assertThat(masterServer.getMemoryUsage()).isEqualTo(0.4); |
||||||
|
Truth.assertThat(masterServer.getServerStatus()).isEqualTo(ServerStatus.NORMAL); |
||||||
|
Truth.assertThat(masterServer.getAddress()).isEqualTo("localhost:12345"); |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,106 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import static com.google.common.truth.Truth.assertThat; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
||||||
|
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
||||||
|
|
||||||
|
import org.junit.jupiter.api.BeforeEach; |
||||||
|
import org.junit.jupiter.api.Test; |
||||||
|
|
||||||
|
class MasterSlotManagerTest { |
||||||
|
|
||||||
|
private MasterSlotManager masterSlotManager; |
||||||
|
|
||||||
|
private ClusterManager clusterManager; |
||||||
|
|
||||||
|
private MasterConfig masterConfig; |
||||||
|
|
||||||
|
@BeforeEach |
||||||
|
public void setUp() { |
||||||
|
clusterManager = new ClusterManager(); |
||||||
|
masterConfig = new MasterConfig(); |
||||||
|
masterConfig.setMasterAddress("127.0.0.1:5678"); |
||||||
|
masterSlotManager = new MasterSlotManager(clusterManager, masterConfig); |
||||||
|
MasterServerMetadata master1 = MasterServerMetadata.builder() |
||||||
|
.cpuUsage(0.2) |
||||||
|
.memoryUsage(0.4) |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.address(masterConfig.getMasterAddress()) |
||||||
|
.build(); |
||||||
|
MasterServerMetadata master2 = MasterServerMetadata.builder() |
||||||
|
.cpuUsage(0.2) |
||||||
|
.memoryUsage(0.4) |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.address("127.0.0.2:5679") |
||||||
|
.build(); |
||||||
|
MasterServerMetadata master3 = MasterServerMetadata.builder() |
||||||
|
.cpuUsage(0.2) |
||||||
|
.memoryUsage(0.4) |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.address("127.0.0.3:5679") |
||||||
|
.build(); |
||||||
|
MasterServerMetadata master4 = MasterServerMetadata.builder() |
||||||
|
.cpuUsage(0.2) |
||||||
|
.memoryUsage(0.4) |
||||||
|
.serverStatus(ServerStatus.BUSY) |
||||||
|
.address("127.0.0.4:5679") |
||||||
|
.build(); |
||||||
|
clusterManager.getMasterClusters().onServerAdded(master1); |
||||||
|
clusterManager.getMasterClusters().onServerAdded(master2); |
||||||
|
clusterManager.getMasterClusters().onServerAdded(master3); |
||||||
|
clusterManager.getMasterClusters().onServerAdded(master4); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getCurrentMasterSlot() { |
||||||
|
assertThat(masterSlotManager.getCurrentMasterSlot()).isAtLeast(0); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void getTotalMasterSlots() { |
||||||
|
assertThat(masterSlotManager.getTotalMasterSlots()).isEqualTo(3); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void checkSlotValid() { |
||||||
|
assertThat(masterSlotManager.checkSlotValid()).isTrue(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void doReBalance() { |
||||||
|
MasterServerMetadata master2 = MasterServerMetadata.builder() |
||||||
|
.cpuUsage(0.2) |
||||||
|
.memoryUsage(0.4) |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.address("127.0.0.3:5679") |
||||||
|
.build(); |
||||||
|
MasterServerMetadata master3 = MasterServerMetadata.builder() |
||||||
|
.cpuUsage(0.2) |
||||||
|
.memoryUsage(0.4) |
||||||
|
.serverStatus(ServerStatus.BUSY) |
||||||
|
.address("127.0.0.4:5679") |
||||||
|
.build(); |
||||||
|
clusterManager.getMasterClusters().onServerRemove(master2); |
||||||
|
clusterManager.getMasterClusters().onServerRemove(master3); |
||||||
|
// After doReBalance, the total master slots should be 2
|
||||||
|
assertThat(masterSlotManager.getTotalMasterSlots()).isEqualTo(2); |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,167 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import static com.google.common.truth.Truth.assertThat; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
||||||
|
import org.apache.dolphinscheduler.dao.entity.WorkerGroup; |
||||||
|
|
||||||
|
import org.junit.jupiter.api.Test; |
||||||
|
|
||||||
|
import com.google.common.collect.Lists; |
||||||
|
import com.google.common.truth.Truth; |
||||||
|
|
||||||
|
class WorkerClustersTest { |
||||||
|
|
||||||
|
@Test |
||||||
|
void testOnWorkerGroupDelete() { |
||||||
|
WorkerClusters workerClusters = new WorkerClusters(); |
||||||
|
WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata(); |
||||||
|
workerClusters.onServerAdded(normalWorkerServerMetadata); |
||||||
|
|
||||||
|
WorkerGroup workerGroup = WorkerGroup.builder() |
||||||
|
.name("flinkCluster") |
||||||
|
.addrList(normalWorkerServerMetadata.getAddress()) |
||||||
|
.build(); |
||||||
|
workerClusters.onWorkerGroupAdd(Lists.newArrayList(workerGroup)); |
||||||
|
assertThat(workerClusters.getWorkerServerAddressByGroup("flinkCluster")) |
||||||
|
.containsExactly(normalWorkerServerMetadata.getAddress()); |
||||||
|
|
||||||
|
workerClusters.onWorkerGroupDelete(Lists.newArrayList(workerGroup)); |
||||||
|
Truth.assertThat(workerClusters.containsWorkerGroup("flinkCluster")).isFalse(); |
||||||
|
assertThat(workerClusters.getWorkerServerAddressByGroup("flinkCluster")).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void testOnWorkerGroupAdd() { |
||||||
|
WorkerClusters workerClusters = new WorkerClusters(); |
||||||
|
WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata(); |
||||||
|
workerClusters.onServerAdded(normalWorkerServerMetadata); |
||||||
|
|
||||||
|
WorkerGroup workerGroup = WorkerGroup.builder() |
||||||
|
.name("flinkCluster") |
||||||
|
.addrList(normalWorkerServerMetadata.getAddress()) |
||||||
|
.build(); |
||||||
|
workerClusters.onWorkerGroupAdd(Lists.newArrayList(workerGroup)); |
||||||
|
assertThat(workerClusters.getWorkerServerAddressByGroup("flinkCluster")) |
||||||
|
.containsExactly(normalWorkerServerMetadata.getAddress()); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void testOnWorkerGroupChange() { |
||||||
|
WorkerClusters workerClusters = new WorkerClusters(); |
||||||
|
WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata(); |
||||||
|
workerClusters.onServerAdded(normalWorkerServerMetadata); |
||||||
|
|
||||||
|
WorkerGroup workerGroup = WorkerGroup.builder() |
||||||
|
.name("flinkCluster") |
||||||
|
.addrList(normalWorkerServerMetadata.getAddress()) |
||||||
|
.build(); |
||||||
|
workerClusters.onWorkerGroupAdd(Lists.newArrayList(workerGroup)); |
||||||
|
assertThat(workerClusters.getWorkerServerAddressByGroup("flinkCluster")) |
||||||
|
.containsExactly(normalWorkerServerMetadata.getAddress()); |
||||||
|
|
||||||
|
WorkerGroup updatedWorkerGroup = WorkerGroup.builder() |
||||||
|
.name("flinkCluster") |
||||||
|
.addrList("") |
||||||
|
.build(); |
||||||
|
workerClusters.onWorkerGroupChange(Lists.newArrayList(updatedWorkerGroup)); |
||||||
|
assertThat(workerClusters.getWorkerServerAddressByGroup("flinkCluster")).isEmpty(); |
||||||
|
assertThat(workerClusters.containsWorkerGroup("flinkCluster")).isTrue(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void testOnServerAdded() { |
||||||
|
WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata(); |
||||||
|
WorkerServerMetadata busyWorkerServerMetadata = getBusyWorkerServerMetadata(); |
||||||
|
|
||||||
|
WorkerClusters workerClusters = new WorkerClusters(); |
||||||
|
workerClusters.onServerAdded(normalWorkerServerMetadata); |
||||||
|
workerClusters.onServerAdded(busyWorkerServerMetadata); |
||||||
|
assertThat(workerClusters.getWorkerServerAddressByGroup("default")) |
||||||
|
.containsExactly(normalWorkerServerMetadata.getAddress(), busyWorkerServerMetadata.getAddress()); |
||||||
|
assertThat(workerClusters.getNormalWorkerServerAddressByGroup("default")) |
||||||
|
.containsExactly(normalWorkerServerMetadata.getAddress()); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void testOnServerRemove() { |
||||||
|
WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata(); |
||||||
|
WorkerServerMetadata busyWorkerServerMetadata = getBusyWorkerServerMetadata(); |
||||||
|
|
||||||
|
WorkerClusters workerClusters = new WorkerClusters(); |
||||||
|
workerClusters.onServerAdded(normalWorkerServerMetadata); |
||||||
|
workerClusters.onServerAdded(busyWorkerServerMetadata); |
||||||
|
workerClusters.onServerRemove(busyWorkerServerMetadata); |
||||||
|
|
||||||
|
assertThat(workerClusters.getWorkerServerAddressByGroup("default")) |
||||||
|
.containsExactly(normalWorkerServerMetadata.getAddress()); |
||||||
|
assertThat(workerClusters.getNormalWorkerServerAddressByGroup("default")) |
||||||
|
.containsExactly(normalWorkerServerMetadata.getAddress()); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void testOnServerUpdate() { |
||||||
|
|
||||||
|
WorkerServerMetadata normalWorkerServerMetadata = getNormalWorkerServerMetadata(); |
||||||
|
WorkerServerMetadata busyWorkerServerMetadata = getBusyWorkerServerMetadata(); |
||||||
|
|
||||||
|
WorkerClusters workerClusters = new WorkerClusters(); |
||||||
|
workerClusters.onServerAdded(normalWorkerServerMetadata); |
||||||
|
workerClusters.onServerAdded(busyWorkerServerMetadata); |
||||||
|
|
||||||
|
WorkerServerMetadata workerServerMetadata = WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.2:1235") |
||||||
|
.cpuUsage(0.3) |
||||||
|
.memoryUsage(0.3) |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.taskThreadPoolUsage(0.3) |
||||||
|
.workerWeight(2) |
||||||
|
.build(); |
||||||
|
|
||||||
|
workerClusters.onServerUpdate(workerServerMetadata); |
||||||
|
|
||||||
|
assertThat(workerClusters.getWorkerServerAddressByGroup("default")) |
||||||
|
.containsExactly(normalWorkerServerMetadata.getAddress(), workerServerMetadata.getAddress()); |
||||||
|
assertThat(workerClusters.getNormalWorkerServerAddressByGroup("default")) |
||||||
|
.containsExactly(normalWorkerServerMetadata.getAddress(), workerServerMetadata.getAddress()); |
||||||
|
assertThat(workerClusters.getServers()).containsExactly(normalWorkerServerMetadata, workerServerMetadata); |
||||||
|
} |
||||||
|
|
||||||
|
private WorkerServerMetadata getNormalWorkerServerMetadata() { |
||||||
|
return WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.1:1235") |
||||||
|
.cpuUsage(0.2) |
||||||
|
.memoryUsage(0.4) |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.taskThreadPoolUsage(0.6) |
||||||
|
.workerWeight(2) |
||||||
|
.build(); |
||||||
|
} |
||||||
|
|
||||||
|
private WorkerServerMetadata getBusyWorkerServerMetadata() { |
||||||
|
return WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.2:1235") |
||||||
|
.cpuUsage(0.8) |
||||||
|
.memoryUsage(0.8) |
||||||
|
.serverStatus(ServerStatus.BUSY) |
||||||
|
.taskThreadPoolUsage(0.9) |
||||||
|
.workerWeight(2) |
||||||
|
.build(); |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,158 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import static com.google.common.truth.Truth.assertThat; |
||||||
|
import static org.mockito.Mockito.when; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.dao.entity.WorkerGroup; |
||||||
|
import org.apache.dolphinscheduler.dao.repository.WorkerGroupDao; |
||||||
|
|
||||||
|
import java.util.List; |
||||||
|
import java.util.concurrent.atomic.AtomicBoolean; |
||||||
|
|
||||||
|
import org.junit.jupiter.api.Test; |
||||||
|
import org.mockito.Mockito; |
||||||
|
|
||||||
|
import com.google.common.collect.Lists; |
||||||
|
|
||||||
|
class WorkerGroupChangeNotifierTest { |
||||||
|
|
||||||
|
@Test |
||||||
|
void detectWorkerGroupChanges_addedWorkerGroup() { |
||||||
|
WorkerGroupDao workerGroupDao = Mockito.mock(WorkerGroupDao.class); |
||||||
|
WorkerGroupChangeNotifier workerGroupChangeNotifier = new WorkerGroupChangeNotifier(workerGroupDao); |
||||||
|
|
||||||
|
WorkerGroup workerGroup1 = WorkerGroup.builder() |
||||||
|
.name("workerGroup1") |
||||||
|
.addrList("") |
||||||
|
.build(); |
||||||
|
when(workerGroupDao.queryAll()).thenReturn(Lists.newArrayList(workerGroup1)); |
||||||
|
|
||||||
|
AtomicBoolean workerGroupAdded = new AtomicBoolean(false); |
||||||
|
AtomicBoolean workerGroupDeleted = new AtomicBoolean(false); |
||||||
|
AtomicBoolean workerGroupChanged = new AtomicBoolean(false); |
||||||
|
workerGroupChangeNotifier.subscribeWorkerGroupsChange(new WorkerGroupChangeNotifier.WorkerGroupListener() { |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onWorkerGroupDelete(List<WorkerGroup> workerGroups) { |
||||||
|
workerGroupDeleted.set(true); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onWorkerGroupAdd(List<WorkerGroup> workerGroups) { |
||||||
|
workerGroupAdded.set(true); |
||||||
|
assertThat(workerGroups).containsExactly(workerGroup1); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onWorkerGroupChange(List<WorkerGroup> workerGroups) { |
||||||
|
workerGroupChanged.set(true); |
||||||
|
} |
||||||
|
}); |
||||||
|
workerGroupChangeNotifier.detectWorkerGroupChanges(); |
||||||
|
assertThat(workerGroupAdded.get()).isTrue(); |
||||||
|
assertThat(workerGroupChanged.get()).isFalse(); |
||||||
|
assertThat(workerGroupDeleted.get()).isFalse(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void detectWorkerGroupChanges_deleteWorkerGroup() { |
||||||
|
WorkerGroupDao workerGroupDao = Mockito.mock(WorkerGroupDao.class); |
||||||
|
WorkerGroupChangeNotifier workerGroupChangeNotifier = new WorkerGroupChangeNotifier(workerGroupDao); |
||||||
|
|
||||||
|
WorkerGroup workerGroup1 = WorkerGroup.builder() |
||||||
|
.name("workerGroup1") |
||||||
|
.addrList("") |
||||||
|
.build(); |
||||||
|
when(workerGroupDao.queryAll()).thenReturn(Lists.newArrayList(workerGroup1)); |
||||||
|
workerGroupChangeNotifier.detectWorkerGroupChanges(); |
||||||
|
|
||||||
|
when(workerGroupDao.queryAll()).thenReturn(Lists.newArrayList()); |
||||||
|
AtomicBoolean workerGroupAdded = new AtomicBoolean(false); |
||||||
|
AtomicBoolean workerGroupDeleted = new AtomicBoolean(false); |
||||||
|
AtomicBoolean workerGroupChanged = new AtomicBoolean(false); |
||||||
|
workerGroupChangeNotifier.subscribeWorkerGroupsChange(new WorkerGroupChangeNotifier.WorkerGroupListener() { |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onWorkerGroupDelete(List<WorkerGroup> workerGroups) { |
||||||
|
workerGroupDeleted.set(true); |
||||||
|
assertThat(workerGroups).containsExactly(workerGroup1); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onWorkerGroupAdd(List<WorkerGroup> workerGroups) { |
||||||
|
workerGroupAdded.set(true); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onWorkerGroupChange(List<WorkerGroup> workerGroups) { |
||||||
|
workerGroupChanged.set(true); |
||||||
|
} |
||||||
|
}); |
||||||
|
workerGroupChangeNotifier.detectWorkerGroupChanges(); |
||||||
|
assertThat(workerGroupAdded.get()).isFalse(); |
||||||
|
assertThat(workerGroupChanged.get()).isFalse(); |
||||||
|
assertThat(workerGroupDeleted.get()).isTrue(); |
||||||
|
assertThat(workerGroupChangeNotifier.getWorkerGroupMap()).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void detectWorkerGroupChanges_updateWorkerGroup() { |
||||||
|
WorkerGroupDao workerGroupDao = Mockito.mock(WorkerGroupDao.class); |
||||||
|
WorkerGroupChangeNotifier workerGroupChangeNotifier = new WorkerGroupChangeNotifier(workerGroupDao); |
||||||
|
|
||||||
|
WorkerGroup workerGroup1 = WorkerGroup.builder() |
||||||
|
.name("workerGroup1") |
||||||
|
.addrList("") |
||||||
|
.build(); |
||||||
|
when(workerGroupDao.queryAll()).thenReturn(Lists.newArrayList(workerGroup1)); |
||||||
|
workerGroupChangeNotifier.detectWorkerGroupChanges(); |
||||||
|
|
||||||
|
WorkerGroup updatedWorkerGroup1 = WorkerGroup.builder() |
||||||
|
.name("workerGroup1") |
||||||
|
.addrList("127.0.0.1:1235") |
||||||
|
.build(); |
||||||
|
when(workerGroupDao.queryAll()).thenReturn(Lists.newArrayList(updatedWorkerGroup1)); |
||||||
|
AtomicBoolean workerGroupAdded = new AtomicBoolean(false); |
||||||
|
AtomicBoolean workerGroupDeleted = new AtomicBoolean(false); |
||||||
|
AtomicBoolean workerGroupChanged = new AtomicBoolean(false); |
||||||
|
workerGroupChangeNotifier.subscribeWorkerGroupsChange(new WorkerGroupChangeNotifier.WorkerGroupListener() { |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onWorkerGroupDelete(List<WorkerGroup> workerGroups) { |
||||||
|
workerGroupDeleted.set(true); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onWorkerGroupAdd(List<WorkerGroup> workerGroups) { |
||||||
|
workerGroupAdded.set(true); |
||||||
|
} |
||||||
|
|
||||||
|
@Override |
||||||
|
public void onWorkerGroupChange(List<WorkerGroup> workerGroups) { |
||||||
|
workerGroupChanged.set(true); |
||||||
|
assertThat(workerGroups).containsExactly(updatedWorkerGroup1); |
||||||
|
} |
||||||
|
}); |
||||||
|
workerGroupChangeNotifier.detectWorkerGroupChanges(); |
||||||
|
assertThat(workerGroupAdded.get()).isFalse(); |
||||||
|
assertThat(workerGroupChanged.get()).isTrue(); |
||||||
|
assertThat(workerGroupDeleted.get()).isFalse(); |
||||||
|
assertThat(workerGroupChangeNotifier.getWorkerGroupMap()).containsEntry("workerGroup1", updatedWorkerGroup1); |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,53 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
package org.apache.dolphinscheduler.server.master.cluster; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
||||||
|
import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; |
||||||
|
|
||||||
|
import org.junit.jupiter.api.Test; |
||||||
|
|
||||||
|
import com.google.common.truth.Truth; |
||||||
|
|
||||||
|
class WorkerServerMetadataTest { |
||||||
|
|
||||||
|
@Test |
||||||
|
void parseFromHeartBeat() { |
||||||
|
WorkerHeartBeat workerHeartBeat = WorkerHeartBeat.builder() |
||||||
|
.processId(30001) |
||||||
|
.startupTime(System.currentTimeMillis()) |
||||||
|
.reportTime(System.currentTimeMillis()) |
||||||
|
.jvmCpuUsage(0.1) |
||||||
|
.cpuUsage(0.2) |
||||||
|
.jvmMemoryUsage(0.3) |
||||||
|
.memoryUsage(0.4) |
||||||
|
.diskUsage(0.5) |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.host("localhost") |
||||||
|
.port(12345) |
||||||
|
.workerHostWeight(2) |
||||||
|
.threadPoolUsage(0.6) |
||||||
|
.build(); |
||||||
|
WorkerServerMetadata workerServerMetadata = WorkerServerMetadata.parseFromHeartBeat(workerHeartBeat); |
||||||
|
Truth.assertThat(workerServerMetadata.getCpuUsage()).isEqualTo(0.2); |
||||||
|
Truth.assertThat(workerServerMetadata.getMemoryUsage()).isEqualTo(0.4); |
||||||
|
Truth.assertThat(workerServerMetadata.getServerStatus()).isEqualTo(ServerStatus.NORMAL); |
||||||
|
Truth.assertThat(workerServerMetadata.getAddress()).isEqualTo("localhost:12345"); |
||||||
|
Truth.assertThat(workerServerMetadata.getWorkerWeight()).isEqualTo(2); |
||||||
|
Truth.assertThat(workerServerMetadata.getTaskThreadPoolUsage()).isEqualTo(0.6); |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,82 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster.loadbalancer; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
||||||
|
import org.apache.dolphinscheduler.dao.entity.WorkerGroup; |
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters; |
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerServerMetadata; |
||||||
|
|
||||||
|
import java.util.List; |
||||||
|
|
||||||
|
import com.google.common.collect.Lists; |
||||||
|
|
||||||
|
public class BaseWorkerLoadBalancerTest { |
||||||
|
|
||||||
|
/** |
||||||
|
* "default" -> 127.0.0.1:1234, 127.0.0.2:1234, 127.0.0.3:1234, 127.0.0.4:1234 |
||||||
|
* "sparkCluster" -> 127.0.0.1:1234 |
||||||
|
* "flinkCluster" -> |
||||||
|
* "busyCluster" -> 127.0.0.4:1234 |
||||||
|
*/ |
||||||
|
protected WorkerClusters getDefaultWorkerClusters() { |
||||||
|
List<WorkerGroup> workerGroups = Lists.newArrayList( |
||||||
|
WorkerGroup.builder() |
||||||
|
.name("sparkCluster") |
||||||
|
.addrList("127.0.0.1:1234") |
||||||
|
.build(), |
||||||
|
WorkerGroup.builder() |
||||||
|
.name("flinkCluster") |
||||||
|
.build(), |
||||||
|
WorkerGroup.builder() |
||||||
|
.name("busyCluster") |
||||||
|
.build()); |
||||||
|
|
||||||
|
List<WorkerServerMetadata> workerServers = Lists.newArrayList( |
||||||
|
WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.1:1234") |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.cpuUsage(0.1) |
||||||
|
.memoryUsage(0.1) |
||||||
|
.build(), |
||||||
|
WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.2:1234") |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.cpuUsage(0.2) |
||||||
|
.memoryUsage(0.2) |
||||||
|
.build(), |
||||||
|
WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.3:1234") |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.cpuUsage(0.3) |
||||||
|
.memoryUsage(0.3) |
||||||
|
.build(), |
||||||
|
WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.4:1234") |
||||||
|
.serverStatus(ServerStatus.BUSY) |
||||||
|
.cpuUsage(0.8) |
||||||
|
.memoryUsage(0.8) |
||||||
|
.build()); |
||||||
|
|
||||||
|
WorkerClusters workerClusters = new WorkerClusters(); |
||||||
|
workerClusters.onWorkerGroupAdd(workerGroups); |
||||||
|
workerServers.forEach(workerClusters::onServerAdded); |
||||||
|
return workerClusters; |
||||||
|
} |
||||||
|
|
||||||
|
} |
@ -0,0 +1,119 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster.loadbalancer; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
||||||
|
import org.apache.dolphinscheduler.dao.entity.WorkerGroup; |
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters; |
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerServerMetadata; |
||||||
|
|
||||||
|
import java.util.HashMap; |
||||||
|
import java.util.List; |
||||||
|
import java.util.Map; |
||||||
|
import java.util.Optional; |
||||||
|
|
||||||
|
import org.junit.jupiter.api.Test; |
||||||
|
|
||||||
|
import com.google.common.collect.Lists; |
||||||
|
import com.google.common.truth.Truth; |
||||||
|
|
||||||
|
class DynamicWeightedRoundRobinWorkerLoadBalancerTest { |
||||||
|
|
||||||
|
@Test |
||||||
|
void select_defaultWorkerGroup() { |
||||||
|
DynamicWeightedRoundRobinWorkerLoadBalancer loadBalancer = createDynamicWeightedRoundRobinWorkerLoadBalancer(); |
||||||
|
|
||||||
|
// select 10000 times to check if the selected worker is in the worker list
|
||||||
|
Map<String, Integer> workerSelectedCount = new HashMap<>(); |
||||||
|
for (int i = 0; i < 10000; i++) { |
||||||
|
Optional<String> selectedWorker = loadBalancer.select("default"); |
||||||
|
Truth.assertThat(selectedWorker).isPresent(); |
||||||
|
workerSelectedCount.put(selectedWorker.get(), |
||||||
|
workerSelectedCount.getOrDefault(selectedWorker.get(), 0) + 1); |
||||||
|
} |
||||||
|
// Assert the three workers selected times are similar
|
||||||
|
Integer times1 = workerSelectedCount.get("127.0.0.1:1234"); |
||||||
|
Integer times2 = workerSelectedCount.get("127.0.0.2:1234"); |
||||||
|
Integer times3 = workerSelectedCount.get("127.0.0.3:1234"); |
||||||
|
Truth.assertThat(Math.abs(times1 - times2 * 2)).isWithin(10); |
||||||
|
Truth.assertThat(Math.abs(times1 - times3 * 3)).isWithin(10); |
||||||
|
Truth.assertThat(times1 + times2 + times3).isEqualTo(10000); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void select_emptyWorkerGroup() { |
||||||
|
DynamicWeightedRoundRobinWorkerLoadBalancer loadBalancer = createDynamicWeightedRoundRobinWorkerLoadBalancer(); |
||||||
|
Truth.assertThat(loadBalancer.select("busyCluster")).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void select_workerGroupNotExist() { |
||||||
|
DynamicWeightedRoundRobinWorkerLoadBalancer loadBalancer = createDynamicWeightedRoundRobinWorkerLoadBalancer(); |
||||||
|
Truth.assertThat(loadBalancer.select("notExistCluster")).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
private DynamicWeightedRoundRobinWorkerLoadBalancer createDynamicWeightedRoundRobinWorkerLoadBalancer() { |
||||||
|
List<WorkerGroup> workerGroups = Lists.newArrayList( |
||||||
|
WorkerGroup.builder() |
||||||
|
.name("sparkCluster") |
||||||
|
.addrList("127.0.0.1:1234") |
||||||
|
.build(), |
||||||
|
WorkerGroup.builder() |
||||||
|
.name("flinkCluster") |
||||||
|
.build(), |
||||||
|
WorkerGroup.builder() |
||||||
|
.name("busyCluster") |
||||||
|
.build()); |
||||||
|
List<WorkerServerMetadata> workerServers = Lists.newArrayList( |
||||||
|
WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.1:1234") |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.cpuUsage(0.1) |
||||||
|
.memoryUsage(0.1) |
||||||
|
.taskThreadPoolUsage(0.1) |
||||||
|
.build(), |
||||||
|
WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.2:1234") |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.cpuUsage(0.2) |
||||||
|
.memoryUsage(0.2) |
||||||
|
.taskThreadPoolUsage(0.2) |
||||||
|
.build(), |
||||||
|
WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.3:1234") |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.cpuUsage(0.3) |
||||||
|
.memoryUsage(0.3) |
||||||
|
.taskThreadPoolUsage(0.3) |
||||||
|
.build(), |
||||||
|
WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.4:1234") |
||||||
|
.serverStatus(ServerStatus.BUSY) |
||||||
|
.cpuUsage(0.8) |
||||||
|
.memoryUsage(0.8) |
||||||
|
.build()); |
||||||
|
WorkerClusters workerClusters = new WorkerClusters(); |
||||||
|
DynamicWeightedRoundRobinWorkerLoadBalancer dynamicWeightedRoundRobinWorkerLoadBalancer = |
||||||
|
new DynamicWeightedRoundRobinWorkerLoadBalancer(workerClusters, |
||||||
|
new WorkerLoadBalancerConfigurationProperties.DynamicWeightConfigProperties()); |
||||||
|
workerClusters.onWorkerGroupChange(workerGroups); |
||||||
|
workerServers.forEach(workerClusters::onServerAdded); |
||||||
|
|
||||||
|
return dynamicWeightedRoundRobinWorkerLoadBalancer; |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,118 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster.loadbalancer; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
||||||
|
import org.apache.dolphinscheduler.dao.entity.WorkerGroup; |
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters; |
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerServerMetadata; |
||||||
|
|
||||||
|
import java.util.HashMap; |
||||||
|
import java.util.List; |
||||||
|
import java.util.Map; |
||||||
|
import java.util.Optional; |
||||||
|
|
||||||
|
import org.junit.jupiter.api.Test; |
||||||
|
|
||||||
|
import com.google.common.collect.Lists; |
||||||
|
import com.google.common.truth.Truth; |
||||||
|
|
||||||
|
class FixedWeightedRoundRobinWorkerLoadBalancerTest { |
||||||
|
|
||||||
|
@Test |
||||||
|
void select_defaultWorkerGroup() { |
||||||
|
FixedWeightedRoundRobinWorkerLoadBalancer loadBalancer = createFixedWeightedRoundRobinWorkerLoadBalancer(); |
||||||
|
|
||||||
|
// select 10000 times to check if the selected worker is in the worker list
|
||||||
|
Map<String, Integer> workerSelectedCount = new HashMap<>(); |
||||||
|
for (int i = 0; i < 10000; i++) { |
||||||
|
Optional<String> selectedWorker = loadBalancer.select("default"); |
||||||
|
Truth.assertThat(selectedWorker).isPresent(); |
||||||
|
workerSelectedCount.put(selectedWorker.get(), |
||||||
|
workerSelectedCount.getOrDefault(selectedWorker.get(), 0) + 1); |
||||||
|
} |
||||||
|
// Assert the three workers selected times are similar
|
||||||
|
Integer times1 = workerSelectedCount.get("127.0.0.1:1234"); |
||||||
|
Integer times2 = workerSelectedCount.get("127.0.0.2:1234"); |
||||||
|
Integer times3 = workerSelectedCount.get("127.0.0.3:1234"); |
||||||
|
Truth.assertThat(Math.abs(times2 - times1 * 2)).isWithin(10); |
||||||
|
Truth.assertThat(Math.abs(times3 - times1 * 3)).isWithin(10); |
||||||
|
Truth.assertThat(times1 + times2 + times3).isEqualTo(10000); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void select_emptyWorkerGroup() { |
||||||
|
FixedWeightedRoundRobinWorkerLoadBalancer loadBalancer = createFixedWeightedRoundRobinWorkerLoadBalancer(); |
||||||
|
Truth.assertThat(loadBalancer.select("busyCluster")).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void select_workerGroupNotExist() { |
||||||
|
FixedWeightedRoundRobinWorkerLoadBalancer loadBalancer = createFixedWeightedRoundRobinWorkerLoadBalancer(); |
||||||
|
Truth.assertThat(loadBalancer.select("notExistCluster")).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
private FixedWeightedRoundRobinWorkerLoadBalancer createFixedWeightedRoundRobinWorkerLoadBalancer() { |
||||||
|
List<WorkerGroup> workerGroups = Lists.newArrayList( |
||||||
|
WorkerGroup.builder() |
||||||
|
.name("sparkCluster") |
||||||
|
.addrList("127.0.0.1:1234") |
||||||
|
.build(), |
||||||
|
WorkerGroup.builder() |
||||||
|
.name("flinkCluster") |
||||||
|
.build(), |
||||||
|
WorkerGroup.builder() |
||||||
|
.name("busyCluster") |
||||||
|
.build()); |
||||||
|
List<WorkerServerMetadata> workerServers = Lists.newArrayList( |
||||||
|
WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.1:1234") |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.cpuUsage(0.1) |
||||||
|
.memoryUsage(0.1) |
||||||
|
.workerWeight(1) |
||||||
|
.build(), |
||||||
|
WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.2:1234") |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.cpuUsage(0.2) |
||||||
|
.memoryUsage(0.2) |
||||||
|
.workerWeight(2) |
||||||
|
.build(), |
||||||
|
WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.3:1234") |
||||||
|
.serverStatus(ServerStatus.NORMAL) |
||||||
|
.cpuUsage(0.3) |
||||||
|
.memoryUsage(0.3) |
||||||
|
.workerWeight(3) |
||||||
|
.build(), |
||||||
|
WorkerServerMetadata.builder() |
||||||
|
.address("127.0.0.4:1234") |
||||||
|
.serverStatus(ServerStatus.BUSY) |
||||||
|
.cpuUsage(0.8) |
||||||
|
.memoryUsage(0.8) |
||||||
|
.build()); |
||||||
|
WorkerClusters workerClusters = new WorkerClusters(); |
||||||
|
FixedWeightedRoundRobinWorkerLoadBalancer loadBalancer = |
||||||
|
new FixedWeightedRoundRobinWorkerLoadBalancer(workerClusters); |
||||||
|
workerClusters.onWorkerGroupChange(workerGroups); |
||||||
|
workerServers.forEach(workerClusters::onServerAdded); |
||||||
|
|
||||||
|
return loadBalancer; |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,67 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster.loadbalancer; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters; |
||||||
|
|
||||||
|
import java.util.HashMap; |
||||||
|
import java.util.Map; |
||||||
|
import java.util.Optional; |
||||||
|
|
||||||
|
import org.junit.jupiter.api.Test; |
||||||
|
|
||||||
|
import com.google.common.truth.Truth; |
||||||
|
|
||||||
|
class RandomWorkerLoadBalancerTest extends BaseWorkerLoadBalancerTest { |
||||||
|
|
||||||
|
@Test |
||||||
|
void select_defaultWorkerGroup() { |
||||||
|
WorkerClusters defaultWorkerClusters = getDefaultWorkerClusters(); |
||||||
|
RandomWorkerLoadBalancer randomWorkerLoadBalancer = new RandomWorkerLoadBalancer(defaultWorkerClusters); |
||||||
|
// select 10000 times to check if the selected worker is in the worker list
|
||||||
|
Map<String, Integer> workerSelectedCount = new HashMap<>(); |
||||||
|
for (int i = 0; i < 10000; i++) { |
||||||
|
Optional<String> selectedWorker = randomWorkerLoadBalancer.select("default"); |
||||||
|
Truth.assertThat(selectedWorker).isPresent(); |
||||||
|
workerSelectedCount.put(selectedWorker.get(), |
||||||
|
workerSelectedCount.getOrDefault(selectedWorker.get(), 0) + 1); |
||||||
|
} |
||||||
|
// Assert the three workers selected times are similar
|
||||||
|
Integer times1 = workerSelectedCount.get("127.0.0.1:1234"); |
||||||
|
Integer times2 = workerSelectedCount.get("127.0.0.2:1234"); |
||||||
|
Integer times3 = workerSelectedCount.get("127.0.0.3:1234"); |
||||||
|
Truth.assertThat(Math.abs(times1 - times2)).isWithin(10); |
||||||
|
Truth.assertThat(Math.abs(times1 - times3)).isWithin(10); |
||||||
|
Truth.assertThat(Math.abs(times2 - times3)).isWithin(10); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void select_emptyWorkerGroup() { |
||||||
|
WorkerClusters defaultWorkerClusters = getDefaultWorkerClusters(); |
||||||
|
RandomWorkerLoadBalancer randomWorkerLoadBalancer = new RandomWorkerLoadBalancer(defaultWorkerClusters); |
||||||
|
Truth.assertThat(randomWorkerLoadBalancer.select("busyCluster")).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void select_workerGroupNotExist() { |
||||||
|
WorkerClusters defaultWorkerClusters = getDefaultWorkerClusters(); |
||||||
|
RandomWorkerLoadBalancer randomWorkerLoadBalancer = new RandomWorkerLoadBalancer(defaultWorkerClusters); |
||||||
|
Truth.assertThat(randomWorkerLoadBalancer.select("notExistCluster")).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
} |
@ -0,0 +1,69 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.cluster.loadbalancer; |
||||||
|
|
||||||
|
import org.apache.dolphinscheduler.server.master.cluster.WorkerClusters; |
||||||
|
|
||||||
|
import java.util.HashMap; |
||||||
|
import java.util.Map; |
||||||
|
import java.util.Optional; |
||||||
|
|
||||||
|
import org.junit.jupiter.api.Test; |
||||||
|
|
||||||
|
import com.google.common.truth.Truth; |
||||||
|
|
||||||
|
class RoundRobinWorkerLoadBalancerTest extends BaseWorkerLoadBalancerTest { |
||||||
|
|
||||||
|
@Test |
||||||
|
void select_defaultWorkerGroup() { |
||||||
|
WorkerClusters defaultWorkerClusters = getDefaultWorkerClusters(); |
||||||
|
RoundRobinWorkerLoadBalancer roundRobinWorkerLoadBalancer = |
||||||
|
new RoundRobinWorkerLoadBalancer(defaultWorkerClusters); |
||||||
|
// select 10000 times to check if the selected worker is in the worker list
|
||||||
|
Map<String, Integer> workerSelectedCount = new HashMap<>(); |
||||||
|
for (int i = 0; i < 10000; i++) { |
||||||
|
Optional<String> selectedWorker = roundRobinWorkerLoadBalancer.select("default"); |
||||||
|
Truth.assertThat(selectedWorker).isPresent(); |
||||||
|
workerSelectedCount.put(selectedWorker.get(), |
||||||
|
workerSelectedCount.getOrDefault(selectedWorker.get(), 0) + 1); |
||||||
|
} |
||||||
|
// Assert the three workers selected times are similar
|
||||||
|
Integer times1 = workerSelectedCount.get("127.0.0.1:1234"); |
||||||
|
Integer times2 = workerSelectedCount.get("127.0.0.2:1234"); |
||||||
|
Integer times3 = workerSelectedCount.get("127.0.0.3:1234"); |
||||||
|
Truth.assertThat(Math.abs(times1 - times2)).isWithin(10); |
||||||
|
Truth.assertThat(Math.abs(times1 - times3)).isWithin(10); |
||||||
|
Truth.assertThat(Math.abs(times2 - times3)).isWithin(10); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void select_emptyWorkerGroup() { |
||||||
|
WorkerClusters defaultWorkerClusters = getDefaultWorkerClusters(); |
||||||
|
RoundRobinWorkerLoadBalancer roundRobinWorkerLoadBalancer = |
||||||
|
new RoundRobinWorkerLoadBalancer(defaultWorkerClusters); |
||||||
|
Truth.assertThat(roundRobinWorkerLoadBalancer.select("busyCluster")).isEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
@Test |
||||||
|
void select_workerGroupNotExist() { |
||||||
|
WorkerClusters defaultWorkerClusters = getDefaultWorkerClusters(); |
||||||
|
RoundRobinWorkerLoadBalancer roundRobinWorkerLoadBalancer = |
||||||
|
new RoundRobinWorkerLoadBalancer(defaultWorkerClusters); |
||||||
|
Truth.assertThat(roundRobinWorkerLoadBalancer.select("notExistCluster")).isEmpty(); |
||||||
|
} |
||||||
|
} |
@ -1,43 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host.assign; |
|
||||||
|
|
||||||
import org.junit.jupiter.api.Assertions; |
|
||||||
import org.junit.jupiter.api.Test; |
|
||||||
|
|
||||||
public class HostWorkerTest { |
|
||||||
|
|
||||||
@Test |
|
||||||
public void testHostWorker1() { |
|
||||||
HostWorker hostWorker = new HostWorker("192.158.2.2", 11, 20, "default"); |
|
||||||
Assertions.assertEquals("192.158.2.2", hostWorker.getIp()); |
|
||||||
Assertions.assertEquals(11, hostWorker.getPort()); |
|
||||||
Assertions.assertEquals(20, hostWorker.getHostWeight()); |
|
||||||
Assertions.assertEquals("default", hostWorker.getWorkerGroup()); |
|
||||||
} |
|
||||||
|
|
||||||
@Test |
|
||||||
public void testHostWorker2() { |
|
||||||
HostWorker hostWorker = HostWorker.of("192.158.2.2:22", 80, "default"); |
|
||||||
Assertions.assertEquals("192.158.2.2", hostWorker.getIp()); |
|
||||||
Assertions.assertEquals(22, hostWorker.getPort()); |
|
||||||
Assertions.assertEquals(80, hostWorker.getHostWeight()); |
|
||||||
Assertions.assertEquals("default", hostWorker.getWorkerGroup()); |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -1,97 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host.assign; |
|
||||||
|
|
||||||
import java.util.ArrayList; |
|
||||||
import java.util.Collection; |
|
||||||
|
|
||||||
import org.junit.jupiter.api.Assertions; |
|
||||||
import org.junit.jupiter.api.Test; |
|
||||||
|
|
||||||
public class LowerWeightRoundRobinTest { |
|
||||||
|
|
||||||
@Test |
|
||||||
public void testSelect() { |
|
||||||
Collection<HostWeight> sources = new ArrayList<>(); |
|
||||||
sources.add(new HostWeight(HostWorker.of("192.158.2.1:11", 100, "default"), 0.06, 0.44, 0.384, 0.1, |
|
||||||
System.currentTimeMillis() - 60 * 8 * 1000)); |
|
||||||
sources.add(new HostWeight(HostWorker.of("192.158.2.2:22", 100, "default"), 0.06, 0.56, 0.324, 0.2, |
|
||||||
System.currentTimeMillis() - 60 * 5 * 1000)); |
|
||||||
sources.add(new HostWeight(HostWorker.of("192.158.2.3:33", 100, "default"), 0.06, 0.80, 0.315, 0.1, |
|
||||||
System.currentTimeMillis() - 60 * 2 * 1000)); |
|
||||||
|
|
||||||
LowerWeightRoundRobin roundRobin = new LowerWeightRoundRobin(); |
|
||||||
HostWeight result; |
|
||||||
result = roundRobin.select(sources); |
|
||||||
Assertions.assertEquals("192.158.2.1", result.getHost().getIp()); |
|
||||||
result = roundRobin.select(sources); |
|
||||||
Assertions.assertEquals("192.158.2.2", result.getHost().getIp()); |
|
||||||
result = roundRobin.select(sources); |
|
||||||
Assertions.assertEquals("192.158.2.1", result.getHost().getIp()); |
|
||||||
result = roundRobin.select(sources); |
|
||||||
Assertions.assertEquals("192.158.2.2", result.getHost().getIp()); |
|
||||||
result = roundRobin.select(sources); |
|
||||||
Assertions.assertEquals("192.158.2.1", result.getHost().getIp()); |
|
||||||
result = roundRobin.select(sources); |
|
||||||
Assertions.assertEquals("192.158.2.2", result.getHost().getIp()); |
|
||||||
} |
|
||||||
|
|
||||||
@Test |
|
||||||
public void testWarmUpSelect() { |
|
||||||
Collection<HostWeight> sources = new ArrayList<>(); |
|
||||||
sources.add(new HostWeight(HostWorker.of("192.158.2.1:11", 100, "default"), 0.06, 0.44, 0.384, 0, |
|
||||||
System.currentTimeMillis() - 60 * 8 * 1000)); |
|
||||||
sources.add(new HostWeight(HostWorker.of("192.158.2.2:22", 100, "default"), 0.06, 0.44, 0.384, 0, |
|
||||||
System.currentTimeMillis() - 60 * 5 * 1000)); |
|
||||||
sources.add(new HostWeight(HostWorker.of("192.158.2.3:33", 100, "default"), 0.06, 0.44, 0.384, 0, |
|
||||||
System.currentTimeMillis() - 60 * 3 * 1000)); |
|
||||||
sources.add(new HostWeight(HostWorker.of("192.158.2.4:33", 100, "default"), 0.06, 0.44, 0.384, 0, |
|
||||||
System.currentTimeMillis() - 60 * 11 * 1000)); |
|
||||||
|
|
||||||
LowerWeightRoundRobin roundRobin = new LowerWeightRoundRobin(); |
|
||||||
HostWeight result; |
|
||||||
result = roundRobin.select(sources); |
|
||||||
Assertions.assertEquals("192.158.2.4", result.getHost().getIp()); |
|
||||||
result = roundRobin.select(sources); |
|
||||||
Assertions.assertEquals("192.158.2.1", result.getHost().getIp()); |
|
||||||
result = roundRobin.select(sources); |
|
||||||
Assertions.assertEquals("192.158.2.2", result.getHost().getIp()); |
|
||||||
result = roundRobin.select(sources); |
|
||||||
Assertions.assertEquals("192.158.2.4", result.getHost().getIp()); |
|
||||||
result = roundRobin.select(sources); |
|
||||||
Assertions.assertEquals("192.158.2.1", result.getHost().getIp()); |
|
||||||
} |
|
||||||
|
|
||||||
@Test |
|
||||||
public void testDoSelect() { |
|
||||||
Collection<HostWeight> sources = new ArrayList<>(); |
|
||||||
LowerWeightRoundRobin roundRobin = new LowerWeightRoundRobin(); |
|
||||||
HostWeight result; |
|
||||||
result = roundRobin.doSelect(sources); |
|
||||||
Assertions.assertNull(result); |
|
||||||
|
|
||||||
sources.add(new HostWeight(HostWorker.of("192.158.2.1:11", 100, "default"), 0.06, 0.44, 0.314, 0.1, |
|
||||||
System.currentTimeMillis() - 60 * 8 * 1000)); |
|
||||||
sources.add(new HostWeight(HostWorker.of("192.158.2.2:22", 100, "default"), 0.06, 0.56, 0.324, 0.2, |
|
||||||
System.currentTimeMillis() - 60 * 5 * 1000)); |
|
||||||
sources.add(new HostWeight(HostWorker.of("192.158.2.3:33", 100, "default"), 0.06, 0.80, 0.315, 0.1, |
|
||||||
System.currentTimeMillis() - 60 * 2 * 1000)); |
|
||||||
result = roundRobin.doSelect(sources); |
|
||||||
Assertions.assertEquals("192.158.2.1", result.getHost().getIp()); |
|
||||||
} |
|
||||||
} |
|
@ -1,53 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host.assign; |
|
||||||
|
|
||||||
import java.util.Arrays; |
|
||||||
|
|
||||||
import org.junit.jupiter.api.Assertions; |
|
||||||
import org.junit.jupiter.api.Test; |
|
||||||
|
|
||||||
/** |
|
||||||
* random selector |
|
||||||
*/ |
|
||||||
public class RandomSelectorTest { |
|
||||||
|
|
||||||
@Test |
|
||||||
public void testSelectWithIllegalArgumentException() { |
|
||||||
RandomSelector selector = new RandomSelector(); |
|
||||||
Assertions.assertThrows(IllegalArgumentException.class, () -> { |
|
||||||
selector.select(null); |
|
||||||
}); |
|
||||||
} |
|
||||||
|
|
||||||
@Test |
|
||||||
public void testSelect1() { |
|
||||||
RandomSelector selector = new RandomSelector(); |
|
||||||
HostWorker result = selector.select(Arrays.asList(new HostWorker("192.168.1.1:11", 100, "default"), |
|
||||||
new HostWorker("192.168.1.2:22", 80, "default"))); |
|
||||||
Assertions.assertNotNull(result); |
|
||||||
} |
|
||||||
|
|
||||||
@Test |
|
||||||
public void testSelect() { |
|
||||||
RandomSelector selector = new RandomSelector(); |
|
||||||
HostWorker result = selector.select(Arrays.asList(new HostWorker("192.168.1.1", 11, 100, "default"), |
|
||||||
new HostWorker("192.168.1.2:", 22, 20, "default"))); |
|
||||||
Assertions.assertNotNull(result); |
|
||||||
} |
|
||||||
} |
|
@ -1,106 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.dispatch.host.assign; |
|
||||||
|
|
||||||
import java.util.Arrays; |
|
||||||
import java.util.List; |
|
||||||
|
|
||||||
import org.junit.jupiter.api.Assertions; |
|
||||||
import org.junit.jupiter.api.Test; |
|
||||||
|
|
||||||
/** |
|
||||||
* round robin selector |
|
||||||
*/ |
|
||||||
public class RoundRobinSelectorTest { |
|
||||||
|
|
||||||
@Test |
|
||||||
public void testSelectWithIllegalArgumentException() { |
|
||||||
RoundRobinSelector selector = new RoundRobinSelector(); |
|
||||||
Assertions.assertThrows(IllegalArgumentException.class, () -> { |
|
||||||
selector.select(null); |
|
||||||
}); |
|
||||||
} |
|
||||||
|
|
||||||
@Test |
|
||||||
public void testSelect1() { |
|
||||||
RoundRobinSelector selector = new RoundRobinSelector(); |
|
||||||
List<HostWorker> hostOneList = Arrays.asList( |
|
||||||
new HostWorker("192.168.1.1", 80, 20, "kris"), |
|
||||||
new HostWorker("192.168.1.2", 80, 10, "kris")); |
|
||||||
|
|
||||||
List<HostWorker> hostTwoList = Arrays.asList( |
|
||||||
new HostWorker("192.168.1.1", 80, 20, "kris"), |
|
||||||
new HostWorker("192.168.1.2", 80, 10, "kris"), |
|
||||||
new HostWorker("192.168.1.3", 80, 10, "kris")); |
|
||||||
|
|
||||||
HostWorker result; |
|
||||||
result = selector.select(hostOneList); |
|
||||||
Assertions.assertEquals("192.168.1.1", result.getIp()); |
|
||||||
|
|
||||||
result = selector.select(hostOneList); |
|
||||||
Assertions.assertEquals("192.168.1.2", result.getIp()); |
|
||||||
|
|
||||||
result = selector.select(hostOneList); |
|
||||||
Assertions.assertEquals("192.168.1.1", result.getIp()); |
|
||||||
|
|
||||||
result = selector.select(hostOneList); |
|
||||||
Assertions.assertEquals("192.168.1.1", result.getIp()); |
|
||||||
|
|
||||||
result = selector.select(hostOneList); |
|
||||||
Assertions.assertEquals("192.168.1.2", result.getIp()); |
|
||||||
|
|
||||||
// add new host
|
|
||||||
result = selector.select(hostTwoList); |
|
||||||
Assertions.assertEquals("192.168.1.1", result.getIp()); |
|
||||||
|
|
||||||
result = selector.select(hostTwoList); |
|
||||||
Assertions.assertEquals("192.168.1.3", result.getIp()); |
|
||||||
|
|
||||||
result = selector.select(hostTwoList); |
|
||||||
Assertions.assertEquals("192.168.1.1", result.getIp()); |
|
||||||
|
|
||||||
result = selector.select(hostTwoList); |
|
||||||
Assertions.assertEquals("192.168.1.2", result.getIp()); |
|
||||||
result = selector.select(hostTwoList); |
|
||||||
Assertions.assertEquals("192.168.1.1", result.getIp()); |
|
||||||
|
|
||||||
result = selector.select(hostTwoList); |
|
||||||
Assertions.assertEquals("192.168.1.3", result.getIp()); |
|
||||||
|
|
||||||
// remove host3
|
|
||||||
result = selector.select(hostOneList); |
|
||||||
Assertions.assertEquals("192.168.1.1", result.getIp()); |
|
||||||
|
|
||||||
result = selector.select(hostOneList); |
|
||||||
Assertions.assertEquals("192.168.1.2", result.getIp()); |
|
||||||
|
|
||||||
result = selector.select(hostOneList); |
|
||||||
Assertions.assertEquals("192.168.1.1", result.getIp()); |
|
||||||
} |
|
||||||
|
|
||||||
@Test |
|
||||||
public void testWeightRoundRobinSelector() { |
|
||||||
RoundRobinSelector selector = new RoundRobinSelector(); |
|
||||||
HostWorker result; |
|
||||||
result = selector.select( |
|
||||||
Arrays.asList(new HostWorker("192.168.1.1", 11, 20, "kris"), |
|
||||||
new HostWorker("192.168.1.2", 22, 80, "kris"))); |
|
||||||
Assertions.assertEquals("192.168.1.2", result.getIp()); |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -1,94 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.registry; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.common.enums.ServerStatus; |
|
||||||
import org.apache.dolphinscheduler.common.model.MasterHeartBeat; |
|
||||||
import org.apache.dolphinscheduler.server.master.config.MasterConfig; |
|
||||||
|
|
||||||
import java.util.HashMap; |
|
||||||
import java.util.Map; |
|
||||||
|
|
||||||
import org.junit.jupiter.api.Assertions; |
|
||||||
import org.junit.jupiter.api.Test; |
|
||||||
import org.junit.jupiter.api.extension.ExtendWith; |
|
||||||
import org.mockito.InjectMocks; |
|
||||||
import org.mockito.Mock; |
|
||||||
import org.mockito.Mockito; |
|
||||||
import org.mockito.junit.jupiter.MockitoExtension; |
|
||||||
|
|
||||||
@ExtendWith(MockitoExtension.class) |
|
||||||
public class MasterSlotManagerTest { |
|
||||||
|
|
||||||
@InjectMocks |
|
||||||
private MasterSlotManager masterSlotManager = Mockito.spy(new MasterSlotManager()); |
|
||||||
|
|
||||||
@Mock |
|
||||||
private MasterConfig masterConfig; |
|
||||||
|
|
||||||
@Test |
|
||||||
void testNormalMasterSlots() { |
|
||||||
// on normal Master side
|
|
||||||
Mockito.when(masterConfig.getMasterAddress()).thenReturn("127.0.0.1:7777"); |
|
||||||
|
|
||||||
sendHeartBeat(ServerStatus.BUSY, ServerStatus.NORMAL); |
|
||||||
Assertions.assertEquals(1, masterSlotManager.getMasterSize()); |
|
||||||
Assertions.assertEquals(0, masterSlotManager.getSlot()); |
|
||||||
|
|
||||||
sendHeartBeat(ServerStatus.NORMAL, ServerStatus.NORMAL); |
|
||||||
Assertions.assertEquals(2, masterSlotManager.getMasterSize()); |
|
||||||
Assertions.assertEquals(1, masterSlotManager.getSlot()); |
|
||||||
} |
|
||||||
|
|
||||||
@Test |
|
||||||
void testOverloadMasterSlots() { |
|
||||||
// on abnormal Master side
|
|
||||||
Mockito.when(masterConfig.getMasterAddress()).thenReturn("127.0.0.1:6666"); |
|
||||||
|
|
||||||
sendHeartBeat(ServerStatus.BUSY, ServerStatus.NORMAL); |
|
||||||
Assertions.assertEquals(0, masterSlotManager.getMasterSize()); |
|
||||||
Assertions.assertEquals(0, masterSlotManager.getSlot()); |
|
||||||
|
|
||||||
sendHeartBeat(ServerStatus.NORMAL, ServerStatus.NORMAL); |
|
||||||
Assertions.assertEquals(2, masterSlotManager.getMasterSize()); |
|
||||||
Assertions.assertEquals(0, masterSlotManager.getSlot()); |
|
||||||
} |
|
||||||
|
|
||||||
public void sendHeartBeat(ServerStatus serverStatus1, ServerStatus serverStatus2) { |
|
||||||
MasterSlotManager.SlotChangeListener slotChangeListener = masterSlotManager.new SlotChangeListener(); |
|
||||||
|
|
||||||
Map<String, MasterHeartBeat> masterNodeInfo = new HashMap<>(); |
|
||||||
// generate heartbeat
|
|
||||||
MasterHeartBeat masterHeartBeat1 = MasterHeartBeat.builder() |
|
||||||
.startupTime(System.currentTimeMillis()) |
|
||||||
.serverStatus(serverStatus1) |
|
||||||
.host("127.0.0.1") |
|
||||||
.port(6666) |
|
||||||
.build(); |
|
||||||
MasterHeartBeat masterHeartBeat2 = MasterHeartBeat.builder() |
|
||||||
.startupTime(System.currentTimeMillis()) |
|
||||||
.serverStatus(serverStatus2) |
|
||||||
.host("127.0.0.1") |
|
||||||
.port(7777) |
|
||||||
.build(); |
|
||||||
masterNodeInfo.put("127.0.0.1:6666", masterHeartBeat1); |
|
||||||
masterNodeInfo.put("127.0.0.1:7777", masterHeartBeat2); |
|
||||||
|
|
||||||
slotChangeListener.notify(masterNodeInfo); |
|
||||||
} |
|
||||||
} |
|
@ -1,93 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.server.master.registry; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.common.model.WorkerHeartBeat; |
|
||||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
|
||||||
import org.apache.dolphinscheduler.dao.AlertDao; |
|
||||||
import org.apache.dolphinscheduler.registry.api.Event; |
|
||||||
import org.apache.dolphinscheduler.registry.api.RegistryClient; |
|
||||||
import org.apache.dolphinscheduler.service.alert.ListenerEventAlertManager; |
|
||||||
|
|
||||||
import java.lang.reflect.InvocationTargetException; |
|
||||||
import java.lang.reflect.Method; |
|
||||||
import java.util.HashMap; |
|
||||||
import java.util.Map; |
|
||||||
|
|
||||||
import org.junit.jupiter.api.Assertions; |
|
||||||
import org.junit.jupiter.api.Test; |
|
||||||
import org.mockito.InjectMocks; |
|
||||||
import org.mockito.Mock; |
|
||||||
import org.mockito.Mockito; |
|
||||||
import org.mockito.MockitoAnnotations; |
|
||||||
|
|
||||||
public class ServerNodeManagerTest { |
|
||||||
|
|
||||||
@Mock |
|
||||||
RegistryClient registryClient; |
|
||||||
|
|
||||||
@Mock |
|
||||||
AlertDao alertDao; |
|
||||||
|
|
||||||
@Mock |
|
||||||
ListenerEventAlertManager listenerEventAlertManager; |
|
||||||
|
|
||||||
@InjectMocks |
|
||||||
ServerNodeManager serverNodeManager; |
|
||||||
|
|
||||||
@Test |
|
||||||
public void updateWorkerNodesTest() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { |
|
||||||
|
|
||||||
MockitoAnnotations.initMocks(this); |
|
||||||
HashMap<String, String> workerNodeMaps = new HashMap<>(); |
|
||||||
workerNodeMaps.put("worker-node-1", JSONUtils.toJsonString(new WorkerHeartBeat())); |
|
||||||
workerNodeMaps.put("worker-node-2", JSONUtils.toJsonString(new WorkerHeartBeat())); |
|
||||||
|
|
||||||
Mockito.when(registryClient.getServerMaps(Mockito.any())).thenReturn(workerNodeMaps); |
|
||||||
Mockito.when(registryClient.isWorkerPath(Mockito.anyString())).thenReturn(true); |
|
||||||
|
|
||||||
// two worker server running (worker-node-1, worker-node-2)
|
|
||||||
Method updateWorkerNodes = serverNodeManager.getClass().getDeclaredMethod("updateWorkerNodes"); |
|
||||||
updateWorkerNodes.setAccessible(true); |
|
||||||
updateWorkerNodes.invoke(serverNodeManager); |
|
||||||
|
|
||||||
Map<String, WorkerHeartBeat> workerNodeInfo = serverNodeManager.getWorkerNodeInfo(); |
|
||||||
Assertions.assertTrue(workerNodeInfo.containsKey("worker-node-1")); |
|
||||||
Assertions.assertTrue(workerNodeInfo.containsKey("worker-node-2")); |
|
||||||
|
|
||||||
// receive remove event when worker-node-1 server stop
|
|
||||||
ServerNodeManager.WorkerDataListener workerDataListener = serverNodeManager.new WorkerDataListener(); |
|
||||||
Event event = new Event("", "/nodes/worker/worker-node-1", "", Event.Type.REMOVE); |
|
||||||
workerDataListener.notify(event); |
|
||||||
|
|
||||||
// check worker-node-1 not exist in cache
|
|
||||||
workerNodeInfo = serverNodeManager.getWorkerNodeInfo(); |
|
||||||
Assertions.assertFalse(workerNodeInfo.containsKey("worker-node-1")); |
|
||||||
Assertions.assertTrue(workerNodeInfo.containsKey("worker-node-2")); |
|
||||||
|
|
||||||
// worker-node-1 restart, getServerMaps(RegistryNodeType.WORKER) method return two worker
|
|
||||||
updateWorkerNodes.invoke(serverNodeManager); |
|
||||||
|
|
||||||
// check cache
|
|
||||||
workerNodeInfo = serverNodeManager.getWorkerNodeInfo(); |
|
||||||
Assertions.assertTrue(workerNodeInfo.containsKey("worker-node-1")); |
|
||||||
Assertions.assertTrue(workerNodeInfo.containsKey("worker-node-2")); |
|
||||||
|
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -0,0 +1,59 @@ |
|||||||
|
/* |
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||||
|
* contributor license agreements. See the NOTICE file distributed with |
||||||
|
* this work for additional information regarding copyright ownership. |
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||||
|
* (the "License"); you may not use this file except in compliance with |
||||||
|
* the License. You may obtain a copy of the License at |
||||||
|
* |
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* |
||||||
|
* Unless required by applicable law or agreed to in writing, software |
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS, |
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||||
|
* See the License for the specific language governing permissions and |
||||||
|
* limitations under the License. |
||||||
|
*/ |
||||||
|
|
||||||
|
package org.apache.dolphinscheduler.server.master.utils; |
||||||
|
|
||||||
|
import java.time.Duration; |
||||||
|
import java.util.concurrent.ScheduledExecutorService; |
||||||
|
import java.util.concurrent.atomic.AtomicBoolean; |
||||||
|
|
||||||
|
import org.awaitility.Awaitility; |
||||||
|
import org.junit.jupiter.api.Test; |
||||||
|
|
||||||
|
import com.google.common.truth.Truth; |
||||||
|
|
||||||
|
class MasterThreadFactoryTest { |
||||||
|
|
||||||
|
@Test |
||||||
|
void getDefaultSchedulerThreadExecutor() { |
||||||
|
ScheduledExecutorService defaultSchedulerThreadExecutor = |
||||||
|
MasterThreadFactory.getDefaultSchedulerThreadExecutor(); |
||||||
|
Truth.assertThat(defaultSchedulerThreadExecutor).isNotNull(); |
||||||
|
|
||||||
|
AtomicBoolean taskOneFlag = new AtomicBoolean(false); |
||||||
|
defaultSchedulerThreadExecutor.scheduleWithFixedDelay(() -> { |
||||||
|
taskOneFlag.set(true); |
||||||
|
}, 0, |
||||||
|
1, |
||||||
|
java.util.concurrent.TimeUnit.SECONDS); |
||||||
|
|
||||||
|
AtomicBoolean taskTwoFlag = new AtomicBoolean(false); |
||||||
|
defaultSchedulerThreadExecutor.scheduleWithFixedDelay(() -> { |
||||||
|
taskTwoFlag.set(true); |
||||||
|
}, 0, |
||||||
|
1, |
||||||
|
java.util.concurrent.TimeUnit.SECONDS); |
||||||
|
|
||||||
|
Awaitility.await() |
||||||
|
.atMost(Duration.ofSeconds(5)) |
||||||
|
.untilAsserted(() -> { |
||||||
|
Truth.assertThat(taskOneFlag.get()).isTrue(); |
||||||
|
Truth.assertThat(taskTwoFlag.get()).isTrue(); |
||||||
|
}); |
||||||
|
|
||||||
|
} |
||||||
|
} |
@ -1,120 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.service.queue; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.common.model.Server; |
|
||||||
import org.apache.dolphinscheduler.common.utils.NetUtils; |
|
||||||
|
|
||||||
import java.util.Arrays; |
|
||||||
import java.util.Collection; |
|
||||||
import java.util.Comparator; |
|
||||||
import java.util.HashMap; |
|
||||||
import java.util.concurrent.PriorityBlockingQueue; |
|
||||||
import java.util.concurrent.TimeUnit; |
|
||||||
|
|
||||||
public class MasterPriorityQueue implements TaskPriorityQueue<Server> { |
|
||||||
|
|
||||||
/** |
|
||||||
* queue size |
|
||||||
*/ |
|
||||||
private static final Integer QUEUE_MAX_SIZE = 20; |
|
||||||
|
|
||||||
/** |
|
||||||
* queue |
|
||||||
*/ |
|
||||||
private PriorityBlockingQueue<Server> queue = new PriorityBlockingQueue<>(QUEUE_MAX_SIZE, new ServerComparator()); |
|
||||||
|
|
||||||
private HashMap<String, Integer> hostIndexMap = new HashMap<>(); |
|
||||||
|
|
||||||
@Override |
|
||||||
public void put(Server serverInfo) { |
|
||||||
this.queue.put(serverInfo); |
|
||||||
refreshMasterList(); |
|
||||||
} |
|
||||||
|
|
||||||
@Override |
|
||||||
public Server take() throws InterruptedException { |
|
||||||
return queue.take(); |
|
||||||
} |
|
||||||
|
|
||||||
@Override |
|
||||||
public Server poll(long timeout, TimeUnit unit) { |
|
||||||
return queue.poll(); |
|
||||||
} |
|
||||||
|
|
||||||
@Override |
|
||||||
public int size() { |
|
||||||
return queue.size(); |
|
||||||
} |
|
||||||
|
|
||||||
public void putAll(Collection<Server> serverList) { |
|
||||||
for (Server server : serverList) { |
|
||||||
this.queue.put(server); |
|
||||||
} |
|
||||||
refreshMasterList(); |
|
||||||
} |
|
||||||
|
|
||||||
public void remove(Server server) { |
|
||||||
this.queue.remove(server); |
|
||||||
} |
|
||||||
|
|
||||||
public void clear() { |
|
||||||
queue.clear(); |
|
||||||
refreshMasterList(); |
|
||||||
} |
|
||||||
|
|
||||||
private void refreshMasterList() { |
|
||||||
hostIndexMap.clear(); |
|
||||||
int index = 0; |
|
||||||
for (Server server : getOrderedElements()) { |
|
||||||
String addr = NetUtils.getAddr(server.getHost(), server.getPort()); |
|
||||||
hostIndexMap.put(addr, index); |
|
||||||
index += 1; |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
/** |
|
||||||
* get ordered collection of priority queue |
|
||||||
* |
|
||||||
* @return ordered collection |
|
||||||
*/ |
|
||||||
Server[] getOrderedElements() { |
|
||||||
Server[] nQueue = queue.toArray(new Server[0]); |
|
||||||
Arrays.sort(nQueue, new ServerComparator()); |
|
||||||
return nQueue; |
|
||||||
} |
|
||||||
|
|
||||||
public int getIndex(String addr) { |
|
||||||
if (!hostIndexMap.containsKey(addr)) { |
|
||||||
return -1; |
|
||||||
} |
|
||||||
return hostIndexMap.get(addr); |
|
||||||
} |
|
||||||
|
|
||||||
/** |
|
||||||
* server comparator, used to sort server by createTime in reverse order. |
|
||||||
*/ |
|
||||||
private class ServerComparator implements Comparator<Server> { |
|
||||||
|
|
||||||
@Override |
|
||||||
public int compare(Server o1, Server o2) { |
|
||||||
return o2.getCreateTime().compareTo(o1.getCreateTime()); |
|
||||||
} |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
@ -1,101 +0,0 @@ |
|||||||
/* |
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
|
||||||
* contributor license agreements. See the NOTICE file distributed with |
|
||||||
* this work for additional information regarding copyright ownership. |
|
||||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
|
||||||
* (the "License"); you may not use this file except in compliance with |
|
||||||
* the License. You may obtain a copy of the License at |
|
||||||
* |
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
* |
|
||||||
* Unless required by applicable law or agreed to in writing, software |
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS, |
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
|
||||||
* See the License for the specific language governing permissions and |
|
||||||
* limitations under the License. |
|
||||||
*/ |
|
||||||
|
|
||||||
package org.apache.dolphinscheduler.service.queue; |
|
||||||
|
|
||||||
import org.apache.dolphinscheduler.common.model.Server; |
|
||||||
|
|
||||||
import java.util.Arrays; |
|
||||||
import java.util.Date; |
|
||||||
import java.util.List; |
|
||||||
import java.util.stream.Collectors; |
|
||||||
|
|
||||||
import org.junit.jupiter.api.Assertions; |
|
||||||
import org.junit.jupiter.api.Test; |
|
||||||
|
|
||||||
public class MasterPriorityQueueTest { |
|
||||||
|
|
||||||
@Test |
|
||||||
public void getOrderedCollection() { |
|
||||||
|
|
||||||
MasterPriorityQueue queue = new MasterPriorityQueue(); |
|
||||||
|
|
||||||
// Test empty queue
|
|
||||||
Server[] emptyElements = queue.getOrderedElements(); |
|
||||||
Assertions.assertArrayEquals(emptyElements, new Server[]{}); |
|
||||||
|
|
||||||
// Test queue with fabricated servers
|
|
||||||
queue.putAll(getServerList()); |
|
||||||
Server[] orderElements = queue.getOrderedElements(); |
|
||||||
Assertions.assertEquals(extractServerIds(orderElements), Arrays.asList(4, 2, 1, 3)); |
|
||||||
|
|
||||||
} |
|
||||||
|
|
||||||
@Test |
|
||||||
public void refreshMasterList() { |
|
||||||
MasterPriorityQueue queue = new MasterPriorityQueue(); |
|
||||||
|
|
||||||
// Test empty queue
|
|
||||||
queue.clear(); |
|
||||||
Assertions.assertEquals(queue.getIndex("127.0.0.1:124"), -1); |
|
||||||
|
|
||||||
// Test queue with fabricated servers
|
|
||||||
queue.putAll(getServerList()); |
|
||||||
|
|
||||||
Assertions.assertEquals(queue.getIndex("127.0.0.1:124"), 0); |
|
||||||
Assertions.assertEquals(queue.getIndex("127.0.0.1:122"), 1); |
|
||||||
Assertions.assertEquals(queue.getIndex("127.0.0.1:121"), 2); |
|
||||||
Assertions.assertEquals(queue.getIndex("127.0.0.1:123"), 3); |
|
||||||
|
|
||||||
} |
|
||||||
|
|
||||||
private List<Server> getServerList() { |
|
||||||
|
|
||||||
long baseTime = new Date().getTime(); |
|
||||||
|
|
||||||
Server server1 = new Server(); |
|
||||||
server1.setId(1); |
|
||||||
server1.setHost("127.0.0.1"); |
|
||||||
server1.setPort(121); |
|
||||||
server1.setCreateTime(new Date(baseTime - 1000)); |
|
||||||
|
|
||||||
Server server2 = new Server(); |
|
||||||
server2.setId(2); |
|
||||||
server2.setHost("127.0.0.1"); |
|
||||||
server2.setPort(122); |
|
||||||
server2.setCreateTime(new Date(baseTime + 1000)); |
|
||||||
|
|
||||||
Server server3 = new Server(); |
|
||||||
server3.setId(3); |
|
||||||
server3.setHost("127.0.0.1"); |
|
||||||
server3.setPort(123); |
|
||||||
server3.setCreateTime(new Date(baseTime - 2000)); |
|
||||||
|
|
||||||
Server server4 = new Server(); |
|
||||||
server4.setId(4); |
|
||||||
server4.setHost("127.0.0.1"); |
|
||||||
server4.setPort(124); |
|
||||||
server4.setCreateTime(new Date(baseTime + 2000)); |
|
||||||
|
|
||||||
return Arrays.asList(server1, server2, server3, server4); |
|
||||||
} |
|
||||||
|
|
||||||
private List<Integer> extractServerIds(Server[] servers) { |
|
||||||
return Arrays.stream(servers).map(Server::getId).collect(Collectors.toList()); |
|
||||||
} |
|
||||||
|
|
||||||
} |
|
Loading…
Reference in new issue