33import com .dtstack .flink .sql .side .AllReqRow ;
44import com .dtstack .flink .sql .side .FieldInfo ;
55import com .dtstack .flink .sql .side .JoinInfo ;
6+ import com .dtstack .flink .sql .side .PredicateInfo ;
67import com .dtstack .flink .sql .side .SideTableInfo ;
78import com .dtstack .flink .sql .side .kudu .table .KuduSideTableInfo ;
9+ import com .dtstack .flink .sql .side .kudu .utils .KuduUtil ;
810import org .apache .calcite .sql .JoinType ;
911import org .apache .commons .collections .CollectionUtils ;
1012import org .apache .flink .api .java .typeutils .RowTypeInfo ;
1618import org .apache .flink .util .Preconditions ;
1719import org .apache .kudu .ColumnSchema ;
1820import org .apache .kudu .Schema ;
19- import org .apache .kudu .Type ;
20- import org .apache .kudu .client .*;
21+ import org .apache .kudu .client .KuduClient ;
22+ import org .apache .kudu .client .KuduException ;
23+ import org .apache .kudu .client .KuduPredicate ;
24+ import org .apache .kudu .client .KuduScanner ;
25+ import org .apache .kudu .client .KuduTable ;
26+ import org .apache .kudu .client .PartialRow ;
27+ import org .apache .kudu .client .RowResult ;
28+ import org .apache .kudu .client .RowResultIterator ;
2129import org .slf4j .Logger ;
2230import org .slf4j .LoggerFactory ;
2331
2432import java .sql .SQLException ;
2533import java .sql .Timestamp ;
26- import java .util .*;
34+ import java .util .Arrays ;
35+ import java .util .Calendar ;
36+ import java .util .HashMap ;
37+ import java .util .List ;
38+ import java .util .Map ;
2739import java .util .concurrent .atomic .AtomicReference ;
2840
2941public class KuduAllReqRow extends AllReqRow {
@@ -158,7 +170,7 @@ private void loadData(Map<String, List<Map<String, Object>>> tmpCache) {
158170 String sideFieldName = sideFieldName1 .trim ();
159171 ColumnSchema columnSchema = table .getSchema ().getColumn (sideFieldName );
160172 if (null != columnSchema ) {
161- setMapValue (columnSchema .getType (), oneRow , sideFieldName , result );
173+ KuduUtil . setMapValue (columnSchema .getType (), oneRow , sideFieldName , result );
162174 }
163175 }
164176 String cacheKey = buildKey (oneRow , sideInfo .getEqualFieldList ());
@@ -244,8 +256,7 @@ private KuduScanner getConn(KuduSideTableInfo tableInfo) {
244256 * @param tableInfo AsyncKuduScanner的配置信息
245257 * @return
246258 */
247- private KuduScanner buildScanner (KuduScanner .KuduScannerBuilder builder , Schema schema , KuduSideTableInfo
248- tableInfo ) {
259+ private KuduScanner buildScanner (KuduScanner .KuduScannerBuilder builder , Schema schema , KuduSideTableInfo tableInfo ) {
249260 Integer batchSizeBytes = tableInfo .getBatchSizeBytes ();
250261 Long limitNum = tableInfo .getLimitNum ();
251262 Boolean isFaultTolerant = tableInfo .getFaultTolerant ();
@@ -268,6 +279,17 @@ private KuduScanner buildScanner(KuduScanner.KuduScannerBuilder builder, Schema
268279 if (null != isFaultTolerant ) {
269280 builder .setFaultTolerant (isFaultTolerant );
270281 }
282+ // 填充谓词信息
283+ List <PredicateInfo > predicateInfoes = sideInfo .getSideTableInfo ().getPredicateInfoes ();
284+ if (predicateInfoes .size () > 0 ) {
285+ predicateInfoes .stream ().map (info -> {
286+ KuduPredicate kuduPredicate = KuduUtil .buildKuduPredicate (schema , info );
287+ if (null != kuduPredicate ) {
288+ builder .addPredicate (kuduPredicate );
289+ }
290+ return info ;
291+ }).count ();
292+ }
271293
272294 if (null != lowerBoundPrimaryKey && null != upperBoundPrimaryKey && null != primaryKeys ) {
273295 List <ColumnSchema > columnSchemas = schema .getPrimaryKeyColumns ();
@@ -282,8 +304,8 @@ private KuduScanner buildScanner(KuduScanner.KuduScannerBuilder builder, Schema
282304 PartialRow upperPartialRow = schema .newPartialRow ();
283305 for (int i = 0 ; i < primaryKey .length ; i ++) {
284306 Integer index = columnName .get (primaryKey [i ]);
285- primaryKeyRange (lowerPartialRow , columnSchemas .get (index ).getType (), primaryKey [i ], lowerBounds [i ]);
286- primaryKeyRange (upperPartialRow , columnSchemas .get (index ).getType (), primaryKey [i ], upperBounds [i ]);
307+ KuduUtil . primaryKeyRange (lowerPartialRow , columnSchemas .get (index ).getType (), primaryKey [i ], lowerBounds [i ]);
308+ KuduUtil . primaryKeyRange (upperPartialRow , columnSchemas .get (index ).getType (), primaryKey [i ], upperBounds [i ]);
287309 }
288310 builder .lowerBound (lowerPartialRow );
289311 builder .exclusiveUpperBound (upperPartialRow );
@@ -296,80 +318,6 @@ private String[] splitString(String data) {
296318 return data .split ("," );
297319 }
298320
299- private void primaryKeyRange (PartialRow partialRow , Type type , String primaryKey , String value ) {
300- switch (type ) {
301- case STRING :
302- partialRow .addString (primaryKey , value );
303- break ;
304- case FLOAT :
305- partialRow .addFloat (primaryKey , Float .valueOf (value ));
306- break ;
307- case INT8 :
308- partialRow .addByte (primaryKey , Byte .valueOf (value ));
309- break ;
310- case INT16 :
311- partialRow .addShort (primaryKey , Short .valueOf (value ));
312- break ;
313- case INT32 :
314- partialRow .addInt (primaryKey , Integer .valueOf (value ));
315- break ;
316- case INT64 :
317- partialRow .addLong (primaryKey , Long .valueOf (value ));
318- break ;
319- case DOUBLE :
320- partialRow .addDouble (primaryKey , Double .valueOf (value ));
321- break ;
322- case BOOL :
323- partialRow .addBoolean (primaryKey , Boolean .valueOf (value ));
324- break ;
325- case UNIXTIME_MICROS :
326- partialRow .addTimestamp (primaryKey , Timestamp .valueOf (value ));
327- break ;
328- case BINARY :
329- partialRow .addBinary (primaryKey , value .getBytes ());
330- break ;
331- default :
332- throw new IllegalArgumentException ("Illegal var type: " + type );
333- }
334- }
335-
336- private void setMapValue (Type type , Map <String , Object > oneRow , String sideFieldName , RowResult result ) {
337- switch (type ) {
338- case STRING :
339- oneRow .put (sideFieldName , result .getString (sideFieldName ));
340- break ;
341- case FLOAT :
342- oneRow .put (sideFieldName , result .getFloat (sideFieldName ));
343- break ;
344- case INT8 :
345- oneRow .put (sideFieldName , result .getFloat (sideFieldName ));
346- break ;
347- case INT16 :
348- oneRow .put (sideFieldName , result .getShort (sideFieldName ));
349- break ;
350- case INT32 :
351- oneRow .put (sideFieldName , result .getInt (sideFieldName ));
352- break ;
353- case INT64 :
354- oneRow .put (sideFieldName , result .getLong (sideFieldName ));
355- break ;
356- case DOUBLE :
357- oneRow .put (sideFieldName , result .getDouble (sideFieldName ));
358- break ;
359- case BOOL :
360- oneRow .put (sideFieldName , result .getBoolean (sideFieldName ));
361- break ;
362- case UNIXTIME_MICROS :
363- oneRow .put (sideFieldName , result .getTimestamp (sideFieldName ));
364- break ;
365- case BINARY :
366- oneRow .put (sideFieldName , result .getBinary (sideFieldName ));
367- break ;
368- default :
369- throw new IllegalArgumentException ("Illegal var type: " + type );
370- }
371- }
372-
373321 @ Override
374322 public void close () throws Exception {
375323 //公用一个client 如果每次刷新间隔时间较长可以每次获取一个
0 commit comments