6
6
* to you under the Apache License, Version 2.0 (the
7
7
* "License"); you may not use this file except in compliance
8
8
* with the License. You may obtain a copy of the License at
9
- *
9
+ * <p>
10
10
* http://www.apache.org/licenses/LICENSE-2.0
11
- *
11
+ * <p>
12
12
* Unless required by applicable law or agreed to in writing, software
13
13
* distributed under the License is distributed on an "AS IS" BASIS,
14
14
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
21
21
import com .dtstack .flink .sql .enums .ClusterMode ;
22
22
import com .dtstack .flink .sql .option .Options ;
23
23
import com .dtstack .flink .sql .util .PluginUtil ;
24
- import com .esotericsoftware .minlog .Log ;
25
24
import org .apache .commons .io .Charsets ;
26
25
import org .apache .commons .lang .StringUtils ;
27
26
import org .apache .flink .client .program .ClusterClient ;
28
- import org .apache .flink .client .program .MiniClusterClient ;
27
+ import org .apache .flink .client .program .rest . RestClusterClient ;
29
28
import org .apache .flink .configuration .Configuration ;
30
29
import org .apache .flink .configuration .GlobalConfiguration ;
31
30
import org .apache .flink .configuration .JobManagerOptions ;
32
31
import org .apache .flink .core .fs .FileSystem ;
33
32
import org .apache .flink .runtime .akka .AkkaUtils ;
34
- import org .apache .flink .runtime .minicluster .MiniCluster ;
35
- import org .apache .flink .runtime .minicluster .MiniClusterConfiguration ;
36
33
import org .apache .flink .runtime .util .LeaderConnectionInfo ;
37
34
import org .apache .flink .yarn .AbstractYarnClusterDescriptor ;
38
35
import org .apache .flink .yarn .YarnClusterDescriptor ;
42
39
import org .apache .hadoop .yarn .client .api .YarnClient ;
43
40
import org .apache .hadoop .yarn .conf .YarnConfiguration ;
44
41
import org .apache .hadoop .yarn .util .StringHelper ;
42
+ import org .slf4j .Logger ;
43
+ import org .slf4j .LoggerFactory ;
45
44
46
45
import java .net .InetSocketAddress ;
47
46
import java .net .URLDecoder ;
48
47
import java .util .EnumSet ;
49
48
import java .util .HashSet ;
50
- import java .util .Iterator ;
49
+ import java .util .Set ;
51
50
import java .util .List ;
52
51
import java .util .Properties ;
53
- import java .util .Set ;
52
+ import java .util .Iterator ;
54
53
55
54
/**
56
55
* @author sishu.yss
57
56
*/
58
57
public class ClusterClientFactory {
59
58
59
+ private static final Logger LOG = LoggerFactory .getLogger (ClusterClientFactory .class );
60
+
61
+ private static final String HA_CLUSTER_ID = "high-availability.cluster-id" ;
62
+
63
+ private static final String HIGH_AVAILABILITY = "high-availability" ;
64
+
65
+ private static final String NODE = "NONE" ;
66
+
67
+ private static final String ZOOKEEPER = "zookeeper" ;
68
+
69
+ private static final String HADOOP_CONF = "fs.hdfs.hadoopconf" ;
70
+
60
71
public static ClusterClient createClusterClient (Options launcherOptions ) throws Exception {
61
72
String mode = launcherOptions .getMode ();
62
73
if (mode .equals (ClusterMode .standalone .name ())) {
@@ -70,10 +81,12 @@ public static ClusterClient createClusterClient(Options launcherOptions) throws
70
81
public static ClusterClient createStandaloneClient (Options launcherOptions ) throws Exception {
71
82
String flinkConfDir = launcherOptions .getFlinkconf ();
72
83
Configuration config = GlobalConfiguration .loadConfiguration (flinkConfDir );
73
- MiniClusterConfiguration .Builder configBuilder = new MiniClusterConfiguration .Builder ();
74
- configBuilder .setConfiguration (config );
75
- MiniCluster miniCluster = new MiniCluster (configBuilder .build ());
76
- MiniClusterClient clusterClient = new MiniClusterClient (config , miniCluster );
84
+
85
+ LOG .info ("------------config params-------------------------" );
86
+ config .toMap ().forEach ((key , value ) -> LOG .info ("{}: {}" , key , value ));
87
+ LOG .info ("-------------------------------------------" );
88
+
89
+ RestClusterClient clusterClient = new RestClusterClient <>(config , "clusterClient" );
77
90
LeaderConnectionInfo connectionInfo = clusterClient .getClusterConnectionInfo ();
78
91
InetSocketAddress address = AkkaUtils .getInetSocketAddressFromAkkaURL (connectionInfo .getAddress ());
79
92
config .setString (JobManagerOptions .ADDRESS , address .getAddress ().getHostName ());
@@ -89,18 +102,21 @@ public static ClusterClient createYarnSessionClient(Options launcherOptions) {
89
102
90
103
if (StringUtils .isNotBlank (yarnConfDir )) {
91
104
try {
92
- config .setString ("fs.hdfs.hadoopconf" , yarnConfDir );
105
+ boolean isHighAvailability ;
106
+
107
+ config .setString (HADOOP_CONF , yarnConfDir );
93
108
FileSystem .initialize (config );
94
109
95
110
YarnConfiguration yarnConf = YarnConfLoader .getYarnConf (yarnConfDir );
96
111
YarnClient yarnClient = YarnClient .createYarnClient ();
97
112
yarnClient .init (yarnConf );
98
113
yarnClient .start ();
99
- ApplicationId applicationId = null ;
114
+ ApplicationId applicationId ;
100
115
101
116
String yarnSessionConf = launcherOptions .getYarnSessionConf ();
102
117
yarnSessionConf = URLDecoder .decode (yarnSessionConf , Charsets .UTF_8 .toString ());
103
118
Properties yarnSessionConfProperties = PluginUtil .jsonStrToObject (yarnSessionConf , Properties .class );
119
+
104
120
Object yid = yarnSessionConfProperties .get ("yid" );
105
121
106
122
if (null != yid ) {
@@ -109,20 +125,30 @@ public static ClusterClient createYarnSessionClient(Options launcherOptions) {
109
125
applicationId = getYarnClusterApplicationId (yarnClient );
110
126
}
111
127
112
- Log .info ("applicationId= {}" , applicationId .toString ());
128
+ LOG .info ("current applicationId = {}" , applicationId .toString ());
113
129
114
130
if (StringUtils .isEmpty (applicationId .toString ())) {
115
131
throw new RuntimeException ("No flink session found on yarn cluster." );
116
132
}
117
133
134
+ isHighAvailability = config .getString (HIGH_AVAILABILITY , NODE ).equals (ZOOKEEPER );
135
+
136
+ if (isHighAvailability && config .getString (HA_CLUSTER_ID , null ) == null ) {
137
+ config .setString (HA_CLUSTER_ID , applicationId .toString ());
138
+ }
139
+
140
+ LOG .info ("------------config params-------------------------" );
141
+ config .toMap ().forEach ((key , value ) -> LOG .info ("{}: {}" , key , value ));
142
+ LOG .info ("-------------------------------------------" );
143
+
118
144
AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor (config , yarnConf , flinkConfDir , yarnClient , false );
119
145
ClusterClient clusterClient = clusterDescriptor .retrieve (applicationId );
120
146
clusterClient .setDetached (true );
121
147
return clusterClient ;
122
148
} catch (Exception e ) {
123
149
throw new RuntimeException (e );
124
150
}
125
- }else {
151
+ } else {
126
152
throw new RuntimeException ("yarn mode must set param of 'yarnconf'!!!" );
127
153
}
128
154
}
@@ -158,7 +184,7 @@ private static ApplicationId getYarnClusterApplicationId(YarnClient yarnClient)
158
184
159
185
}
160
186
161
- if (null == applicationId ) {
187
+ if (applicationId == null || StringUtils . isEmpty ( applicationId . toString ()) ) {
162
188
throw new RuntimeException ("No flink session found on yarn cluster." );
163
189
}
164
190
return applicationId ;
0 commit comments