Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
apache
DolphinScheduler
提交
b100f6c4
DolphinScheduler
项目概览
apache
/
DolphinScheduler
上一次同步 1 年多
通知
704
Star
9572
Fork
3514
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
DolphinScheduler
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
提交
Issue看板
前往新版Gitcode,体验更适合开发者的 AI 搜索 >>
未验证
提交
b100f6c4
编写于
6月 21, 2022
作者:
W
Wenjun Ruan
提交者:
GitHub
6月 21, 2022
1
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
Remove the schedule thread in LowerWeightHostManager (#10310)
上级
36e20cdf
变更
3
隐藏空白更改
内联
并排
Showing
3 changed file
with
100 addition
and
95 deletion
+100
-95
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/LowerWeightHostManager.java
...r/server/master/dispatch/host/LowerWeightHostManager.java
+65
-71
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManager.java
...inscheduler/server/master/registry/ServerNodeManager.java
+22
-0
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/WorkerInfoChangeListener.java
...uler/server/master/registry/WorkerInfoChangeListener.java
+13
-24
未找到文件。
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/LowerWeightHostManager.java
浏览文件 @
b100f6c4
...
...
@@ -20,11 +20,11 @@ package org.apache.dolphinscheduler.server.master.dispatch.host;
import
org.apache.dolphinscheduler.common.Constants
;
import
org.apache.dolphinscheduler.common.utils.HeartBeat
;
import
org.apache.dolphinscheduler.remote.utils.Host
;
import
org.apache.dolphinscheduler.remote.utils.NamedThreadFactory
;
import
org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext
;
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.dolphinscheduler.spi.utils.StringUtils
;
import
org.apache.commons.collections.CollectionUtils
;
...
...
@@ -36,14 +36,10 @@ 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.Lock
;
import
java.util.concurrent.locks.ReentrantLock
;
import
javax.annotation.PostConstruct
;
import
javax.annotation.PreDestroy
;
import
org.slf4j.Logger
;
import
org.slf4j.LoggerFactory
;
...
...
@@ -70,23 +66,12 @@ public class LowerWeightHostManager extends CommonHostManager {
*/
private
Lock
lock
;
/**
* executor service
*/
private
ScheduledExecutorService
executorService
;
@PostConstruct
public
void
init
()
{
this
.
selector
=
new
LowerWeightRoundRobin
();
this
.
workerHostWeightsMap
=
new
ConcurrentHashMap
<>();
this
.
lock
=
new
ReentrantLock
();
this
.
executorService
=
Executors
.
newSingleThreadScheduledExecutor
(
new
NamedThreadFactory
(
"LowerWeightHostManagerExecutor"
));
this
.
executorService
.
scheduleWithFixedDelay
(
new
RefreshResourceTask
(),
0
,
1
,
TimeUnit
.
SECONDS
);
}
@PreDestroy
public
void
close
()
{
this
.
executorService
.
shutdownNow
();
serverNodeManager
.
addWorkerInfoChangeListener
(
new
WorkerWeightListener
());
}
/**
...
...
@@ -109,6 +94,69 @@ public class LowerWeightHostManager extends CommonHostManager {
throw
new
UnsupportedOperationException
(
"not support"
);
}
private
class
WorkerWeightListener
implements
WorkerInfoChangeListener
{
@Override
public
void
notify
(
Map
<
String
,
Set
<
String
>>
workerGroups
,
Map
<
String
,
String
>
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
,
String
>
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
)
{
String
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
)
{
logger
.
error
(
"Sync worker resource error"
,
ex
);
}
}
private
Optional
<
HostWeight
>
getHostWeight
(
String
addr
,
String
workerGroup
,
String
heartBeatInfo
)
{
if
(
StringUtils
.
isEmpty
(
heartBeatInfo
))
{
logger
.
warn
(
"worker {} in work group {} have not received the heartbeat"
,
addr
,
workerGroup
);
return
Optional
.
empty
();
}
HeartBeat
heartBeat
=
HeartBeat
.
decodeHeartBeat
(
heartBeatInfo
);
if
(
heartBeat
==
null
)
{
return
Optional
.
empty
();
}
if
(
Constants
.
ABNORMAL_NODE_STATUS
==
heartBeat
.
getServerStatus
())
{
logger
.
warn
(
"worker {} current cpu load average {} is too high or available memory {}G is too low"
,
addr
,
heartBeat
.
getLoadAverage
(),
heartBeat
.
getAvailablePhysicalMemorySize
());
return
Optional
.
empty
();
}
if
(
Constants
.
BUSY_NODE_STATUE
==
heartBeat
.
getServerStatus
())
{
logger
.
warn
(
"worker {} is busy, current waiting task count {} is large than worker thread count {}"
,
addr
,
heartBeat
.
getWorkerWaitingTaskCount
(),
heartBeat
.
getWorkerExecThreadCount
());
return
Optional
.
empty
();
}
return
Optional
.
of
(
new
HostWeight
(
HostWorker
.
of
(
addr
,
heartBeat
.
getWorkerHostWeight
(),
workerGroup
),
heartBeat
.
getCpuUsage
(),
heartBeat
.
getMemoryUsage
(),
heartBeat
.
getLoadAverage
(),
heartBeat
.
getWorkerWaitingTaskCount
(),
heartBeat
.
getStartupTime
()));
}
private
void
syncWorkerHostWeight
(
Map
<
String
,
Set
<
HostWeight
>>
workerHostWeights
)
{
lock
.
lock
();
try
{
...
...
@@ -128,58 +176,4 @@ public class LowerWeightHostManager extends CommonHostManager {
}
}
class
RefreshResourceTask
implements
Runnable
{
@Override
public
void
run
()
{
try
{
Map
<
String
,
Set
<
HostWeight
>>
workerHostWeights
=
new
HashMap
<>();
Map
<
String
,
Set
<
String
>>
workerGroupNodes
=
serverNodeManager
.
getWorkerGroupNodes
();
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
)
{
String
heartbeat
=
serverNodeManager
.
getWorkerNodeInfo
(
node
);
Optional
<
HostWeight
>
hostWeightOpt
=
getHostWeight
(
node
,
workerGroup
,
heartbeat
);
if
(
hostWeightOpt
.
isPresent
())
{
hostWeights
.
add
(
hostWeightOpt
.
get
());
}
}
if
(!
hostWeights
.
isEmpty
())
{
workerHostWeights
.
put
(
workerGroup
,
hostWeights
);
}
}
syncWorkerHostWeight
(
workerHostWeights
);
}
catch
(
Throwable
ex
)
{
logger
.
error
(
"RefreshResourceTask error"
,
ex
);
}
}
public
Optional
<
HostWeight
>
getHostWeight
(
String
addr
,
String
workerGroup
,
String
heartBeatInfo
)
{
if
(
StringUtils
.
isEmpty
(
heartBeatInfo
))
{
logger
.
warn
(
"worker {} in work group {} have not received the heartbeat"
,
addr
,
workerGroup
);
return
Optional
.
empty
();
}
HeartBeat
heartBeat
=
HeartBeat
.
decodeHeartBeat
(
heartBeatInfo
);
if
(
heartBeat
==
null
)
{
return
Optional
.
empty
();
}
if
(
Constants
.
ABNORMAL_NODE_STATUS
==
heartBeat
.
getServerStatus
())
{
logger
.
warn
(
"worker {} current cpu load average {} is too high or available memory {}G is too low"
,
addr
,
heartBeat
.
getLoadAverage
(),
heartBeat
.
getAvailablePhysicalMemorySize
());
return
Optional
.
empty
();
}
if
(
Constants
.
BUSY_NODE_STATUE
==
heartBeat
.
getServerStatus
())
{
logger
.
warn
(
"worker {} is busy, current waiting task count {} is large than worker thread count {}"
,
addr
,
heartBeat
.
getWorkerWaitingTaskCount
(),
heartBeat
.
getWorkerExecThreadCount
());
return
Optional
.
empty
();
}
return
Optional
.
of
(
new
HostWeight
(
HostWorker
.
of
(
addr
,
heartBeat
.
getWorkerHostWeight
(),
workerGroup
),
heartBeat
.
getCpuUsage
(),
heartBeat
.
getMemoryUsage
(),
heartBeat
.
getLoadAverage
(),
heartBeat
.
getWorkerWaitingTaskCount
(),
heartBeat
.
getStartupTime
()));
}
}
}
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManager.java
浏览文件 @
b100f6c4
...
...
@@ -38,6 +38,7 @@ import org.apache.dolphinscheduler.service.registry.RegistryClient;
import
org.apache.commons.collections.CollectionUtils
;
import
org.apache.commons.lang3.StringUtils
;
import
java.util.ArrayList
;
import
java.util.Collection
;
import
java.util.Collections
;
import
java.util.HashMap
;
...
...
@@ -131,6 +132,8 @@ public class ServerNodeManager implements InitializingBean {
@Autowired
private
MasterConfig
masterConfig
;
private
List
<
WorkerInfoChangeListener
>
workerInfoChangeListeners
=
new
ArrayList
<>();
private
static
volatile
int
MASTER_SLOT
=
0
;
private
static
volatile
int
MASTER_SIZE
=
0
;
...
...
@@ -217,6 +220,7 @@ public class ServerNodeManager implements InitializingBean {
}
}
}
notifyWorkerInfoChangeListeners
();
}
catch
(
Exception
e
)
{
logger
.
error
(
"WorkerNodeInfoAndGroupDbSyncTask error:"
,
e
);
}
...
...
@@ -256,6 +260,7 @@ public class ServerNodeManager implements InitializingBean {
String
node
=
parseNode
(
path
);
syncSingleWorkerNodeInfo
(
node
,
data
);
}
notifyWorkerInfoChangeListeners
();
}
catch
(
IllegalArgumentException
ex
)
{
logger
.
warn
(
ex
.
getMessage
());
}
catch
(
Exception
ex
)
{
...
...
@@ -457,6 +462,23 @@ public class ServerNodeManager implements InitializingBean {
}
}
/**
* 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
>>
workerGroupNodes
=
getWorkerGroupNodes
();
Map
<
String
,
String
>
workerNodeInfo
=
getWorkerNodeInfo
();
for
(
WorkerInfoChangeListener
listener
:
workerInfoChangeListeners
)
{
listener
.
notify
(
workerGroupNodes
,
workerNodeInfo
);
}
}
/**
* destroy
*/
...
...
dolphinscheduler-master/src/
test/java/org/apache/dolphinscheduler/server/master/dispatch/host/RefreshResourceTaskTest
.java
→
dolphinscheduler-master/src/
main/java/org/apache/dolphinscheduler/server/master/registry/WorkerInfoChangeListener
.java
浏览文件 @
b100f6c4
...
...
@@ -15,33 +15,22 @@
* limitations under the License.
*/
package
org.apache.dolphinscheduler.server.master.
dispatch.host
;
package
org.apache.dolphinscheduler.server.master.
registry
;
import
org.apache.dolphinscheduler.server.master.registry.ServerNodeManager
;
import
org.junit.Assert
;
import
org.junit.Test
;
import
org.junit.runner.RunWith
;
import
org.mockito.InjectMocks
;
import
org.mockito.Mock
;
import
org.mockito.junit.MockitoJUnitRunner
;
import
java.util.Map
;
import
java.util.Set
;
/**
*
RefreshResourceTask test
*
The listener used in {@link ServerNodeManager} to notify the change of worker info.
*/
@RunWith
(
MockitoJUnitRunner
.
class
)
public
class
RefreshResourceTaskTest
{
@Mock
private
ServerNodeManager
serverNodeManager
;
public
interface
WorkerInfoChangeListener
{
@InjectMocks
LowerWeightHostManager
lowerWeightHostManager
;
/**
* 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
,
String
>
workerNodeInfo
);
@Test
public
void
testGetHostWeightWithResult
()
{
Assert
.
assertTrue
(!
lowerWeightHostManager
.
new
RefreshResourceTask
()
.
getHostWeight
(
"192.168.1.1:22"
,
"default"
,
null
)
.
isPresent
());
}
}
\ No newline at end of file
}
GitCode官方
@csdn_codechina
mentioned in commit
9a28d320
·
9月 10, 2022
mentioned in commit
9a28d320
mentioned in commit 9a28d320570e9c6768382584a97ab9d2d9225d96
开关提交列表
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录