6464 * @author xuchao
6565 */
6666
67- public abstract class BaseAsyncReqRow extends RichAsyncFunction <Tuple2 < Boolean , Row >, Tuple2 < Boolean , BaseRow > > implements ISideReqRow {
67+ public abstract class BaseAsyncReqRow extends RichAsyncFunction <Row , BaseRow > implements ISideReqRow {
6868 private static final Logger LOG = LoggerFactory .getLogger (BaseAsyncReqRow .class );
6969 private static final long serialVersionUID = 2098635244857937717L ;
7070 private RuntimeContext runtimeContext ;
@@ -135,12 +135,12 @@ protected boolean openCache() {
135135 return sideInfo .getSideCache () != null ;
136136 }
137137
138- protected void dealMissKey (Tuple2 < Boolean , Row > input , ResultFuture <Tuple2 < Boolean , BaseRow > > resultFuture ) {
138+ protected void dealMissKey (Row input , ResultFuture <BaseRow > resultFuture ) {
139139 if (sideInfo .getJoinType () == JoinType .LEFT ) {
140140 //Reserved left table data
141141 try {
142- Row row = fillData (input . f1 , null );
143- RowDataComplete .completeTupleRows (resultFuture , Collections . singleton ( new Tuple2 <>( input . f0 , row )) );
142+ Row row = fillData (input , null );
143+ RowDataComplete .completeRow (resultFuture , row );
144144 } catch (Exception e ) {
145145 dealFillDataError (input , resultFuture , e );
146146 }
@@ -156,7 +156,7 @@ protected void dealCacheData(String key, CacheObj missKeyObj) {
156156 }
157157
158158 @ Override
159- public void timeout (Tuple2 < Boolean , Row > input , ResultFuture <Tuple2 < Boolean , BaseRow > > resultFuture ) throws Exception {
159+ public void timeout (Row input , ResultFuture <BaseRow > resultFuture ) throws Exception {
160160
161161 if (timeOutNum % TIMEOUT_LOG_FLUSH_NUM == 0 ) {
162162 LOG .info ("Async function call has timed out. input:{}, timeOutNum:{}" , input .toString (), timeOutNum );
@@ -173,14 +173,14 @@ public void timeout(Tuple2<Boolean, Row> input, ResultFuture<Tuple2<Boolean, Bas
173173 resultFuture .complete (null );
174174 }
175175
176- protected void preInvoke (Tuple2 < Boolean , Row > input , ResultFuture <Tuple2 < Boolean , BaseRow > > resultFuture )
176+ protected void preInvoke (Row input , ResultFuture <BaseRow > resultFuture )
177177 throws InvocationTargetException , IllegalAccessException {
178178 registerTimerAndAddToHandler (input , resultFuture );
179179 }
180180
181181 @ Override
182- public void asyncInvoke (Tuple2 < Boolean , Row > row , ResultFuture <Tuple2 < Boolean , BaseRow > > resultFuture ) throws Exception {
183- Tuple2 < Boolean , Row > input = Tuple2 . of ( row . f0 , Row .copy (row . f1 ) );
182+ public void asyncInvoke (Row row , ResultFuture <BaseRow > resultFuture ) throws Exception {
183+ Row input = Row .copy (row );
184184 preInvoke (input , resultFuture );
185185 Map <String , Object > inputParams = parseInputParam (input );
186186 if (MapUtils .isEmpty (inputParams )) {
@@ -194,11 +194,11 @@ public void asyncInvoke(Tuple2<Boolean, Row> row, ResultFuture<Tuple2<Boolean, B
194194 handleAsyncInvoke (inputParams , input , resultFuture );
195195 }
196196
197- private Map <String , Object > parseInputParam (Tuple2 < Boolean , Row > input ) {
197+ private Map <String , Object > parseInputParam (Row input ) {
198198 Map <String , Object > inputParams = Maps .newHashMap ();
199199 for (int i = 0 ; i < sideInfo .getEqualValIndex ().size (); i ++) {
200200 Integer conValIndex = sideInfo .getEqualValIndex ().get (i );
201- Object equalObj = input .f1 . getField (conValIndex );
201+ Object equalObj = input .getField (conValIndex );
202202 if (equalObj == null ) {
203203 return inputParams ;
204204 }
@@ -212,7 +212,7 @@ protected boolean isUseCache(Map<String, Object> inputParams) {
212212 return openCache () && getFromCache (buildCacheKey (inputParams )) != null ;
213213 }
214214
215- private void invokeWithCache (Map <String , Object > inputParams , Tuple2 < Boolean , Row > input , ResultFuture <Tuple2 < Boolean , BaseRow > > resultFuture ) {
215+ private void invokeWithCache (Map <String , Object > inputParams , Row input , ResultFuture <BaseRow > resultFuture ) {
216216 if (openCache ()) {
217217 CacheObj val = getFromCache (buildCacheKey (inputParams ));
218218 if (val != null ) {
@@ -221,21 +221,19 @@ private void invokeWithCache(Map<String, Object> inputParams, Tuple2<Boolean, Ro
221221 return ;
222222 } else if (ECacheContentType .SingleLine == val .getType ()) {
223223 try {
224- Row row = fillData (input .f1 , val .getContent ());
225- BaseRow baseRow = RowDataConvert .convertToBaseRow (row );
226- resultFuture .complete (Collections .singleton (Tuple2 .of (input .f0 , baseRow )));
224+ Row row = fillData (input , val .getContent ());
225+ RowDataComplete .completeRow (resultFuture , row );
227226 } catch (Exception e ) {
228227 dealFillDataError (input , resultFuture , e );
229228 }
230229 } else if (ECacheContentType .MultiLine == val .getType ()) {
231230 try {
232- List <Tuple2 < Boolean , BaseRow > > rowList = Lists .newArrayList ();
231+ List <Row > rowList = Lists .newArrayList ();
233232 for (Object one : (List ) val .getContent ()) {
234- Row row = fillData (input .f1 , one );
235- BaseRow baseRow = RowDataConvert .convertToBaseRow (row );
236- rowList .add (Tuple2 .of (input .f0 , baseRow ));
233+ Row row = fillData (input , one );
234+ rowList .add (row );
237235 }
238- resultFuture . complete ( rowList );
236+ RowDataComplete . completeRow ( resultFuture , rowList );
239237 } catch (Exception e ) {
240238 dealFillDataError (input , resultFuture , e );
241239 }
@@ -247,22 +245,22 @@ private void invokeWithCache(Map<String, Object> inputParams, Tuple2<Boolean, Ro
247245 }
248246 }
249247
250- public abstract void handleAsyncInvoke (Map <String , Object > inputParams , Tuple2 < Boolean , Row > input , ResultFuture <Tuple2 < Boolean , BaseRow > > resultFuture ) throws Exception ;
248+ public abstract void handleAsyncInvoke (Map <String , Object > inputParams , Row input , ResultFuture <BaseRow > resultFuture ) throws Exception ;
251249
252250 public abstract String buildCacheKey (Map <String , Object > inputParams );
253251
254252 private ProcessingTimeService getProcessingTimeService () {
255253 return ((StreamingRuntimeContext ) this .runtimeContext ).getProcessingTimeService ();
256254 }
257255
258- protected ScheduledFuture <?> registerTimer (Tuple2 < Boolean , Row > input , ResultFuture <Tuple2 < Boolean , BaseRow > > resultFuture ) {
256+ protected ScheduledFuture <?> registerTimer (Row input , ResultFuture <BaseRow > resultFuture ) {
259257 long timeoutTimestamp = sideInfo .getSideTableInfo ().getAsyncTimeout () + getProcessingTimeService ().getCurrentProcessingTime ();
260258 return getProcessingTimeService ().registerTimer (
261259 timeoutTimestamp ,
262260 timestamp -> timeout (input , resultFuture ));
263261 }
264262
265- protected void registerTimerAndAddToHandler (Tuple2 < Boolean , Row > input , ResultFuture <Tuple2 < Boolean , BaseRow > > resultFuture )
263+ protected void registerTimerAndAddToHandler (Row input , ResultFuture <BaseRow > resultFuture )
266264 throws InvocationTargetException , IllegalAccessException {
267265 ScheduledFuture <?> timeFuture = registerTimer (input , resultFuture );
268266 // resultFuture 是ResultHandler 的实例
@@ -272,7 +270,7 @@ protected void registerTimerAndAddToHandler(Tuple2<Boolean, Row> input, ResultFu
272270 }
273271
274272
275- protected void dealFillDataError (Tuple2 < Boolean , Row > input , ResultFuture <Tuple2 < Boolean , BaseRow > > resultFuture , Throwable e ) {
273+ protected void dealFillDataError (Row input , ResultFuture <BaseRow > resultFuture , Throwable e ) {
276274 parseErrorRecords .inc ();
277275 if (parseErrorRecords .getCount () > sideInfo .getSideTableInfo ().getAsyncFailMaxNum (Long .MAX_VALUE )) {
278276 LOG .info ("dealFillDataError" , e );
0 commit comments