30
30
import com .google .common .collect .Maps ;
31
31
import org .apache .calcite .sql .JoinType ;
32
32
import org .apache .commons .collections .CollectionUtils ;
33
- import org .apache .commons .lang3 .StringUtils ;
34
33
import org .apache .commons .pool2 .impl .GenericObjectPoolConfig ;
35
34
import org .apache .flink .api .java .typeutils .RowTypeInfo ;
36
35
import org .apache .flink .table .dataformat .BaseRow ;
48
47
import java .io .Closeable ;
49
48
import java .io .IOException ;
50
49
import java .sql .SQLException ;
50
+ import java .util .Arrays ;
51
51
import java .util .Calendar ;
52
52
import java .util .HashSet ;
53
53
import java .util .List ;
54
54
import java .util .Map ;
55
+ import java .util .Objects ;
55
56
import java .util .Set ;
56
57
import java .util .TreeSet ;
57
58
import java .util .concurrent .atomic .AtomicReference ;
59
+ import java .util .regex .Matcher ;
60
+ import java .util .regex .Pattern ;
61
+
58
62
/**
59
63
* @author yanxi
60
64
*/
61
65
public class RedisAllReqRow extends BaseAllReqRow {
62
66
63
67
private static final long serialVersionUID = 7578879189085344807L ;
64
68
69
+ private static final Pattern HOST_PORT_PATTERN = Pattern .compile ("(?<host>(.*)):(?<port>\\ d+)*" );
70
+
65
71
private static final Logger LOG = LoggerFactory .getLogger (RedisAllReqRow .class );
66
72
67
73
private static final int CONN_RETRY_NUM = 3 ;
@@ -72,9 +78,9 @@ public class RedisAllReqRow extends BaseAllReqRow {
72
78
73
79
private RedisSideTableInfo tableInfo ;
74
80
75
- private AtomicReference <Map <String , Map <String , String >>> cacheRef = new AtomicReference <>();
81
+ private final AtomicReference <Map <String , Map <String , String >>> cacheRef = new AtomicReference <>();
76
82
77
- private RedisSideReqRow redisSideReqRow ;
83
+ private final RedisSideReqRow redisSideReqRow ;
78
84
79
85
public RedisAllReqRow (RowTypeInfo rowTypeInfo , JoinInfo joinInfo , List <FieldInfo > outFieldInfoList , AbstractSideTableInfo sideTableInfo ) {
80
86
super (new RedisAllSideInfo (rowTypeInfo , joinInfo , outFieldInfoList , sideTableInfo ));
@@ -112,9 +118,9 @@ protected void reloadCache() {
112
118
public void flatMap (BaseRow input , Collector <BaseRow > out ) throws Exception {
113
119
GenericRow genericRow = (GenericRow ) input ;
114
120
Map <String , Object > inputParams = Maps .newHashMap ();
115
- for (Integer conValIndex : sideInfo .getEqualValIndex ()){
121
+ for (Integer conValIndex : sideInfo .getEqualValIndex ()) {
116
122
Object equalObj = genericRow .getField (conValIndex );
117
- if (equalObj == null ){
123
+ if (equalObj == null ) {
118
124
if (sideInfo .getJoinType () == JoinType .LEFT ) {
119
125
BaseRow data = fillData (input , null );
120
126
RowDataComplete .collectBaseRow (out , data );
@@ -128,11 +134,11 @@ public void flatMap(BaseRow input, Collector<BaseRow> out) throws Exception {
128
134
129
135
Map <String , String > cacheMap = cacheRef .get ().get (key );
130
136
131
- if (cacheMap == null ){
132
- if (sideInfo .getJoinType () == JoinType .LEFT ){
137
+ if (cacheMap == null ) {
138
+ if (sideInfo .getJoinType () == JoinType .LEFT ) {
133
139
BaseRow data = fillData (input , null );
134
140
RowDataComplete .collectBaseRow (out , data );
135
- }else {
141
+ } else {
136
142
return ;
137
143
}
138
144
@@ -147,7 +153,7 @@ private void loadData(Map<String, Map<String, String>> tmpCache) throws SQLExcep
147
153
JedisCommands jedis = null ;
148
154
try {
149
155
StringBuilder keyPattern = new StringBuilder (tableInfo .getTableName ());
150
- for (String key : tableInfo .getPrimaryKeys ()) {
156
+ for (int i = 0 ; i < tableInfo .getPrimaryKeys (). size (); i ++ ) {
151
157
keyPattern .append ("_" ).append ("*" );
152
158
}
153
159
jedis = getJedisWithRetry (CONN_RETRY_NUM );
@@ -182,41 +188,54 @@ private JedisCommands getJedis(RedisSideTableInfo tableInfo) {
182
188
String url = tableInfo .getUrl ();
183
189
String password = tableInfo .getPassword ();
184
190
String database = tableInfo .getDatabase () == null ? "0" : tableInfo .getDatabase ();
191
+ String masterName = tableInfo .getMasterName ();
185
192
int timeout = tableInfo .getTimeout ();
186
- if (timeout == 0 ){
187
- timeout = 1000 ;
193
+ if (timeout == 0 ) {
194
+ timeout = 10000 ;
188
195
}
189
196
190
- String [] nodes = StringUtils .split (url , "," );
191
- String [] firstIpPort = StringUtils .split (nodes [0 ], ":" );
192
- String firstIp = firstIpPort [0 ];
193
- String firstPort = firstIpPort [1 ];
194
- Set <HostAndPort > addresses = new HashSet <>();
195
- Set <String > ipPorts = new HashSet <>();
196
- for (String ipPort : nodes ) {
197
- ipPorts .add (ipPort );
198
- String [] ipPortPair = ipPort .split (":" );
199
- addresses .add (new HostAndPort (ipPortPair [0 ].trim (), Integer .valueOf (ipPortPair [1 ].trim ())));
200
- }
201
- if (timeout == 0 ){
202
- timeout = 1000 ;
203
- }
197
+ String [] nodes = url .split ("," );
204
198
JedisCommands jedis = null ;
205
199
GenericObjectPoolConfig poolConfig = setPoolConfig (tableInfo .getMaxTotal (), tableInfo .getMaxIdle (), tableInfo .getMinIdle ());
206
- switch (RedisType .parse (tableInfo .getRedisType ())){
207
- //单机
200
+ switch (RedisType .parse (tableInfo .getRedisType ())) {
208
201
case STANDALONE :
209
- pool = new JedisPool (poolConfig , firstIp , Integer .parseInt (firstPort ), timeout , password , Integer .parseInt (database ));
210
- jedis = pool .getResource ();
202
+ String firstIp = null ;
203
+ String firstPort = null ;
204
+ Matcher standalone = HOST_PORT_PATTERN .matcher (nodes [0 ]);
205
+ if (standalone .find ()) {
206
+ firstIp = standalone .group ("host" ).trim ();
207
+ firstPort = standalone .group ("port" ).trim ();
208
+ }
209
+ if (Objects .nonNull (firstIp )) {
210
+ pool = new JedisPool (poolConfig , firstIp , Integer .parseInt (firstPort ), timeout , password , Integer .parseInt (database ));
211
+ jedis = pool .getResource ();
212
+ } else {
213
+ throw new IllegalArgumentException (
214
+ String .format ("redis url error. current url [%s]" , nodes [0 ]));
215
+ }
211
216
break ;
212
- //哨兵
213
217
case SENTINEL :
214
- jedisSentinelPool = new JedisSentinelPool (tableInfo .getMasterName (), ipPorts , poolConfig , timeout , password , Integer .parseInt (database ));
218
+ Set <String > ipPorts = new HashSet <>(Arrays .asList (nodes ));
219
+ jedisSentinelPool = new JedisSentinelPool (masterName , ipPorts , poolConfig , timeout , password , Integer .parseInt (database ));
215
220
jedis = jedisSentinelPool .getResource ();
216
221
break ;
217
- //集群
218
222
case CLUSTER :
219
- jedis = new JedisCluster (addresses , timeout , timeout ,1 , poolConfig );
223
+ Set <HostAndPort > addresses = new HashSet <>();
224
+ // 对ipv6 支持
225
+ for (String node : nodes ) {
226
+ Matcher matcher = HOST_PORT_PATTERN .matcher (node );
227
+ if (matcher .find ()) {
228
+ String host = matcher .group ("host" ).trim ();
229
+ String portStr = matcher .group ("port" ).trim ();
230
+ if (org .apache .commons .lang3 .StringUtils .isNotBlank (host ) && org .apache .commons .lang3 .StringUtils .isNotBlank (portStr )) {
231
+ // 转化为int格式的端口
232
+ int port = Integer .parseInt (portStr );
233
+ addresses .add (new HostAndPort (host , port ));
234
+ }
235
+ }
236
+ }
237
+ jedis = new JedisCluster (addresses , timeout , timeout , 10 , password , poolConfig );
238
+ break ;
220
239
default :
221
240
break ;
222
241
}
@@ -244,35 +263,32 @@ private JedisCommands getJedisWithRetry(int retryNum) {
244
263
return null ;
245
264
}
246
265
247
- private Set <String > getRedisKeys (RedisType redisType , JedisCommands jedis , String keyPattern ){
248
- if (!redisType .equals (RedisType .CLUSTER )){
266
+ private Set <String > getRedisKeys (RedisType redisType , JedisCommands jedis , String keyPattern ) {
267
+ if (!redisType .equals (RedisType .CLUSTER )) {
249
268
return ((Jedis ) jedis ).keys (keyPattern );
250
269
}
251
270
Set <String > keys = new TreeSet <>();
252
- Map <String , JedisPool > clusterNodes = ((JedisCluster )jedis ).getClusterNodes ();
253
- for (String k : clusterNodes .keySet ()){
271
+ Map <String , JedisPool > clusterNodes = ((JedisCluster ) jedis ).getClusterNodes ();
272
+ for (String k : clusterNodes .keySet ()) {
254
273
JedisPool jp = clusterNodes .get (k );
255
- Jedis connection = jp .getResource ();
256
- try {
274
+ try (Jedis connection = jp .getResource ()) {
257
275
keys .addAll (connection .keys (keyPattern ));
258
- } catch (Exception e ){
259
- LOG .error ("Getting keys error: {}" , e );
260
- } finally {
261
- connection .close ();
276
+ } catch (Exception e ) {
277
+ LOG .error ("Getting keys error" , e );
262
278
}
263
279
}
264
280
return keys ;
265
281
}
266
282
267
- private GenericObjectPoolConfig setPoolConfig (String maxTotal , String maxIdle , String minIdle ){
283
+ private GenericObjectPoolConfig setPoolConfig (String maxTotal , String maxIdle , String minIdle ) {
268
284
GenericObjectPoolConfig config = new GenericObjectPoolConfig ();
269
- if (maxTotal != null ){
285
+ if (maxTotal != null ) {
270
286
config .setMaxTotal (Integer .parseInt (maxTotal ));
271
287
}
272
- if (maxIdle != null ){
288
+ if (maxIdle != null ) {
273
289
config .setMaxIdle (Integer .parseInt (maxIdle ));
274
290
}
275
- if (minIdle != null ){
291
+ if (minIdle != null ) {
276
292
config .setMinIdle (Integer .parseInt (minIdle ));
277
293
}
278
294
return config ;
0 commit comments