[ 
https://issues.apache.org/jira/browse/FLINK-7423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16124791#comment-16124791
 ] 

ASF GitHub Bot commented on FLINK-7423:
---------------------------------------

Github user XuPingyong commented on the issue:

    https://github.com/apache/flink/pull/4525
  
    @greghogan, if the object passed to nextRecord may be reused internally by 
the InputFormat, do the similar cases need to  be re-considered?
    
    In `DataSourceTask.java`:
                  
                  OT reuse = serializer.createInstance();
    
                // as long as there is data to read
                while (!this.taskCanceled && !format.reachedEnd()) {
                    OT returned;
                    if ((returned = format.nextRecord(reuse)) != null) {
                        output.collect(returned);
                    }
                }
    
    And in many batch drivers:
                  
                  final MutableObjectIterator<T> in = taskContext.getInput(0);
                  T value = serializer.createInstance();
                  while (running && (value = in.next(value)) != null) {
                      .......
                  } 
    
    
    In my opinion:
         1.  `Null` records are meaningless, but `null` is meaningful for input 
or format which means the end. If a user only call `InputFormat#nextRecord` 
without `InputFormat#reachedEnd`, only `null` can be returned. 
         2.  The returned object of `InputFormat#nextRecord` should not need to 
be considered that it may be passed again. If a immutable object is returned, 
an exception will be thrown  when it is reused again in 
`InputFormat#nextRecord`.
    
    @greghogan, could you please offer some cases that the object passed to 
nextRecord can be reused internally by the InputFormat?  Thanks.


> Always reuse an instance  to get elements from the inputFormat 
> ---------------------------------------------------------------
>
>                 Key: FLINK-7423
>                 URL: https://issues.apache.org/jira/browse/FLINK-7423
>             Project: Flink
>          Issue Type: Bug
>          Components: DataStream API
>            Reporter: Xu Pingyong
>            Assignee: Xu Pingyong
>
> In InputFormatSourceFunction.java:
> {code:java}
> OUT nextElement = serializer.createInstance();
>                       while (isRunning) {
>                               format.open(splitIterator.next());
>                               // for each element we also check if cancel
>                               // was called by checking the isRunning flag
>                               while (isRunning && !format.reachedEnd()) {
>                                       nextElement = 
> format.nextRecord(nextElement);
>                                       if (nextElement != null) {
>                                               ctx.collect(nextElement);
>                                       } else {
>                                               break;
>                                       }
>                               }
>                               format.close();
>                               completedSplitsCounter.inc();
>                               if (isRunning) {
>                                       isRunning = splitIterator.hasNext();
>                               }
>                       }
> {code}
> the format may return other element or null when nextRecord, that will may 
> cause exception.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to