Skip to content

Commit 5d71d4a

Browse files
author
dapeng
committed
增加缺失的参数
1 parent a44c2be commit 5d71d4a

File tree

5 files changed

+128
-54
lines changed

5 files changed

+128
-54
lines changed

hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseConfigUtils.java

Lines changed: 6 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -18,19 +18,12 @@
1818

1919
package com.dtstack.flink.sql.sink.hbase;
2020

21-
import org.apache.commons.collections.MapUtils;
22-
import org.apache.commons.lang3.StringUtils;
2321
import org.apache.hadoop.conf.Configuration;
24-
import org.apache.hadoop.hbase.HBaseConfiguration;
2522
import org.apache.hadoop.security.UserGroupInformation;
2623
import org.slf4j.Logger;
2724
import org.slf4j.LoggerFactory;
2825

29-
import java.io.File;
3026
import java.io.IOException;
31-
import java.util.Arrays;
32-
import java.util.List;
33-
import java.util.Map;
3427

3528
/**
3629
*
@@ -52,13 +45,18 @@ public class HbaseConfigUtils {
5245
public final static String KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL = "hbase.regionserver.kerberos.principal";
5346

5447
public final static String KEY_HBASE_ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum";
55-
public final static String KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM = "hbase.zookeeper.znode.parent";
48+
public final static String KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM = "zookeeper.znode.parent";
49+
50+
public final static String KEY_HBASE_CLIENT_KEYTAB_FILE = "hbase.client.keytab.file";
51+
public final static String KEY_HBASE_CLIENT_KERBEROS_PRINCIPAL = "hbase.client.kerberos.principal";
5652

5753

5854
public static final String KEY_JAVA_SECURITY_KRB5_CONF = "java.security.krb5.conf";
5955
public static final String KEY_ZOOKEEPER_SASL_CLIENT = "zookeeper.sasl.client";
6056

6157
public static UserGroupInformation loginAndReturnUGI(Configuration conf, String principal, String keytab) throws IOException {
58+
LOG.info("loginAndReturnUGI principal {}",principal);
59+
LOG.info("loginAndReturnUGI keytab {}",keytab);
6260
if (conf == null) {
6361
throw new IllegalArgumentException("kerberos conf can not be null");
6462
}

hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java

Lines changed: 86 additions & 46 deletions
Original file line numberDiff line numberDiff line change
@@ -29,8 +29,7 @@
2929
import org.apache.flink.configuration.Configuration;
3030
import org.apache.flink.types.Row;
3131
import org.apache.flink.util.Preconditions;
32-
import org.apache.hadoop.hbase.HBaseConfiguration;
33-
import org.apache.hadoop.hbase.TableName;
32+
import org.apache.hadoop.hbase.*;
3433
import org.apache.hadoop.hbase.client.Connection;
3534
import org.apache.hadoop.hbase.client.ConnectionFactory;
3635
import org.apache.hadoop.hbase.client.Delete;
@@ -70,6 +69,8 @@ public class HbaseOutputFormat extends AbstractDtRichOutputFormat<Tuple2> {
7069
private String regionserverPrincipal;
7170
private String securityKrb5Conf;
7271
private String zookeeperSaslClient;
72+
private String clientPrincipal;
73+
private String clientKeytabFile;
7374

7475
private String[] families;
7576
private String[] qualifiers;
@@ -78,48 +79,78 @@ public class HbaseOutputFormat extends AbstractDtRichOutputFormat<Tuple2> {
7879
private transient Connection conn;
7980
private transient Table table;
8081

82+
private transient ChoreService choreService;
83+
8184
@Override
8285
public void configure(Configuration parameters) {
8386
LOG.warn("---configure---");
8487
conf = HBaseConfiguration.create();
85-
conf.set("hbase.zookeeper.quorum", host);
86-
if (zkParent != null && !"".equals(zkParent)) {
87-
conf.set("zookeeper.znode.parent", zkParent);
88-
}
89-
LOG.warn("---configure end ---");
9088
}
9189

9290
@Override
9391
public void open(int taskNumber, int numTasks) throws IOException {
9492
LOG.warn("---open---");
95-
if (kerberosAuthEnable) {
96-
conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_QUORUM, host);
97-
conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM, zkParent);
98-
fillSyncKerberosConfig(conf, regionserverKeytabFile, regionserverPrincipal, zookeeperSaslClient, securityKrb5Conf);
99-
100-
UserGroupInformation userGroupInformation = HbaseConfigUtils.loginAndReturnUGI(conf, regionserverPrincipal, regionserverKeytabFile);
101-
org.apache.hadoop.conf.Configuration finalConf = conf;
102-
conn = userGroupInformation.doAs(new PrivilegedAction<Connection>() {
103-
@Override
104-
public Connection run() {
105-
try {
106-
return ConnectionFactory.createConnection(finalConf);
107-
} catch (IOException e) {
108-
LOG.error("Get connection fail with config:{}", finalConf);
109-
throw new RuntimeException(e);
110-
}
111-
}
112-
});
113-
} else {
114-
conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_QUORUM, host);
115-
conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM, zkParent);
116-
conn = ConnectionFactory.createConnection(conf);
117-
}
93+
openConn();
11894
table = conn.getTable(TableName.valueOf(tableName));
11995
LOG.warn("---open end(get table from hbase) ---");
12096
initMetric();
12197
}
12298

99+
private void openConn(){
100+
try{
101+
if (kerberosAuthEnable) {
102+
LOG.info("open kerberos conn");
103+
openKerberosConn();
104+
} else {
105+
LOG.info("open conn");
106+
conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_QUORUM, host);
107+
conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM, zkParent);
108+
conn = ConnectionFactory.createConnection(conf);
109+
}
110+
}catch (Exception e){
111+
throw new RuntimeException(e);
112+
}
113+
114+
}
115+
private void openKerberosConn() throws IOException {
116+
conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_QUORUM, host);
117+
conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM, zkParent);
118+
119+
LOG.info("kerberos config:{}", this.toString());
120+
Preconditions.checkArgument(!StringUtils.isEmpty(clientPrincipal), " clientPrincipal not null!");
121+
Preconditions.checkArgument(!StringUtils.isEmpty(clientKeytabFile), " clientKeytabFile not null!");
122+
123+
fillSyncKerberosConfig(conf, regionserverPrincipal, zookeeperSaslClient, securityKrb5Conf);
124+
125+
clientKeytabFile = System.getProperty("user.dir") + File.separator + clientKeytabFile;
126+
clientPrincipal = !StringUtils.isEmpty(clientPrincipal) ? clientPrincipal : regionserverPrincipal;
127+
128+
conf.set(HbaseConfigUtils.KEY_HBASE_CLIENT_KEYTAB_FILE, clientKeytabFile);
129+
conf.set(HbaseConfigUtils.KEY_HBASE_CLIENT_KERBEROS_PRINCIPAL, clientPrincipal);
130+
131+
UserGroupInformation userGroupInformation = HbaseConfigUtils.loginAndReturnUGI(conf, clientPrincipal, clientKeytabFile);
132+
org.apache.hadoop.conf.Configuration finalConf = conf;
133+
conn = userGroupInformation.doAs(new PrivilegedAction<Connection>() {
134+
@Override
135+
public Connection run() {
136+
try {
137+
ScheduledChore authChore = AuthUtil.getAuthChore(finalConf);
138+
if (authChore != null) {
139+
choreService = new ChoreService("hbaseKerberosSink");
140+
choreService.scheduleChore(authChore);
141+
}
142+
143+
return ConnectionFactory.createConnection(finalConf);
144+
} catch (IOException e) {
145+
LOG.error("Get connection fail with config:{}", finalConf);
146+
throw new RuntimeException(e);
147+
}
148+
}
149+
});
150+
}
151+
152+
153+
123154
@Override
124155
public void writeRecord(Tuple2 tuple2) {
125156
Tuple2<Boolean, Row> tupleTrans = tuple2;
@@ -227,7 +258,6 @@ public void close() throws IOException {
227258
conn = null;
228259
}
229260
}
230-
231261
private HbaseOutputFormat() {
232262
}
233263

@@ -264,11 +294,6 @@ public HbaseOutputFormatBuilder setRowkey(String rowkey) {
264294
return this;
265295
}
266296

267-
public HbaseOutputFormatBuilder setUpdateMode(String updateMode) {
268-
format.updateMode = updateMode;
269-
return this;
270-
}
271-
272297
public HbaseOutputFormatBuilder setColumnNames(String[] columnNames) {
273298
format.columnNames = columnNames;
274299
return this;
@@ -283,6 +308,7 @@ public HbaseOutputFormatBuilder setColumnNameFamily(Map<String, String> columnNa
283308
format.columnNameFamily = columnNameFamily;
284309
return this;
285310
}
311+
286312
public HbaseOutputFormatBuilder setKerberosAuthEnable(boolean kerberosAuthEnable) {
287313
format.kerberosAuthEnable = kerberosAuthEnable;
288314
return this;
@@ -308,6 +334,16 @@ public HbaseOutputFormatBuilder setZookeeperSaslClient(String zookeeperSaslClien
308334
return this;
309335
}
310336

337+
public HbaseOutputFormatBuilder setClientPrincipal(String clientPrincipal) {
338+
format.clientPrincipal = clientPrincipal;
339+
return this;
340+
}
341+
342+
public HbaseOutputFormatBuilder setClientKeytabFile(String clientKeytabFile) {
343+
format.clientKeytabFile = clientKeytabFile;
344+
return this;
345+
}
346+
311347

312348
public HbaseOutputFormat finish() {
313349
Preconditions.checkNotNull(format.host, "zookeeperQuorum should be specified");
@@ -323,7 +359,7 @@ public HbaseOutputFormat finish() {
323359
String[] columns = keySet.toArray(new String[keySet.size()]);
324360
for (int i = 0; i < columns.length; ++i) {
325361
String col = columns[i];
326-
String[] part = StringUtils.split(col, ":");;
362+
String[] part = col.split(":");
327363
families[i] = part[0];
328364
qualifiers[i] = part[1];
329365
}
@@ -336,16 +372,8 @@ public HbaseOutputFormat finish() {
336372

337373
}
338374

339-
private void fillSyncKerberosConfig( org.apache.hadoop.conf.Configuration config, String regionserverKeytabFile, String regionserverPrincipal,
375+
private void fillSyncKerberosConfig(org.apache.hadoop.conf.Configuration config, String regionserverPrincipal,
340376
String zookeeperSaslClient, String securityKrb5Conf) throws IOException {
341-
if (StringUtils.isEmpty(regionserverKeytabFile)) {
342-
throw new IllegalArgumentException("Must provide regionserverKeytabFile when authentication is Kerberos");
343-
}
344-
String regionserverKeytabFilePath = System.getProperty("user.dir") + File.separator + regionserverKeytabFile;
345-
LOG.info("regionserverKeytabFilePath:{}",regionserverKeytabFilePath);
346-
config.set(HbaseConfigUtils.KEY_HBASE_MASTER_KEYTAB_FILE, regionserverKeytabFilePath);
347-
config.set(HbaseConfigUtils.KEY_HBASE_REGIONSERVER_KEYTAB_FILE, regionserverKeytabFilePath);
348-
349377
if (StringUtils.isEmpty(regionserverPrincipal)) {
350378
throw new IllegalArgumentException("Must provide regionserverPrincipal when authentication is Kerberos");
351379
}
@@ -366,5 +394,17 @@ private void fillSyncKerberosConfig( org.apache.hadoop.conf.Configuration config
366394
}
367395
}
368396

397+
@Override
398+
public String toString() {
399+
return "HbaseOutputFormat kerberos{" +
400+
"kerberosAuthEnable=" + kerberosAuthEnable +
401+
", regionserverKeytabFile='" + regionserverKeytabFile + '\'' +
402+
", regionserverPrincipal='" + regionserverPrincipal + '\'' +
403+
", securityKrb5Conf='" + securityKrb5Conf + '\'' +
404+
", zookeeperSaslClient='" + zookeeperSaslClient + '\'' +
405+
", clientPrincipal='" + clientPrincipal + '\'' +
406+
", clientKeytabFile='" + clientKeytabFile + '\'' +
407+
'}';
408+
}
369409

370410
}

hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -59,6 +59,9 @@ public class HbaseSink implements RetractStreamTableSink<Row>, IStreamSinkGener<
5959
protected String regionserverPrincipal;
6060
protected String securityKrb5Conf;
6161
protected String zookeeperSaslClient;
62+
63+
private String clientPrincipal;
64+
private String clientKeytabFile;
6265
private int parallelism = -1;
6366

6467

@@ -83,6 +86,9 @@ public HbaseSink genStreamSink(AbstractTargetTableInfo targetTableInfo) {
8386
this.securityKrb5Conf = hbaseTableInfo.getSecurityKrb5Conf();
8487
this.zookeeperSaslClient = hbaseTableInfo.getZookeeperSaslClient();
8588

89+
this.clientKeytabFile = hbaseTableInfo.getClientKeytabFile();
90+
this.clientPrincipal = hbaseTableInfo.getClientPrincipal();
91+
8692
Integer tmpSinkParallelism = hbaseTableInfo.getParallelism();
8793
if (tmpSinkParallelism != null) {
8894
this.parallelism = tmpSinkParallelism;
@@ -104,6 +110,9 @@ public void emitDataStream(DataStream<Tuple2<Boolean, Row>> dataStream) {
104110
builder.setSecurityKrb5Conf(securityKrb5Conf);
105111
builder.setZookeeperSaslClient(zookeeperSaslClient);
106112

113+
builder.setClientPrincipal(clientPrincipal);
114+
builder.setClientKeytabFile(clientKeytabFile);
115+
107116
HbaseOutputFormat outputFormat = builder.finish();
108117
RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(outputFormat);
109118
dataStream.addSink(richSinkFunction);

hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -61,6 +61,10 @@ public class HbaseSinkParser extends AbstractTableParser {
6161
public static final String SECURITY_KRB5_CONF_KEY = "securityKrb5Conf";
6262
public static final String ZOOKEEPER_SASL_CLINT_KEY = "zookeeperSaslClient";
6363

64+
public static final String CLIENT_PRINCIPAL_KEY = "clientPrincipal";
65+
public static final String CLIENT_KEYTABFILE_KEY = "clientKeytabFile";
66+
67+
6468
@Override
6569
protected boolean fieldNameNeedsUpperCase() {
6670
return false;
@@ -85,6 +89,9 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map<S
8589
hbaseTableInfo.setRegionserverPrincipal((String) props.get(REGIONSERVER_PRINCIPAL_KEY.toLowerCase()));
8690
hbaseTableInfo.setSecurityKrb5Conf((String) props.get(SECURITY_KRB5_CONF_KEY.toLowerCase()));
8791
hbaseTableInfo.setZookeeperSaslClient((String) props.get(ZOOKEEPER_SASL_CLINT_KEY.toLowerCase()));
92+
93+
hbaseTableInfo.setClientPrincipal((String) props.get(CLIENT_PRINCIPAL_KEY.toLowerCase()));
94+
hbaseTableInfo.setClientKeytabFile((String) props.get(CLIENT_KEYTABFILE_KEY.toLowerCase()));
8895
return hbaseTableInfo;
8996
}
9097

hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -66,6 +66,10 @@ public class HbaseTableInfo extends AbstractTargetTableInfo {
6666

6767
private String zookeeperSaslClient;
6868

69+
private String clientPrincipal;
70+
71+
private String clientKeytabFile;
72+
6973
private Map<String,Object> hbaseConfig = Maps.newHashMap();
7074

7175
public HbaseTableInfo(){
@@ -212,4 +216,20 @@ public void setZookeeperSaslClient(String zookeeperSaslClient) {
212216
this.zookeeperSaslClient = zookeeperSaslClient;
213217
}
214218

219+
public String getClientPrincipal() {
220+
return clientPrincipal;
221+
}
222+
223+
public void setClientPrincipal(String clientPrincipal) {
224+
this.clientPrincipal = clientPrincipal;
225+
}
226+
227+
public String getClientKeytabFile() {
228+
return clientKeytabFile;
229+
}
230+
231+
public void setClientKeytabFile(String clientKeytabFile) {
232+
this.clientKeytabFile = clientKeytabFile;
233+
}
234+
215235
}

0 commit comments

Comments
 (0)