Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
63b13c5f
F
flink
项目概览
doujutun3207
/
flink
与 Fork 源项目一致
从无法访问的项目Fork
通知
24
Star
0
Fork
0
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
F
flink
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
提交
Issue看板
体验新版 GitCode,发现更多精彩内容 >>
未验证
提交
63b13c5f
编写于
3月 10, 2020
作者:
X
Xintong Song
提交者:
Till Rohrmann
4月 27, 2020
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[hotfix][runtime] Code deduplication in ResourceManagerFactory and its implementations.
上级
b9fe201b
变更
10
隐藏空白更改
内联
并排
Showing
10 changed file
with
125 addition
and
79 deletion
+125
-79
flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java
.../container/entrypoint/StandaloneJobClusterEntryPoint.java
+1
-1
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/entrypoint/KubernetesResourceManagerFactory.java
...bernetes/entrypoint/KubernetesResourceManagerFactory.java
+13
-11
flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerFactory.java
...runtime/clusterframework/MesosResourceManagerFactory.java
+12
-10
flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java
...untime/entrypoint/StandaloneSessionClusterEntrypoint.java
+1
-1
flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
...ava/org/apache/flink/runtime/minicluster/MiniCluster.java
+1
-1
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactory.java
...runtime/resourcemanager/ActiveResourceManagerFactory.java
+3
-14
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerFactory.java
...flink/runtime/resourcemanager/ResourceManagerFactory.java
+54
-12
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerFactory.java
...ime/resourcemanager/StandaloneResourceManagerFactory.java
+27
-18
flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
...he/flink/test/recovery/ProcessFailureCancelingITCase.java
+1
-1
flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnResourceManagerFactory.java
...che/flink/yarn/entrypoint/YarnResourceManagerFactory.java
+12
-10
未找到文件。
flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java
浏览文件 @
63b13c5f
...
...
@@ -80,7 +80,7 @@ public final class StandaloneJobClusterEntryPoint extends ClusterEntrypoint {
new
DefaultDispatcherRunnerFactory
(
ApplicationDispatcherLeaderProcessFactoryFactory
.
create
(
configuration
,
SessionDispatcherFactory
.
INSTANCE
,
program
)),
StandaloneResourceManagerFactory
.
INSTANCE
,
StandaloneResourceManagerFactory
.
getInstance
()
,
JobRestEndpointFactory
.
INSTANCE
);
}
...
...
flink-kubernetes/src/main/java/org/apache/flink/kubernetes/entrypoint/KubernetesResourceManagerFactory.java
浏览文件 @
63b13c5f
...
...
@@ -38,6 +38,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices;
import
org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration
;
import
org.apache.flink.runtime.rpc.FatalErrorHandler
;
import
org.apache.flink.runtime.rpc.RpcService
;
import
org.apache.flink.util.ConfigurationException
;
import
javax.annotation.Nullable
;
...
...
@@ -57,7 +58,7 @@ public class KubernetesResourceManagerFactory extends ActiveResourceManagerFacto
}
@Override
public
ResourceManager
<
KubernetesWorkerNode
>
create
Active
ResourceManager
(
public
ResourceManager
<
KubernetesWorkerNode
>
createResourceManager
(
Configuration
configuration
,
ResourceID
resourceId
,
RpcService
rpcService
,
...
...
@@ -66,14 +67,9 @@ public class KubernetesResourceManagerFactory extends ActiveResourceManagerFacto
FatalErrorHandler
fatalErrorHandler
,
ClusterInformation
clusterInformation
,
@Nullable
String
webInterfaceUrl
,
ResourceManagerMetricGroup
resourceManagerMetricGroup
)
throws
Exception
{
final
ResourceManagerRuntimeServicesConfiguration
rmServicesConfiguration
=
ResourceManagerRuntimeServicesConfiguration
.
fromConfiguration
(
configuration
,
KubernetesWorkerResourceSpecFactory
.
INSTANCE
);
final
ResourceManagerRuntimeServices
rmRuntimeServices
=
ResourceManagerRuntimeServices
.
fromConfiguration
(
rmServicesConfiguration
,
highAvailabilityServices
,
rpcService
.
getScheduledExecutor
());
ResourceManagerMetricGroup
resourceManagerMetricGroup
,
ResourceManagerRuntimeServices
resourceManagerRuntimeServices
)
{
final
KubernetesResourceManagerConfiguration
kubernetesResourceManagerConfiguration
=
new
KubernetesResourceManagerConfiguration
(
configuration
.
getString
(
KubernetesConfigOptions
.
CLUSTER_ID
),
...
...
@@ -85,13 +81,19 @@ public class KubernetesResourceManagerFactory extends ActiveResourceManagerFacto
configuration
,
highAvailabilityServices
,
heartbeatServices
,
r
m
RuntimeServices
.
getSlotManager
(),
r
esourceManager
RuntimeServices
.
getSlotManager
(),
ResourceManagerPartitionTrackerImpl:
:
new
,
r
m
RuntimeServices
.
getJobLeaderIdService
(),
r
esourceManager
RuntimeServices
.
getJobLeaderIdService
(),
clusterInformation
,
fatalErrorHandler
,
resourceManagerMetricGroup
,
KubeClientFactory
.
fromConfiguration
(
configuration
),
kubernetesResourceManagerConfiguration
);
}
@Override
protected
ResourceManagerRuntimeServicesConfiguration
createResourceManagerRuntimeServicesConfiguration
(
Configuration
configuration
)
throws
ConfigurationException
{
return
ResourceManagerRuntimeServicesConfiguration
.
fromConfiguration
(
configuration
,
KubernetesWorkerResourceSpecFactory
.
INSTANCE
);
}
}
flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerFactory.java
浏览文件 @
63b13c5f
...
...
@@ -36,6 +36,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices;
import
org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration
;
import
org.apache.flink.runtime.rpc.FatalErrorHandler
;
import
org.apache.flink.runtime.rpc.RpcService
;
import
org.apache.flink.util.ConfigurationException
;
import
org.slf4j.Logger
;
import
org.slf4j.LoggerFactory
;
...
...
@@ -62,7 +63,7 @@ public class MesosResourceManagerFactory extends ActiveResourceManagerFactory<Re
}
@Override
public
ResourceManager
<
RegisteredMesosWorkerNode
>
create
Active
ResourceManager
(
public
ResourceManager
<
RegisteredMesosWorkerNode
>
createResourceManager
(
Configuration
configuration
,
ResourceID
resourceId
,
RpcService
rpcService
,
...
...
@@ -71,13 +72,8 @@ public class MesosResourceManagerFactory extends ActiveResourceManagerFactory<Re
FatalErrorHandler
fatalErrorHandler
,
ClusterInformation
clusterInformation
,
@Nullable
String
webInterfaceUrl
,
ResourceManagerMetricGroup
resourceManagerMetricGroup
)
throws
Exception
{
final
ResourceManagerRuntimeServicesConfiguration
rmServicesConfiguration
=
ResourceManagerRuntimeServicesConfiguration
.
fromConfiguration
(
configuration
,
MesosWorkerResourceSpecFactory
.
INSTANCE
);
final
ResourceManagerRuntimeServices
rmRuntimeServices
=
ResourceManagerRuntimeServices
.
fromConfiguration
(
rmServicesConfiguration
,
highAvailabilityServices
,
rpcService
.
getScheduledExecutor
());
ResourceManagerMetricGroup
resourceManagerMetricGroup
,
ResourceManagerRuntimeServices
resourceManagerRuntimeServices
)
throws
Exception
{
final
MesosTaskManagerParameters
taskManagerParameters
=
MesosUtils
.
createTmParameters
(
configuration
,
LOG
);
final
ContainerSpecification
taskManagerContainerSpec
=
MesosUtils
.
createContainerSpec
(
configuration
);
...
...
@@ -87,9 +83,9 @@ public class MesosResourceManagerFactory extends ActiveResourceManagerFactory<Re
resourceId
,
highAvailabilityServices
,
heartbeatServices
,
r
m
RuntimeServices
.
getSlotManager
(),
r
esourceManager
RuntimeServices
.
getSlotManager
(),
ResourceManagerPartitionTrackerImpl:
:
new
,
r
m
RuntimeServices
.
getJobLeaderIdService
(),
r
esourceManager
RuntimeServices
.
getJobLeaderIdService
(),
clusterInformation
,
fatalErrorHandler
,
configuration
,
...
...
@@ -100,4 +96,10 @@ public class MesosResourceManagerFactory extends ActiveResourceManagerFactory<Re
webInterfaceUrl
,
resourceManagerMetricGroup
);
}
@Override
protected
ResourceManagerRuntimeServicesConfiguration
createResourceManagerRuntimeServicesConfiguration
(
Configuration
configuration
)
throws
ConfigurationException
{
return
ResourceManagerRuntimeServicesConfiguration
.
fromConfiguration
(
configuration
,
MesosWorkerResourceSpecFactory
.
INSTANCE
);
}
}
flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java
浏览文件 @
63b13c5f
...
...
@@ -37,7 +37,7 @@ public class StandaloneSessionClusterEntrypoint extends SessionClusterEntrypoint
@Override
protected
DefaultDispatcherResourceManagerComponentFactory
createDispatcherResourceManagerComponentFactory
(
Configuration
configuration
)
{
return
DefaultDispatcherResourceManagerComponentFactory
.
createSessionComponentFactory
(
StandaloneResourceManagerFactory
.
INSTANCE
);
return
DefaultDispatcherResourceManagerComponentFactory
.
createSessionComponentFactory
(
StandaloneResourceManagerFactory
.
getInstance
()
);
}
public
static
void
main
(
String
[]
args
)
{
...
...
flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
浏览文件 @
63b13c5f
...
...
@@ -417,7 +417,7 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync {
@Nonnull
DispatcherResourceManagerComponentFactory
createDispatcherResourceManagerComponentFactory
()
{
return
DefaultDispatcherResourceManagerComponentFactory
.
createSessionComponentFactory
(
StandaloneResourceManagerFactory
.
INSTANCE
);
return
DefaultDispatcherResourceManagerComponentFactory
.
createSessionComponentFactory
(
StandaloneResourceManagerFactory
.
getInstance
()
);
}
@VisibleForTesting
...
...
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerFactory.java
浏览文件 @
63b13c5f
...
...
@@ -41,7 +41,7 @@ import javax.annotation.Nullable;
*
* @param <T> type of the {@link ResourceIDRetrievable}
*/
public
abstract
class
ActiveResourceManagerFactory
<
T
extends
ResourceIDRetrievable
>
implement
s
ResourceManagerFactory
<
T
>
{
public
abstract
class
ActiveResourceManagerFactory
<
T
extends
ResourceIDRetrievable
>
extend
s
ResourceManagerFactory
<
T
>
{
@Override
public
ResourceManager
<
T
>
createResourceManager
(
...
...
@@ -54,7 +54,7 @@ public abstract class ActiveResourceManagerFactory<T extends ResourceIDRetrievab
ClusterInformation
clusterInformation
,
@Nullable
String
webInterfaceUrl
,
ResourceManagerMetricGroup
resourceManagerMetricGroup
)
throws
Exception
{
return
createActiv
eResourceManager
(
return
super
.
creat
eResourceManager
(
createActiveResourceManagerConfiguration
(
configuration
),
resourceId
,
rpcService
,
...
...
@@ -66,19 +66,8 @@ public abstract class ActiveResourceManagerFactory<T extends ResourceIDRetrievab
resourceManagerMetricGroup
);
}
private
static
Configuration
createActiveResourceManagerConfiguration
(
Configuration
originalConfiguration
)
{
private
Configuration
createActiveResourceManagerConfiguration
(
Configuration
originalConfiguration
)
{
return
TaskExecutorProcessUtils
.
getConfigurationMapLegacyTaskManagerHeapSizeToConfigOption
(
originalConfiguration
,
TaskManagerOptions
.
TOTAL_PROCESS_MEMORY
);
}
protected
abstract
ResourceManager
<
T
>
createActiveResourceManager
(
Configuration
configuration
,
ResourceID
resourceId
,
RpcService
rpcService
,
HighAvailabilityServices
highAvailabilityServices
,
HeartbeatServices
heartbeatServices
,
FatalErrorHandler
fatalErrorHandler
,
ClusterInformation
clusterInformation
,
@Nullable
String
webInterfaceUrl
,
ResourceManagerMetricGroup
resourceManagerMetricGroup
)
throws
Exception
;
}
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerFactory.java
浏览文件 @
63b13c5f
...
...
@@ -27,6 +27,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
import
org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup
;
import
org.apache.flink.runtime.rpc.FatalErrorHandler
;
import
org.apache.flink.runtime.rpc.RpcService
;
import
org.apache.flink.util.ConfigurationException
;
import
javax.annotation.Nullable
;
...
...
@@ -35,16 +36,57 @@ import javax.annotation.Nullable;
*
* @param <T> type of the workers of the ResourceManager
*/
public
interface
ResourceManagerFactory
<
T
extends
ResourceIDRetrievable
>
{
ResourceManager
<
T
>
createResourceManager
(
Configuration
configuration
,
ResourceID
resourceId
,
RpcService
rpcService
,
HighAvailabilityServices
highAvailabilityServices
,
HeartbeatServices
heartbeatServices
,
FatalErrorHandler
fatalErrorHandler
,
ClusterInformation
clusterInformation
,
@Nullable
String
webInterfaceUrl
,
ResourceManagerMetricGroup
resourceManagerMetricGroup
)
throws
Exception
;
public
abstract
class
ResourceManagerFactory
<
T
extends
ResourceIDRetrievable
>
{
public
ResourceManager
<
T
>
createResourceManager
(
Configuration
configuration
,
ResourceID
resourceId
,
RpcService
rpcService
,
HighAvailabilityServices
highAvailabilityServices
,
HeartbeatServices
heartbeatServices
,
FatalErrorHandler
fatalErrorHandler
,
ClusterInformation
clusterInformation
,
@Nullable
String
webInterfaceUrl
,
ResourceManagerMetricGroup
resourceManagerMetricGroup
)
throws
Exception
{
final
ResourceManagerRuntimeServices
resourceManagerRuntimeServices
=
createResourceManagerRuntimeServices
(
configuration
,
rpcService
,
highAvailabilityServices
);
return
createResourceManager
(
configuration
,
resourceId
,
rpcService
,
highAvailabilityServices
,
heartbeatServices
,
fatalErrorHandler
,
clusterInformation
,
webInterfaceUrl
,
resourceManagerMetricGroup
,
resourceManagerRuntimeServices
);
}
protected
abstract
ResourceManager
<
T
>
createResourceManager
(
Configuration
configuration
,
ResourceID
resourceId
,
RpcService
rpcService
,
HighAvailabilityServices
highAvailabilityServices
,
HeartbeatServices
heartbeatServices
,
FatalErrorHandler
fatalErrorHandler
,
ClusterInformation
clusterInformation
,
@Nullable
String
webInterfaceUrl
,
ResourceManagerMetricGroup
resourceManagerMetricGroup
,
ResourceManagerRuntimeServices
resourceManagerRuntimeServices
)
throws
Exception
;
private
ResourceManagerRuntimeServices
createResourceManagerRuntimeServices
(
Configuration
configuration
,
RpcService
rpcService
,
HighAvailabilityServices
highAvailabilityServices
)
throws
ConfigurationException
{
return
ResourceManagerRuntimeServices
.
fromConfiguration
(
createResourceManagerRuntimeServicesConfiguration
(
configuration
),
highAvailabilityServices
,
rpcService
.
getScheduledExecutor
());
}
protected
abstract
ResourceManagerRuntimeServicesConfiguration
createResourceManagerRuntimeServicesConfiguration
(
Configuration
configuration
)
throws
ConfigurationException
;
}
flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerFactory.java
浏览文件 @
63b13c5f
...
...
@@ -30,32 +30,35 @@ import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTra
import
org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup
;
import
org.apache.flink.runtime.rpc.FatalErrorHandler
;
import
org.apache.flink.runtime.rpc.RpcService
;
import
org.apache.flink.util.ConfigurationException
;
import
javax.annotation.Nullable
;
/**
* {@link ResourceManagerFactory} which creates a {@link StandaloneResourceManager}.
*/
public
enum
StandaloneResourceManagerFactory
implements
ResourceManagerFactory
<
ResourceID
>
{
INSTANCE
;
public
final
class
StandaloneResourceManagerFactory
extends
ResourceManagerFactory
<
ResourceID
>
{
private
static
final
StandaloneResourceManagerFactory
INSTANCE
=
new
StandaloneResourceManagerFactory
();
private
StandaloneResourceManagerFactory
()
{}
public
static
StandaloneResourceManagerFactory
getInstance
()
{
return
INSTANCE
;
}
@Override
public
ResourceManager
<
ResourceID
>
createResourceManager
(
Configuration
configuration
,
ResourceID
resourceId
,
RpcService
rpcService
,
HighAvailabilityServices
highAvailabilityServices
,
HeartbeatServices
heartbeatServices
,
FatalErrorHandler
fatalErrorHandler
,
ClusterInformation
clusterInformation
,
@Nullable
String
webInterfaceUrl
,
ResourceManagerMetricGroup
resourceManagerMetricGroup
)
throws
Exception
{
final
ResourceManagerRuntimeServicesConfiguration
resourceManagerRuntimeServicesConfiguration
=
ResourceManagerRuntimeServicesConfiguration
.
fromConfiguration
(
configuration
,
ArbitraryWorkerResourceSpecFactory
.
INSTANCE
);
final
ResourceManagerRuntimeServices
resourceManagerRuntimeServices
=
ResourceManagerRuntimeServices
.
fromConfiguration
(
resourceManagerRuntimeServicesConfiguration
,
highAvailabilityServices
,
rpcService
.
getScheduledExecutor
());
protected
ResourceManager
<
ResourceID
>
createResourceManager
(
Configuration
configuration
,
ResourceID
resourceId
,
RpcService
rpcService
,
HighAvailabilityServices
highAvailabilityServices
,
HeartbeatServices
heartbeatServices
,
FatalErrorHandler
fatalErrorHandler
,
ClusterInformation
clusterInformation
,
@Nullable
String
webInterfaceUrl
,
ResourceManagerMetricGroup
resourceManagerMetricGroup
,
ResourceManagerRuntimeServices
resourceManagerRuntimeServices
)
{
final
Time
standaloneClusterStartupPeriodTime
=
ConfigurationUtils
.
getStandaloneClusterStartupPeriodTime
(
configuration
);
...
...
@@ -73,4 +76,10 @@ public enum StandaloneResourceManagerFactory implements ResourceManagerFactory<R
standaloneClusterStartupPeriodTime
,
AkkaUtils
.
getTimeoutAsTime
(
configuration
));
}
@Override
protected
ResourceManagerRuntimeServicesConfiguration
createResourceManagerRuntimeServicesConfiguration
(
Configuration
configuration
)
throws
ConfigurationException
{
return
ResourceManagerRuntimeServicesConfiguration
.
fromConfiguration
(
configuration
,
ArbitraryWorkerResourceSpecFactory
.
INSTANCE
);
}
}
flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java
浏览文件 @
63b13c5f
...
...
@@ -128,7 +128,7 @@ public class ProcessFailureCancelingITCase extends TestLogger {
config
.
setInteger
(
JobManagerOptions
.
PORT
,
jobManagerPort
);
final
DispatcherResourceManagerComponentFactory
resourceManagerComponentFactory
=
DefaultDispatcherResourceManagerComponentFactory
.
createSessionComponentFactory
(
StandaloneResourceManagerFactory
.
INSTANCE
);
StandaloneResourceManagerFactory
.
getInstance
()
);
DispatcherResourceManagerComponent
dispatcherResourceManagerComponent
=
null
;
final
ScheduledExecutorService
ioExecutor
=
TestingUtils
.
defaultExecutor
();
...
...
flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnResourceManagerFactory.java
浏览文件 @
63b13c5f
...
...
@@ -32,6 +32,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices;
import
org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration
;
import
org.apache.flink.runtime.rpc.FatalErrorHandler
;
import
org.apache.flink.runtime.rpc.RpcService
;
import
org.apache.flink.util.ConfigurationException
;
import
org.apache.flink.yarn.YarnResourceManager
;
import
org.apache.flink.yarn.YarnWorkerNode
;
...
...
@@ -51,7 +52,7 @@ public class YarnResourceManagerFactory extends ActiveResourceManagerFactory<Yar
}
@Override
public
ResourceManager
<
YarnWorkerNode
>
create
Active
ResourceManager
(
public
ResourceManager
<
YarnWorkerNode
>
createResourceManager
(
Configuration
configuration
,
ResourceID
resourceId
,
RpcService
rpcService
,
...
...
@@ -60,13 +61,8 @@ public class YarnResourceManagerFactory extends ActiveResourceManagerFactory<Yar
FatalErrorHandler
fatalErrorHandler
,
ClusterInformation
clusterInformation
,
@Nullable
String
webInterfaceUrl
,
ResourceManagerMetricGroup
resourceManagerMetricGroup
)
throws
Exception
{
final
ResourceManagerRuntimeServicesConfiguration
rmServicesConfiguration
=
ResourceManagerRuntimeServicesConfiguration
.
fromConfiguration
(
configuration
,
YarnWorkerResourceSpecFactory
.
INSTANCE
);
final
ResourceManagerRuntimeServices
rmRuntimeServices
=
ResourceManagerRuntimeServices
.
fromConfiguration
(
rmServicesConfiguration
,
highAvailabilityServices
,
rpcService
.
getScheduledExecutor
());
ResourceManagerMetricGroup
resourceManagerMetricGroup
,
ResourceManagerRuntimeServices
resourceManagerRuntimeServices
)
{
return
new
YarnResourceManager
(
rpcService
,
...
...
@@ -75,12 +71,18 @@ public class YarnResourceManagerFactory extends ActiveResourceManagerFactory<Yar
System
.
getenv
(),
highAvailabilityServices
,
heartbeatServices
,
r
m
RuntimeServices
.
getSlotManager
(),
r
esourceManager
RuntimeServices
.
getSlotManager
(),
ResourceManagerPartitionTrackerImpl:
:
new
,
r
m
RuntimeServices
.
getJobLeaderIdService
(),
r
esourceManager
RuntimeServices
.
getJobLeaderIdService
(),
clusterInformation
,
fatalErrorHandler
,
webInterfaceUrl
,
resourceManagerMetricGroup
);
}
@Override
protected
ResourceManagerRuntimeServicesConfiguration
createResourceManagerRuntimeServicesConfiguration
(
Configuration
configuration
)
throws
ConfigurationException
{
return
ResourceManagerRuntimeServicesConfiguration
.
fromConfiguration
(
configuration
,
YarnWorkerResourceSpecFactory
.
INSTANCE
);
}
}
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录