66 * to you under the Apache License, Version 2.0 (the
77 * "License"); you may not use this file except in compliance
88 * with the License. You may obtain a copy of the License at
9- *
9+ * <p>
1010 * http://www.apache.org/licenses/LICENSE-2.0
11- *
11+ * <p>
1212 * Unless required by applicable law or agreed to in writing, software
1313 * distributed under the License is distributed on an "AS IS" BASIS,
1414 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
2121import com .dtstack .flink .sql .enums .ClusterMode ;
2222import com .dtstack .flink .sql .option .Options ;
2323import com .dtstack .flink .sql .util .PluginUtil ;
24- import com .esotericsoftware .minlog .Log ;
2524import org .apache .commons .io .Charsets ;
2625import org .apache .commons .lang .StringUtils ;
2726import org .apache .flink .client .program .ClusterClient ;
28- import org .apache .flink .client .program .MiniClusterClient ;
27+ import org .apache .flink .client .program .rest . RestClusterClient ;
2928import org .apache .flink .configuration .Configuration ;
3029import org .apache .flink .configuration .GlobalConfiguration ;
3130import org .apache .flink .configuration .JobManagerOptions ;
3231import org .apache .flink .core .fs .FileSystem ;
3332import org .apache .flink .runtime .akka .AkkaUtils ;
34- import org .apache .flink .runtime .minicluster .MiniCluster ;
35- import org .apache .flink .runtime .minicluster .MiniClusterConfiguration ;
3633import org .apache .flink .runtime .util .LeaderConnectionInfo ;
3734import org .apache .flink .yarn .AbstractYarnClusterDescriptor ;
3835import org .apache .flink .yarn .YarnClusterDescriptor ;
4239import org .apache .hadoop .yarn .client .api .YarnClient ;
4340import org .apache .hadoop .yarn .conf .YarnConfiguration ;
4441import org .apache .hadoop .yarn .util .StringHelper ;
42+ import org .slf4j .Logger ;
43+ import org .slf4j .LoggerFactory ;
4544
4645import java .net .InetSocketAddress ;
4746import java .net .URLDecoder ;
4847import java .util .EnumSet ;
4948import java .util .HashSet ;
50- import java .util .Iterator ;
49+ import java .util .Set ;
5150import java .util .List ;
5251import java .util .Properties ;
53- import java .util .Set ;
52+ import java .util .Iterator ;
5453
5554/**
5655 * @author sishu.yss
5756 */
5857public class ClusterClientFactory {
5958
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+
6071 public static ClusterClient createClusterClient (Options launcherOptions ) throws Exception {
6172 String mode = launcherOptions .getMode ();
6273 if (mode .equals (ClusterMode .standalone .name ())) {
@@ -70,10 +81,12 @@ public static ClusterClient createClusterClient(Options launcherOptions) throws
7081 public static ClusterClient createStandaloneClient (Options launcherOptions ) throws Exception {
7182 String flinkConfDir = launcherOptions .getFlinkconf ();
7283 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" );
7790 LeaderConnectionInfo connectionInfo = clusterClient .getClusterConnectionInfo ();
7891 InetSocketAddress address = AkkaUtils .getInetSocketAddressFromAkkaURL (connectionInfo .getAddress ());
7992 config .setString (JobManagerOptions .ADDRESS , address .getAddress ().getHostName ());
@@ -89,18 +102,21 @@ public static ClusterClient createYarnSessionClient(Options launcherOptions) {
89102
90103 if (StringUtils .isNotBlank (yarnConfDir )) {
91104 try {
92- config .setString ("fs.hdfs.hadoopconf" , yarnConfDir );
105+ boolean isHighAvailability ;
106+
107+ config .setString (HADOOP_CONF , yarnConfDir );
93108 FileSystem .initialize (config );
94109
95110 YarnConfiguration yarnConf = YarnConfLoader .getYarnConf (yarnConfDir );
96111 YarnClient yarnClient = YarnClient .createYarnClient ();
97112 yarnClient .init (yarnConf );
98113 yarnClient .start ();
99- ApplicationId applicationId = null ;
114+ ApplicationId applicationId ;
100115
101116 String yarnSessionConf = launcherOptions .getYarnSessionConf ();
102117 yarnSessionConf = URLDecoder .decode (yarnSessionConf , Charsets .UTF_8 .toString ());
103118 Properties yarnSessionConfProperties = PluginUtil .jsonStrToObject (yarnSessionConf , Properties .class );
119+
104120 Object yid = yarnSessionConfProperties .get ("yid" );
105121
106122 if (null != yid ) {
@@ -109,20 +125,30 @@ public static ClusterClient createYarnSessionClient(Options launcherOptions) {
109125 applicationId = getYarnClusterApplicationId (yarnClient );
110126 }
111127
112- Log .info ("applicationId= {}" , applicationId .toString ());
128+ LOG .info ("current applicationId = {}" , applicationId .toString ());
113129
114130 if (StringUtils .isEmpty (applicationId .toString ())) {
115131 throw new RuntimeException ("No flink session found on yarn cluster." );
116132 }
117133
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+
118144 AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor (config , yarnConf , flinkConfDir , yarnClient , false );
119145 ClusterClient clusterClient = clusterDescriptor .retrieve (applicationId );
120146 clusterClient .setDetached (true );
121147 return clusterClient ;
122148 } catch (Exception e ) {
123149 throw new RuntimeException (e );
124150 }
125- }else {
151+ } else {
126152 throw new RuntimeException ("yarn mode must set param of 'yarnconf'!!!" );
127153 }
128154 }
@@ -158,7 +184,7 @@ private static ApplicationId getYarnClusterApplicationId(YarnClient yarnClient)
158184
159185 }
160186
161- if (null == applicationId ) {
187+ if (applicationId == null || StringUtils . isEmpty ( applicationId . toString ()) ) {
162188 throw new RuntimeException ("No flink session found on yarn cluster." );
163189 }
164190 return applicationId ;
0 commit comments