|
19 | 19 | package com.dtstack.flink.sql.side.redis; |
20 | 20 |
|
21 | 21 | import com.dtstack.flink.sql.side.*; |
| 22 | +import com.dtstack.flink.sql.side.redis.enums.RedisType; |
22 | 23 | import com.dtstack.flink.sql.side.redis.table.RedisSideReqRow; |
23 | 24 | import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; |
24 | 25 | import com.esotericsoftware.minlog.Log; |
25 | 26 | import org.apache.calcite.sql.JoinType; |
| 27 | +import org.apache.commons.collections.CollectionUtils; |
| 28 | +import org.apache.commons.collections.MapUtils; |
| 29 | +import org.apache.commons.lang.StringUtils; |
26 | 30 | import org.apache.commons.pool2.impl.GenericObjectPoolConfig; |
27 | 31 | import org.apache.flink.api.java.typeutils.RowTypeInfo; |
28 | 32 | import com.google.common.collect.Maps; |
@@ -103,104 +107,51 @@ public void flatMap(Row row, Collector<Row> out) throws Exception { |
103 | 107 | String columnName = sideInfo.getEqualFieldList().get(conValIndex); |
104 | 108 | inputParams.put(columnName, equalObj.toString()); |
105 | 109 | } |
106 | | - String key = buildKey(inputParams); |
| 110 | + String key = buildCacheKey(inputParams); |
| 111 | + if(StringUtils.isBlank(key)){ |
| 112 | + return; |
| 113 | + } |
107 | 114 |
|
108 | 115 | Map<String, String> cacheMap = cacheRef.get().get(key); |
109 | | - |
110 | | - if (cacheMap == null){ |
111 | | - if(sideInfo.getJoinType() == JoinType.LEFT){ |
112 | | - Row data = fillData(row, null); |
113 | | - out.collect(data); |
114 | | - }else{ |
| 116 | + if(MapUtils.isEmpty(cacheMap)){ |
| 117 | + if(sideInfo.getJoinType() != JoinType.LEFT){ |
115 | 118 | return; |
116 | 119 | } |
117 | | - |
| 120 | + Row data = fillData(row, null); |
| 121 | + out.collect(data); |
118 | 122 | return; |
119 | | - } |
120 | 123 |
|
| 124 | + } |
121 | 125 | Row newRow = fillData(row, cacheMap); |
122 | 126 | out.collect(newRow); |
123 | 127 | } |
124 | 128 |
|
125 | | - private String buildKey(Map<String, String> inputParams) { |
126 | | - String tableName = tableInfo.getTableName(); |
127 | | - StringBuilder key = new StringBuilder(); |
128 | | - for (int i=0; i<inputParams.size(); i++){ |
129 | | - key.append(tableName).append(":").append(inputParams.keySet().toArray()[i]).append(":") |
130 | | - .append(inputParams.get(inputParams.keySet().toArray()[i])); |
| 129 | + private String buildCacheKey(Map<String, String> refData) { |
| 130 | + StringBuilder keyBuilder = new StringBuilder(tableInfo.getTableName()); |
| 131 | + List<String> primaryKeys = tableInfo.getPrimaryKeys(); |
| 132 | + for(String primaryKey : primaryKeys){ |
| 133 | + if(!refData.containsKey(primaryKey)){ |
| 134 | + return null; |
| 135 | + } |
| 136 | + keyBuilder.append("_").append(refData.get(primaryKey)); |
131 | 137 | } |
132 | | - return key.toString(); |
| 138 | + return keyBuilder.toString(); |
133 | 139 | } |
134 | 140 |
|
| 141 | + |
135 | 142 | private void loadData(Map<String, Map<String, String>> tmpCache) throws SQLException { |
136 | 143 | JedisCommands jedis = null; |
137 | | - |
138 | 144 | try { |
139 | | - for(int i=0; i<CONN_RETRY_NUM; i++){ |
140 | | - |
141 | | - try{ |
142 | | - jedis = getJedis(tableInfo); |
143 | | - break; |
144 | | - }catch (Exception e){ |
145 | | - if(i == CONN_RETRY_NUM - 1){ |
146 | | - throw new RuntimeException("", e); |
147 | | - } |
148 | | - |
149 | | - try { |
150 | | - String jedisInfo = "url:" + tableInfo.getUrl() + ",pwd:" + tableInfo.getPassword() + ",database:" + tableInfo.getDatabase(); |
151 | | - LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + jedisInfo); |
152 | | - Thread.sleep(5 * 1000); |
153 | | - } catch (InterruptedException e1) { |
154 | | - LOG.error("", e1); |
155 | | - } |
156 | | - } |
157 | | - } |
158 | | - |
159 | | - if (tableInfo.getRedisType() != 3){ |
160 | | - String perKey = tableInfo.getTableName() + "*"; |
161 | | - Set<String> keys = ((Jedis) jedis).keys(perKey); |
162 | | - List<String> newPerKeys = new LinkedList<>(); |
163 | | - for (String key : keys){ |
164 | | - String[] splitKey = key.split(":"); |
165 | | - String newKey = splitKey[0] + ":" + splitKey[1] + ":" + splitKey[2]; |
166 | | - newPerKeys.add(newKey); |
167 | | - } |
168 | | - List<String> list = newPerKeys.stream().distinct().collect(Collectors.toList()); |
169 | | - for(String key : list){ |
170 | | - Map<String, String> kv = Maps.newHashMap(); |
171 | | - String[] primaryKv = key.split(":"); |
172 | | - kv.put(primaryKv[1], primaryKv[2]); |
173 | | - String pattern = key + "*"; |
174 | | - Set<String> realKeys = ((Jedis) jedis).keys(pattern); |
175 | | - for (String realKey : realKeys){ |
176 | | - kv.put(realKey.split(":")[3], jedis.get(realKey)); |
177 | | - } |
178 | | - tmpCache.put(key, kv); |
179 | | - } |
180 | | - } else { |
181 | | - String perKey = tableInfo.getTableName() + "*"; |
182 | | - Set<String> keys = keys((JedisCluster) jedis, perKey); |
183 | | - List<String> newPerKeys = new LinkedList<>(); |
184 | | - for (String key : keys){ |
185 | | - String[] splitKey = key.split(":"); |
186 | | - String newKey = splitKey[0] + ":" + splitKey[1] + ":" + splitKey[2]; |
187 | | - newPerKeys.add(newKey); |
188 | | - } |
189 | | - List<String> list = newPerKeys.stream().distinct().collect(Collectors.toList()); |
190 | | - for(String key : list){ |
191 | | - Map<String, String> kv = Maps.newHashMap(); |
192 | | - String[] primaryKv = key.split(":"); |
193 | | - kv.put(primaryKv[1], primaryKv[2]); |
194 | | - String pattern = key + "*"; |
195 | | - Set<String> realKeys = keys((JedisCluster) jedis, pattern); |
196 | | - for (String realKey : realKeys){ |
197 | | - kv.put(realKey.split(":")[3], jedis.get(realKey)); |
198 | | - } |
199 | | - tmpCache.put(key, kv); |
200 | | - } |
| 145 | + String keyPattern = tableInfo.getTableName() + "*"; |
| 146 | + jedis = getJedisWithRetry(CONN_RETRY_NUM); |
| 147 | + Set<String> keys = getRedisKeys(RedisType.parse(tableInfo.getRedisType()), jedis, keyPattern); |
| 148 | + if(CollectionUtils.isEmpty(keys)){ |
| 149 | + return; |
201 | 150 | } |
202 | | - |
203 | | - |
| 151 | + JedisCommands redis = jedis; |
| 152 | + keys.forEach(k ->{ |
| 153 | + tmpCache.put(k, redis.hgetAll(k)); |
| 154 | + }); |
204 | 155 | } catch (Exception e){ |
205 | 156 | LOG.error("", e); |
206 | 157 | } finally { |
@@ -245,33 +196,58 @@ private JedisCommands getJedis(RedisSideTableInfo tableInfo) { |
245 | 196 | } |
246 | 197 | JedisCommands jedis = null; |
247 | 198 | GenericObjectPoolConfig poolConfig = setPoolConfig(tableInfo.getMaxTotal(), tableInfo.getMaxIdle(), tableInfo.getMinIdle()); |
248 | | - switch (tableInfo.getRedisType()){ |
| 199 | + switch (RedisType.parse(tableInfo.getRedisType())){ |
249 | 200 | //单机 |
250 | | - case 1: |
| 201 | + case STANDALONE: |
251 | 202 | pool = new JedisPool(poolConfig, firstIp, Integer.parseInt(firstPort), timeout, password, Integer.parseInt(database)); |
252 | 203 | jedis = pool.getResource(); |
253 | 204 | break; |
254 | 205 | //哨兵 |
255 | | - case 2: |
| 206 | + case SENTINEL: |
256 | 207 | jedisSentinelPool = new JedisSentinelPool(tableInfo.getMasterName(), ipPorts, poolConfig, timeout, password, Integer.parseInt(database)); |
257 | 208 | jedis = jedisSentinelPool.getResource(); |
258 | 209 | break; |
259 | 210 | //集群 |
260 | | - case 3: |
| 211 | + case CLUSTER: |
261 | 212 | jedis = new JedisCluster(addresses, timeout, timeout,1, poolConfig); |
| 213 | + default: |
| 214 | + break; |
262 | 215 | } |
263 | 216 |
|
264 | 217 | return jedis; |
265 | 218 | } |
266 | 219 |
|
267 | | - private Set<String> keys(JedisCluster jedisCluster, String pattern){ |
| 220 | + private JedisCommands getJedisWithRetry(int retryNum) { |
| 221 | + while (retryNum-- > 0){ |
| 222 | + try { |
| 223 | + return getJedis(tableInfo); |
| 224 | + } catch (Exception e) { |
| 225 | + if(retryNum <= 0){ |
| 226 | + throw new RuntimeException("getJedisWithRetry error", e); |
| 227 | + } |
| 228 | + try { |
| 229 | + String jedisInfo = "url:" + tableInfo.getUrl() + ",pwd:" + tableInfo.getPassword() + ",database:" + tableInfo.getDatabase(); |
| 230 | + LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + jedisInfo); |
| 231 | + Thread.sleep(5 * 1000); |
| 232 | + } catch (InterruptedException e1) { |
| 233 | + LOG.error("", e1); |
| 234 | + } |
| 235 | + } |
| 236 | + } |
| 237 | + return null; |
| 238 | + } |
| 239 | + |
| 240 | + private Set<String> getRedisKeys(RedisType redisType, JedisCommands jedis, String keyPattern){ |
| 241 | + if(!redisType.equals(RedisType.CLUSTER)){ |
| 242 | + return ((Jedis) jedis).keys(keyPattern); |
| 243 | + } |
268 | 244 | Set<String> keys = new TreeSet<>(); |
269 | | - Map<String, JedisPool> clusterNodes = jedisCluster.getClusterNodes(); |
| 245 | + Map<String, JedisPool> clusterNodes = ((JedisCluster)jedis).getClusterNodes(); |
270 | 246 | for(String k : clusterNodes.keySet()){ |
271 | 247 | JedisPool jp = clusterNodes.get(k); |
272 | 248 | Jedis connection = jp.getResource(); |
273 | 249 | try { |
274 | | - keys.addAll(connection.keys(pattern)); |
| 250 | + keys.addAll(connection.keys(keyPattern)); |
275 | 251 | } catch (Exception e){ |
276 | 252 | LOG.error("Getting keys error: {}", e); |
277 | 253 | } finally { |
|
0 commit comments