Skip to content

Commit ef72fc1

Browse files
author
dapeng
committed
add hbase kerbos
1 parent 33047f6 commit ef72fc1

File tree

4 files changed

+272
-9
lines changed

4 files changed

+272
-9
lines changed

hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java

Lines changed: 77 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -25,8 +25,10 @@
2525
import com.dtstack.flink.sql.side.FieldInfo;
2626
import com.dtstack.flink.sql.side.JoinInfo;
2727
import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo;
28+
import com.dtstack.flink.sql.side.hbase.utils.HbaseConfigUtils;
2829
import org.apache.calcite.sql.JoinType;
2930
import org.apache.commons.collections.map.HashedMap;
31+
import org.apache.commons.lang.StringUtils;
3032
import org.apache.flink.api.java.typeutils.RowTypeInfo;
3133
import com.google.common.collect.Maps;
3234
import org.apache.flink.table.runtime.types.CRow;
@@ -36,6 +38,7 @@
3638
import org.apache.hadoop.conf.Configuration;
3739
import org.apache.hadoop.hbase.Cell;
3840
import org.apache.hadoop.hbase.CellUtil;
41+
import org.apache.hadoop.hbase.HBaseConfiguration;
3942
import org.apache.hadoop.hbase.TableName;
4043
import org.apache.hadoop.hbase.client.Connection;
4144
import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -44,10 +47,13 @@
4447
import org.apache.hadoop.hbase.client.Scan;
4548
import org.apache.hadoop.hbase.client.Table;
4649
import org.apache.hadoop.hbase.util.Bytes;
50+
import org.apache.hadoop.security.UserGroupInformation;
4751
import org.slf4j.Logger;
4852
import org.slf4j.LoggerFactory;
4953

54+
import java.io.File;
5055
import java.io.IOException;
56+
import java.security.PrivilegedAction;
5157
import java.sql.SQLException;
5258
import java.sql.Timestamp;
5359
import java.util.Calendar;
@@ -166,13 +172,44 @@ public void flatMap(CRow input, Collector<CRow> out) throws Exception {
166172
private void loadData(Map<String, Map<String, Object>> tmpCache) throws SQLException {
167173
AbstractSideTableInfo sideTableInfo = sideInfo.getSideTableInfo();
168174
HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo;
169-
Configuration conf = new Configuration();
170-
conf.set("hbase.zookeeper.quorum", hbaseSideTableInfo.getHost());
171-
Connection conn = null;
172-
Table table = null;
173-
ResultScanner resultScanner = null;
175+
boolean openKerberos = hbaseSideTableInfo.isKerberosAuthEnable();
176+
int loadDataCount = 0;
174177
try {
175-
conn = ConnectionFactory.createConnection(conf);
178+
conf = HBaseConfiguration.create();
179+
if (openKerberos) {
180+
conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_QUORUM, hbaseSideTableInfo.getHost());
181+
conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM_SYNC, hbaseSideTableInfo.getParent());
182+
183+
fillSyncKerberosConfig(conf,hbaseSideTableInfo);
184+
LOG.info("hbase.security.authentication:{}", conf.get("hbase.security.authentication"));
185+
LOG.info("hbase.security.authorization:{}", conf.get("hbase.security.authorization"));
186+
LOG.info("hbase.master.keytab.file:{}", conf.get("hbase.master.keytab.file"));
187+
LOG.info("hbase.master.kerberos.principal:{}", conf.get("hbase.master.kerberos.principal"));
188+
LOG.info("hbase.regionserver.keytab.file:{}", conf.get("hbase.regionserver.keytab.file"));
189+
LOG.info("hbase.regionserver.kerberos.principal:{}", conf.get("hbase.regionserver.kerberos.principal"));
190+
191+
UserGroupInformation userGroupInformation = HbaseConfigUtils.loginAndReturnUGI(conf, hbaseSideTableInfo.getRegionserverPrincipal(),
192+
hbaseSideTableInfo.getRegionserverKeytabFile());
193+
194+
Configuration finalConf = conf;
195+
conn = userGroupInformation.doAs(new PrivilegedAction<Connection>() {
196+
@Override
197+
public Connection run() {
198+
try {
199+
return ConnectionFactory.createConnection(finalConf);
200+
} catch (IOException e) {
201+
LOG.error("Get connection fail with config:{}", finalConf);
202+
throw new RuntimeException(e);
203+
}
204+
}
205+
});
206+
207+
} else {
208+
conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_QUORUM, hbaseSideTableInfo.getHost());
209+
conf.set(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM_SYNC, hbaseSideTableInfo.getParent());
210+
conn = ConnectionFactory.createConnection(conf);
211+
}
212+
176213
table = conn.getTable(TableName.valueOf(tableName));
177214
resultScanner = table.getScanner(new Scan());
178215
for (Result r : resultScanner) {
@@ -187,13 +224,15 @@ private void loadData(Map<String, Map<String, Object>> tmpCache) throws SQLExcep
187224

188225
kv.put(aliasNameInversion.get(key.toString()), value);
189226
}
227+
loadDataCount++;
190228
tmpCache.put(new String(r.getRow()), kv);
191229
}
192230
} catch (IOException e) {
193-
LOG.error("", e);
231+
throw new RuntimeException(e);
194232
} finally {
233+
LOG.info("load Data count: {}", loadDataCount);
195234
try {
196-
if (null != conn && !conn.isClosed()) {
235+
if (null != conn) {
197236
conn.close();
198237
}
199238

@@ -209,4 +248,34 @@ private void loadData(Map<String, Map<String, Object>> tmpCache) throws SQLExcep
209248
}
210249
}
211250
}
251+
252+
private void fillSyncKerberosConfig(Configuration config, HbaseSideTableInfo hbaseSideTableInfo) throws IOException {
253+
String regionserverKeytabFile = hbaseSideTableInfo.getRegionserverKeytabFile();
254+
if (StringUtils.isEmpty(regionserverKeytabFile)) {
255+
throw new IllegalArgumentException("Must provide regionserverKeytabFile when authentication is Kerberos");
256+
}
257+
String regionserverKeytabFilePath = System.getProperty("user.dir") + File.separator + regionserverKeytabFile;
258+
LOG.info("regionserverKeytabFilePath:{}", regionserverKeytabFilePath);
259+
config.set(HbaseConfigUtils.KEY_HBASE_MASTER_KEYTAB_FILE, regionserverKeytabFilePath);
260+
config.set(HbaseConfigUtils.KEY_HBASE_REGIONSERVER_KEYTAB_FILE, regionserverKeytabFilePath);
261+
262+
String regionserverPrincipal = hbaseSideTableInfo.getRegionserverPrincipal();
263+
if (StringUtils.isEmpty(regionserverPrincipal)) {
264+
throw new IllegalArgumentException("Must provide regionserverPrincipal when authentication is Kerberos");
265+
}
266+
config.set(HbaseConfigUtils.KEY_HBASE_MASTER_KERBEROS_PRINCIPAL, regionserverPrincipal);
267+
config.set(HbaseConfigUtils.KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL, regionserverPrincipal);
268+
config.set(HbaseConfigUtils.KEY_HBASE_SECURITY_AUTHORIZATION, "true");
269+
config.set(HbaseConfigUtils.KEY_HBASE_SECURITY_AUTHENTICATION, "kerberos");
270+
271+
if (!StringUtils.isEmpty(hbaseSideTableInfo.getZookeeperSaslClient())) {
272+
System.setProperty(HbaseConfigUtils.KEY_ZOOKEEPER_SASL_CLIENT, hbaseSideTableInfo.getZookeeperSaslClient());
273+
}
274+
275+
if (!StringUtils.isEmpty(hbaseSideTableInfo.getSecurityKrb5Conf())) {
276+
String krb5ConfPath = System.getProperty("user.dir") + File.separator + hbaseSideTableInfo.getSecurityKrb5Conf();
277+
LOG.info("krb5ConfPath:{}", krb5ConfPath);
278+
System.setProperty(HbaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF, krb5ConfPath);
279+
}
280+
}
212281
}

hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java

Lines changed: 39 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import com.dtstack.flink.sql.side.hbase.rowkeydealer.RowKeyEqualModeDealer;
3232
import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo;
3333
import com.dtstack.flink.sql.factory.DTThreadFactory;
34+
import com.dtstack.flink.sql.side.hbase.utils.HbaseConfigUtils;
3435
import com.google.common.collect.Maps;
3536
import com.stumbleupon.async.Deferred;
3637
import org.apache.commons.lang3.StringUtils;
@@ -40,10 +41,13 @@
4041
import org.apache.flink.table.runtime.types.CRow;
4142
import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo;
4243
import org.apache.flink.types.Row;
44+
import org.hbase.async.Config;
4345
import org.hbase.async.HBaseClient;
4446
import org.slf4j.Logger;
4547
import org.slf4j.LoggerFactory;
4648

49+
import java.io.File;
50+
import java.io.IOException;
4751
import java.sql.Timestamp;
4852
import java.util.Collections;
4953
import java.util.List;
@@ -93,11 +97,19 @@ public void open(Configuration parameters) throws Exception {
9397
super.open(parameters);
9498
AbstractSideTableInfo sideTableInfo = sideInfo.getSideTableInfo();
9599
HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo;
100+
96101
ExecutorService executorService =new ThreadPoolExecutor(DEFAULT_POOL_SIZE, DEFAULT_POOL_SIZE,
97102
0L, TimeUnit.MILLISECONDS,
98103
new LinkedBlockingQueue<>(), new DTThreadFactory("hbase-aysnc"));
99104

100-
hBaseClient = new HBaseClient(hbaseSideTableInfo.getHost(), hbaseSideTableInfo.getParent(), executorService);
105+
Config config = new Config();
106+
config.overrideConfig(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_QUORUM, hbaseSideTableInfo.getHost());
107+
config.overrideConfig(HbaseConfigUtils.KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM_ASYNC, hbaseSideTableInfo.getParent());
108+
109+
if (hbaseSideTableInfo.isKerberosAuthEnable()) {
110+
fillAsyncKerberosConfig(config, hbaseSideTableInfo);
111+
}
112+
hBaseClient = new HBaseClient(config, executorService);
101113

102114
try {
103115
Deferred deferred = hBaseClient.ensureTableExists(tableName)
@@ -166,6 +178,32 @@ public void close() throws Exception {
166178
hBaseClient.shutdown();
167179
}
168180

181+
private void fillAsyncKerberosConfig(Config config, HbaseSideTableInfo hbaseSideTableInfo) throws IOException {
182+
AuthUtil.JAASConfig jaasConfig = HbaseConfigUtils.buildJaasConfig(hbaseSideTableInfo);
183+
LOG.info("jaasConfig file:\n {}", jaasConfig.toString());
184+
String jaasFilePath = AuthUtil.creatJaasFile("JAAS", ".conf", jaasConfig);
185+
config.overrideConfig(HbaseConfigUtils.KEY_JAVA_SECURITY_AUTH_LOGIN_CONF, jaasFilePath);
186+
config.overrideConfig(HbaseConfigUtils.KEY_HBASE_SECURITY_AUTH_ENABLE, "true");
187+
config.overrideConfig(HbaseConfigUtils.KEY_HBASE_SASL_CLIENTCONFIG, "Client");
188+
config.overrideConfig(HbaseConfigUtils.KEY_HBASE_SECURITY_AUTHENTICATION, "kerberos");
189+
190+
String regionserverPrincipal = hbaseSideTableInfo.getRegionserverPrincipal();
191+
if (StringUtils.isEmpty(regionserverPrincipal)) {
192+
throw new IllegalArgumentException("Must provide regionserverPrincipal when authentication is Kerberos");
193+
}
194+
config.overrideConfig(HbaseConfigUtils.KEY_HBASE_KERBEROS_REGIONSERVER_PRINCIPAL, regionserverPrincipal);
195+
196+
if (!StringUtils.isEmpty(hbaseSideTableInfo.getZookeeperSaslClient())) {
197+
System.setProperty(HbaseConfigUtils.KEY_ZOOKEEPER_SASL_CLIENT, hbaseSideTableInfo.getZookeeperSaslClient());
198+
}
199+
200+
if (!StringUtils.isEmpty(hbaseSideTableInfo.getSecurityKrb5Conf())) {
201+
String krb5ConfPath = System.getProperty("user.dir") + File.separator + hbaseSideTableInfo.getSecurityKrb5Conf();
202+
LOG.info("krb5ConfPath:{}", krb5ConfPath);
203+
System.setProperty(HbaseConfigUtils.KEY_JAVA_SECURITY_KRB5_CONF, krb5ConfPath);
204+
}
205+
}
206+
169207

170208
class CheckResult{
171209

hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java

Lines changed: 48 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -148,6 +148,54 @@ public void setPreRowKey(boolean preRowKey) {
148148
this.preRowKey = preRowKey;
149149
}
150150

151+
public boolean isKerberosAuthEnable() {
152+
return kerberosAuthEnable;
153+
}
154+
155+
public void setKerberosAuthEnable(boolean kerberosAuthEnable) {
156+
this.kerberosAuthEnable = kerberosAuthEnable;
157+
}
158+
159+
public String getRegionserverKeytabFile() {
160+
return regionserverKeytabFile;
161+
}
162+
163+
public void setRegionserverKeytabFile(String regionserverKeytabFile) {
164+
this.regionserverKeytabFile = regionserverKeytabFile;
165+
}
166+
167+
public String getRegionserverPrincipal() {
168+
return regionserverPrincipal;
169+
}
170+
171+
public void setRegionserverPrincipal(String regionserverPrincipal) {
172+
this.regionserverPrincipal = regionserverPrincipal;
173+
}
174+
175+
public String getJaasPrincipal() {
176+
return jaasPrincipal;
177+
}
178+
179+
public void setJaasPrincipal(String jaasPrincipal) {
180+
this.jaasPrincipal = jaasPrincipal;
181+
}
182+
183+
public String getSecurityKrb5Conf() {
184+
return securityKrb5Conf;
185+
}
186+
187+
public void setSecurityKrb5Conf(String securityKrb5Conf) {
188+
this.securityKrb5Conf = securityKrb5Conf;
189+
}
190+
191+
public String getZookeeperSaslClient() {
192+
return zookeeperSaslClient;
193+
}
194+
195+
public void setZookeeperSaslClient(String zookeeperSaslClient) {
196+
this.zookeeperSaslClient = zookeeperSaslClient;
197+
}
198+
151199
@Override
152200
public void finish(){
153201
super.finish();
Lines changed: 108 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,108 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package com.dtstack.flink.sql.side.hbase.utils;
20+
21+
import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo;
22+
import com.dtstack.flink.sql.util.AuthUtil;
23+
import org.apache.commons.collections.MapUtils;
24+
import org.apache.commons.lang3.StringUtils;
25+
import org.apache.hadoop.conf.Configuration;
26+
import org.apache.hadoop.hbase.HBaseConfiguration;
27+
import org.apache.hadoop.security.UserGroupInformation;
28+
import org.slf4j.Logger;
29+
import org.slf4j.LoggerFactory;
30+
31+
import java.io.BufferedWriter;
32+
import java.io.File;
33+
import java.io.FileWriter;
34+
import java.io.IOException;
35+
import java.util.Arrays;
36+
import java.util.HashMap;
37+
import java.util.List;
38+
import java.util.Map;
39+
import java.util.UUID;
40+
41+
/**
42+
*
43+
* The utility class of HBase connection
44+
*
45+
* Date: 2019/12/24
46+
* Company: www.dtstack.com
47+
* @author maqi
48+
*/
49+
public class HbaseConfigUtils {
50+
51+
private static final Logger LOG = LoggerFactory.getLogger(HbaseConfigUtils.class);
52+
// sync side kerberos
53+
public final static String KEY_HBASE_SECURITY_AUTHENTICATION = "hbase.security.authentication";
54+
public final static String KEY_HBASE_SECURITY_AUTHORIZATION = "hbase.security.authorization";
55+
public final static String KEY_HBASE_MASTER_KEYTAB_FILE = "hbase.master.keytab.file";
56+
public final static String KEY_HBASE_MASTER_KERBEROS_PRINCIPAL = "hbase.master.kerberos.principal";
57+
public final static String KEY_HBASE_REGIONSERVER_KEYTAB_FILE = "hbase.regionserver.keytab.file";
58+
public final static String KEY_HBASE_REGIONSERVER_KERBEROS_PRINCIPAL = "hbase.regionserver.kerberos.principal";
59+
60+
// async side kerberos
61+
public final static String KEY_HBASE_SECURITY_AUTH_ENABLE = "hbase.security.auth.enable";
62+
public final static String KEY_HBASE_SASL_CLIENTCONFIG = "hbase.sasl.clientconfig";
63+
public final static String KEY_HBASE_KERBEROS_REGIONSERVER_PRINCIPAL = "hbase.kerberos.regionserver.principal";
64+
65+
public final static String KEY_HBASE_ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum";
66+
public final static String KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM_SYNC = "zookeeper.znode.parent";
67+
public final static String KEY_HBASE_ZOOKEEPER_ZNODE_QUORUM_ASYNC = "hbase.zookeeper.znode.parent";
68+
69+
70+
public static final String KEY_JAVA_SECURITY_KRB5_CONF = "java.security.krb5.conf";
71+
public static final String KEY_ZOOKEEPER_SASL_CLIENT = "zookeeper.sasl.client";
72+
73+
public static final String KEY_JAVA_SECURITY_AUTH_LOGIN_CONF = "java.security.auth.login.config";
74+
75+
76+
public static AuthUtil.JAASConfig buildJaasConfig(HbaseSideTableInfo hbaseSideTableInfo) {
77+
String keytabPath = System.getProperty("user.dir") + File.separator + hbaseSideTableInfo.getRegionserverKeytabFile();
78+
Map<String, String> loginModuleOptions = new HashMap<>();
79+
loginModuleOptions.put("useKeyTab", "true");
80+
loginModuleOptions.put("useTicketCache", "false");
81+
loginModuleOptions.put("keyTab", "\"" + keytabPath + "\"");
82+
loginModuleOptions.put("principal", "\"" + hbaseSideTableInfo.getJaasPrincipal() + "\"");
83+
return AuthUtil.JAASConfig.builder().setEntryName("Client")
84+
.setLoginModule("com.sun.security.auth.module.Krb5LoginModule")
85+
.setLoginModuleFlag("required").setLoginModuleOptions(loginModuleOptions).build();
86+
}
87+
88+
89+
public static UserGroupInformation loginAndReturnUGI(Configuration conf, String principal, String keytab) throws IOException {
90+
if (conf == null) {
91+
throw new IllegalArgumentException("kerberos conf can not be null");
92+
}
93+
94+
if (org.apache.commons.lang.StringUtils.isEmpty(principal)) {
95+
throw new IllegalArgumentException("principal can not be null");
96+
}
97+
98+
if (org.apache.commons.lang.StringUtils.isEmpty(keytab)) {
99+
throw new IllegalArgumentException("keytab can not be null");
100+
}
101+
102+
conf.set("hadoop.security.authentication", "Kerberos");
103+
UserGroupInformation.setConfiguration(conf);
104+
105+
return UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab);
106+
}
107+
108+
}

0 commit comments

Comments
 (0)