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

Sarthak Dubey commented on NIFI-4395:
-------------------------------------

Hello Deon Huang,

On reference to this issue

https://issues.apache.org/jira/browse/NIFI-4395?focusedCommentId=16330518&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16330518

There is a GenerateTableFetch processor which has states stored for multiple 
tables. When the flow is run after reboot one of the table which has the stored 
state gets through however the rest of the tables stay in the queue throwing  
the exception (reference above comment link). 

In our analysis,when we reboot the instance and run the flow the map 
columnTypeMap on line number 419 which is used to get the value of "type" in  
org.apache.nifi.processors.standard.GenerateTableFetch.getColumnType(GenerateTableFetch.java:423)
 method contains null so it throws an exception and the flowfile gets stuck 
there and for every other flowfile  with respect to other remaining tables the 
code(GenerateTableFetch.getColumnType method) tries to get "type" from the same 
map and it gets  "type" as null and so it throws an Exception.

But when i return 93(column type expected is timestamp) commenting out the 
exception (changing code in GenerateTableFetch.getColumnType) , map gets 
updated with the other entries as well and the flow starts running as it should.

 

 

> GenerateTableFetch can't fetch column type by state after instance reboot
> -------------------------------------------------------------------------
>
>                 Key: NIFI-4395
>                 URL: https://issues.apache.org/jira/browse/NIFI-4395
>             Project: Apache NiFi
>          Issue Type: Bug
>    Affects Versions: 1.3.0
>            Reporter: Deon Huang
>            Assignee: Deon Huang
>            Priority: Major
>             Fix For: 1.4.0
>
>         Attachments: GenerateTableFetch_Exception.png
>
>
> The problem can easily be reproduce.
> Once GenerateTableFetch store state and encounter NiFi instance reboot.
> (Dynamic naming table by expression language)
> The exception will occur.
> The error in source code is list below.
> ```
>     if (type == null) {
>     // This shouldn't happen as we are populating columnTypeMap when the 
> processor is scheduled or when the first maximum is observed
>     throw new IllegalArgumentException("No column type found for: " + 
> colName);
>     }
> ```
> When this situation happened. The FlowFile will also be grab and can't 
> release or observed.
> Processor can't grab existing  column type from *columnTypeMap* through 
> instance reboot.
> Hence will inevidible get this exception, rollback FlowFile and never success.
> QueryDatabaseTable processor will not encounter this exception due to it 
> setup(context) every time,
> While GenerateTableFetch will not pass the condition and thus try to fetch 
> column type from 0 length columnTypeMap.
> ---
>     if (!isDynamicTableName && !isDynamicMaxValues) {
>     super.setup(context);
>     }
> ---
> I can take the issue if it is recognize as bug.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to