Skip to content

Commit 3761c86

Browse files
author
dujie
committed
hotfix【30257】简化对kerberos条件判断,认证之前刷新keytab
1 parent 049440a commit 3761c86

File tree

2 files changed

+39
-3
lines changed

2 files changed

+39
-3
lines changed

flinkx-core/src/main/java/com/dtstack/flinkx/authenticate/KerberosUtil.java

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import org.apache.commons.lang.StringUtils;
2626
import org.apache.hadoop.conf.Configuration;
2727
import org.apache.hadoop.security.UserGroupInformation;
28+
import org.apache.hadoop.security.authentication.util.KerberosName;
2829
import org.slf4j.Logger;
2930
import org.slf4j.LoggerFactory;
3031
import sun.security.krb5.Config;
@@ -33,6 +34,7 @@
3334

3435
import java.io.File;
3536
import java.io.IOException;
37+
import java.lang.reflect.Field;
3638
import java.util.Map;
3739

3840
/**
@@ -122,6 +124,7 @@ public static void loadKrb5Conf(Map<String, Object> kerberosConfig){
122124

123125
krb5FilePath = loadFile(kerberosConfig, krb5FilePath);
124126
kerberosConfig.put(KEY_JAVA_SECURITY_KRB5_CONF, krb5FilePath);
127+
System.setProperty(KEY_JAVA_SECURITY_KRB5_CONF, krb5FilePath);
125128
}
126129

127130
/**
@@ -249,4 +252,21 @@ public static String getPrincipalFileName(Map<String, Object> config) {
249252

250253
return fileName;
251254
}
255+
256+
/**
257+
* 刷新krb内容信息
258+
*/
259+
public static void refreshConfig() {
260+
try{
261+
sun.security.krb5.Config.refresh();
262+
Field defaultRealmField = KerberosName.class.getDeclaredField("defaultRealm");
263+
defaultRealmField.setAccessible(true);
264+
defaultRealmField.set(null, org.apache.hadoop.security.authentication.util.KerberosUtil.getDefaultRealm());
265+
//reload java.security.auth.login.config
266+
javax.security.auth.login.Configuration.setConfiguration(null);
267+
}catch (Exception e){
268+
LOG.warn("resetting default realm failed, current default realm will still be used.", e);
269+
}
270+
271+
}
252272
}

flinkx-hbase/flinkx-hbase-core/src/main/java/com/dtstack/flinkx/hbase/HbaseHelper.java

Lines changed: 19 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -53,6 +53,7 @@ public class HbaseHelper {
5353
private final static String AUTHENTICATION_TYPE = "Kerberos";
5454
private final static String KEY_HBASE_SECURITY_AUTHENTICATION = "hbase.security.authentication";
5555
private final static String KEY_HBASE_SECURITY_AUTHORIZATION = "hbase.security.authorization";
56+
private final static String KEY_HBASE_SECURITY_AUTH_ENABLE = "hbase.security.auth.enable";
5657

5758
public static org.apache.hadoop.hbase.client.Connection getHbaseConnection(Map<String,Object> hbaseConfigMap) {
5859
Validate.isTrue(MapUtils.isNotEmpty(hbaseConfigMap), "hbaseConfig不能为空Map结构!");
@@ -72,6 +73,7 @@ public static org.apache.hadoop.hbase.client.Connection getHbaseConnection(Map<S
7273

7374
private static org.apache.hadoop.hbase.client.Connection getConnectionWithKerberos(Map<String,Object> hbaseConfigMap){
7475
try {
76+
setKerberosConf(hbaseConfigMap);
7577
UserGroupInformation ugi = getUgi(hbaseConfigMap);
7678
return ugi.doAs(new PrivilegedAction<Connection>() {
7779
@Override
@@ -118,11 +120,25 @@ public static Configuration getConfig(Map<String,Object> hbaseConfigMap){
118120
}
119121

120122
public static boolean openKerberos(Map<String,Object> hbaseConfigMap){
121-
if(!MapUtils.getBooleanValue(hbaseConfigMap, KEY_HBASE_SECURITY_AUTHORIZATION)){
122-
return false;
123+
if(AUTHENTICATION_TYPE.equalsIgnoreCase(MapUtils.getString(hbaseConfigMap, KEY_HBASE_SECURITY_AUTHORIZATION))
124+
|| AUTHENTICATION_TYPE.equalsIgnoreCase(MapUtils.getString(hbaseConfigMap, KEY_HBASE_SECURITY_AUTHENTICATION))
125+
|| MapUtils.getBooleanValue(hbaseConfigMap, KEY_HBASE_SECURITY_AUTH_ENABLE)){
126+
LOG.info("open kerberos for hbase.");
127+
return true;
123128
}
124129

125-
return AUTHENTICATION_TYPE.equalsIgnoreCase(MapUtils.getString(hbaseConfigMap, KEY_HBASE_SECURITY_AUTHENTICATION));
130+
return false;
131+
}
132+
133+
134+
/**
135+
* 设置hbase 开启kerberos 连接必要的固定参数
136+
* @param hbaseConfigMap
137+
*/
138+
public static void setKerberosConf(Map<String,Object> hbaseConfigMap){
139+
hbaseConfigMap.put(KEY_HBASE_SECURITY_AUTHORIZATION, AUTHENTICATION_TYPE);
140+
hbaseConfigMap.put(KEY_HBASE_SECURITY_AUTHENTICATION, AUTHENTICATION_TYPE);
141+
hbaseConfigMap.put(KEY_HBASE_SECURITY_AUTH_ENABLE, true);
126142
}
127143

128144
public static RegionLocator getRegionLocator(Connection hConnection, String userTable){

0 commit comments

Comments
 (0)