Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-8423

OperatorChain#pushToOperator catch block may fail with NPE

    Details

    • Type: Bug
    • Status: Closed
    • Priority: Critical
    • Resolution: Fixed
    • Affects Version/s: 1.4.0, 1.5.0
    • Fix Version/s: 1.5.0, 1.4.2
    • Component/s: Streaming
    • Labels:
      None

      Description

      @Override
      protected <X> void pushToOperator(StreamRecord<X> record) {
      	try {
      		// we know that the given outputTag matches our OutputTag so the record
      		// must be of the type that our operator (and Serializer) expects.
      		@SuppressWarnings("unchecked")
      		StreamRecord<T> castRecord = (StreamRecord<T>) record;
      
      		numRecordsIn.inc();
      		StreamRecord<T> copy = castRecord.copy(serializer.copy(castRecord.getValue()));
      		operator.setKeyContextElement1(copy);
      		operator.processElement(copy);
      	} catch (ClassCastException e) {
      		// Enrich error message
      		ClassCastException replace = new ClassCastException(
      			String.format(
      				"%s. Failed to push OutputTag with id '%s' to operator. " +
      				"This can occur when multiple OutputTags with different types " +
      				"but identical names are being used.",
      				e.getMessage(),
      				outputTag.getId()));
      
      		throw new ExceptionInChainedOperatorException(replace);
      
      	} catch (Exception e) {
      		throw new ExceptionInChainedOperatorException(e);
      	}
      }
      

      If outputTag is null (as is the case when no sideOutput was defined) the catch block will crash with a NullPointerException. This may happen if operator.processElement throws a ClassCastException.

        Attachments

          Issue Links

            Activity

              People

              • Assignee:
                mingleizhang zhangminglei
                Reporter:
                Zentol Chesnay Schepler
              • Votes:
                0 Vote for this issue
                Watchers:
                3 Start watching this issue

                Dates

                • Created:
                  Updated:
                  Resolved: