thanks Matt.
 
Uwe
Gesendet: Mittwoch, 04. Oktober 2017 um 17:32 Uhr
Von: "Matt Burgess" <mattyb...@apache.org>
An: users@nifi.apache.org
Betreff: Re: Re: Nifi 1.4: problem with QueryRecord Precessor
All,

The known issue Mark is referring to is NIFI-4349 [1], however it is
not causing the problem; rather it is hiding the problem. If an error
occurs (due to misonfiguration, schema errors, etc.), certain cleanup
activities aren't being performed so the processor ends up with the
error about processing flow files, InputStreams, etc. instead.
Regarding it working in 1.3.0 and not in 1.4.0, I think some of the
logic changes in NIFI-3921 [2] may have changed the handling of
inherited schemas and now generates an exception where it may have
been ignored before.

Regards,
Matt

[1] https://issues.apache.org/jira/browse/NIFI-4349
[2] https://issues.apache.org/jira/browse/NIFI-3921

On Wed, Oct 4, 2017 at 11:22 AM, Uwe Geercken <uwe.geerc...@web.de> wrote:
>
> Mark,
>
> I stopped 1.4 and started 1.3 and created the same flow. And in 1.3 it works
> without a problem.
>
> And I looked into the 1.4 log (I should have looked before, but only looked
> at the processor's tooltip). You are right, it's a problem with the schema.
> Here is the real error:
>
> 2017-10-04 16:01:16,640 ERROR [Timer-Driven Process Thread-10]
> o.a.nifi.processors.standard.QueryRecord
> QueryRecord[id=e798adc7-015e-1000-a43d-68a02b5d425a] Unable to qu$
> org.apache.nifi.processor.exception.ProcessException: IOException thrown
> from QueryRecord[id=e798adc7-015e-1000-a43d-68a02b5d425a]:
> java.io.IOException: org.apache.nifi.$
> at
> org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:2589)
> at
> org.apache.nifi.processors.standard.QueryRecord.onTrigger(QueryRecord.java:284)
> at
> org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
> at
> org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1119)
> at
> org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:147)
> at
> org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:47)
> at
> org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:128)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.io.IOException:
> org.apache.nifi.schema.access.SchemaNotFoundException: Cannot write Schema
> Name As Attribute because the Schema Name is not known
> at
> org.apache.nifi.processors.standard.QueryRecord$1.process(QueryRecord.java:302)
> at
> org.apache.nifi.controller.repository.StandardProcessSession.write(StandardProcessSession.java:2570)
> ... 13 common frames omitted
> Caused by: org.apache.nifi.schema.access.SchemaNotFoundException: Cannot
> write Schema Name As Attribute because the Schema Name is not known
> at
> org.apache.nifi.schema.access.SchemaNameAsAttribute.validateSchema(SchemaNameAsAttribute.java:53)
> at
> org.apache.nifi.serialization.SchemaRegistryRecordSetWriter.getSchemaAccessWriter(SchemaRegistryRecordSetWriter.java:137)
> at
> org.apache.nifi.csv.CSVRecordSetWriter.createWriter(CSVRecordSetWriter.java:72)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>
>
>
> I am using an AvroReader using the embedded schema. Maybe this helps.
>
> Rgds,
>
> Uwe
>
>
> Gesendet: Mittwoch, 04. Oktober 2017 um 16:55 Uhr
> Von: "Mark Payne" <marka...@hotmail.com>
> An: "users@nifi.apache.org" <users@nifi.apache.org>
>
> Betreff: Re: Nifi 1.4: problem with QueryRecord Precessor
> Dan, Uwe,
>
> This is something that I will be looking into shortly. It is a known issue
> that exists in both 1.3
> and 1.4. This occurs whenever there is some other sort of failure, when the
> processor attempts
> to route the FlowFile to 'failure'. This occurs most often when there is a
> problem obtaining the
> schema for the Record. Can you check your logs and see what other error may
> be present
> in the logs?
>
> Thanks
> -Mark
>
>
>
> On Oct 4, 2017, at 10:39 AM, dan young <danoyo...@gmail.com> wrote:
>
>
> It might be...maybe others can share experience with 1.4...
>
>
> On Wed, Oct 4, 2017, 8:37 AM Uwe Geercken <uwe.geerc...@web.de> wrote:
>>
>> thanks Dano.
>>
>> Any other comments? Is this a bug in 1.4 then?
>>
>> Rgds,
>>
>> Uwe
>>
>> Gesendet: Mittwoch, 04. Oktober 2017 um 16:33 Uhr
>> Von: "dan young" <danoyo...@gmail.com>
>> An: nifi <users@nifi.apache.org>
>> Betreff: Re: Nifi 1.4: problem with QueryRecord Precessor
>>
>> We're seeing the same thing. Works fine in 1.3
>>
>> Dano
>>
>>
>> On Wed, Oct 4, 2017, 8:13 AM Uwe Geercken <uwe.geerc...@web.de> wrote:
>>>
>>> Hello,
>>>
>>> I have created a flow: GetFile >> QueryRecord >> Putfile. GetFile reads
>>> an avro file. QueryRecord has one property/sql and the result is routed to
>>> PutFile.
>>>
>>> When I run the processor, I get following error:
>>>
>>> failed to process session due to java.lang.IllegalStateException
>>> ....
>>> <filename here> already in use or an active callback or an inputstream
>>> created by ProcessSession.read(FlowFile) has not been closed.
>>> ....
>>>
>>> Can somebody help?
>>>
>>> Rgds,
>>>
>>> Uwe

Reply via email to