Hi Tony, This looks like it may be the behavior described in NIFI-8320. Matt has already said he is looking at the problem I described. I'm curious to see what he finds.
Mit freundlichen Grüßen / best regards Kay-Uwe Moosheimer > Am 26.03.2021 um 14:46 schrieb Anton Kovaľ <[email protected]>: > > Hi Uwe, > I was digging at something similar in PutDatabaseRecord 1.13.2 and I found > out that problem is the order of the fields in JSON. I wrote test > (TestPutDatabaseRecord) that shows: > > @Test > void testInsertJsonChangeOrderFields() throws InitializationException, > ProcessException, SQLException, IOException { > recreateTable(createPersons) > > final jsonReader = new JsonTreeReader() > runner.addControllerService("jsonParser", jsonReader) > runner.enableControllerService(jsonReader) > > final input = ''' > [ > { > "id": 1, > "code": 101, > "name": "rec1" > }, > { > "id": 2, > "name": "rec2", > "code": 102 > } > ] > ''' > > runner.setProperty(PutDatabaseRecord.RECORD_READER_FACTORY, > 'jsonParser') > runner.setProperty(PutDatabaseRecord.STATEMENT_TYPE, > PutDatabaseRecord.INSERT_TYPE) > runner.setProperty(PutDatabaseRecord.TABLE_NAME, 'PERSONS') > > runner.enqueue(input) > runner.run() > > runner.assertTransferCount(PutDatabaseRecord.REL_SUCCESS, 1) > final Connection conn = dbcp.getConnection() > final Statement stmt = conn.createStatement() > final ResultSet rs = stmt.executeQuery('SELECT * FROM PERSONS') > assertTrue(rs.next()) > assertEquals(1, rs.getInt(1)) > assertEquals('rec1', rs.getString(2)) > assertEquals(101, rs.getInt(3)) > assertTrue(rs.next()) > assertEquals(2, rs.getInt(1)) > assertEquals('rec2', rs.getString(2)) > assertEquals(102, rs.getInt(3)) > > stmt.close() > conn.close() > } > > Routing to failure.: java.lang.NumberFormatException: For input string: "rec1" > > Tony Koval > >> On 2021/03/25 17:59:07, "[email protected]" <[email protected]> wrote: >> Matt, >> >> Thank you for taking care of this problem! >> >> I have added the "active=true" in the JSON. >> Unfortunately exactly the same error. >> >> I can't add the primary key (unless I read the sequence myself which I >> want to avoid). >> The attribute "ts" must be set only by the database, because it must be >> the exact time. >> >> With this I have unfortunately no more possibility. Or have I overlooked >> something? >> >> Regards, >> Uwe >> >>> Am 25.03.21 um 17:07 schrieb Matt Burgess: >>> Uwe, >>> >>> I think it's related to [1], which tries to use the field's datatype >>> rather than the column's datatype if they're compatible but not >>> identical. I'm guessing if your record doesn't have the field, we're >>> mishandling the datatype and associated INSERT statement. I will >>> reproduce, write up the Jira, and fix it shortly. A workaround should >>> be to fill the records with all fields using the DB's default values. >>> >>> Regards, >>> Matt >>> >>> [1] https://issues.apache.org/jira/browse/NIFI-8223 >>> >>> On Thu, Mar 25, 2021 at 11:20 AM [email protected] <[email protected]> >>> wrote: >>>> Hi Dev-Team, >>>> >>>> We still have a problem with PutDatabaseRecord on NiFi 1.13.2, which >>>> used to run on NiFi 1.12.1. >>>> >>>> We have a JSON that has both more attributes than exist in the table and >>>> missing attributes that are defined in the table with default value. >>>> The table has a bool value that is set "default true" and is not passed >>>> in the JSON. >>>> >>>> Processor settings are: >>>> Record Reader - JsonTreeReader >>>> Database Type - PostgreSQL (tried also with Generic) >>>> Statement Type - INSERT >>>> Translate Field Names - false >>>> Quote Column Identifier - false >>>> Quate Table identifier - false >>>> Unmatched Field Behavior - Ignore Unmatched Fields >>>> Unmatched Column Behavior - Ignore Unmatched Columns >>>> >>>> As error message we get: >>>> >>>> 2021-03-25 15:00:24,509 ERROR [Timer-Driven Process Thread-7] >>>> o.a.n.p.standard.PutDatabaseRecord >>>> PutDatabaseRecord[id=a1ef9918-0177-1000-ffff-ffffba128239] Failed to put >>>> Records to database for >>>> StandardFlowFileRecord[uuid=47bacb24-718b-42e0-97a9-588ab628a4af,claim=StandardContentClaim >>>> [resourceClaim=StandardResourceClaim[id=1616614332963-2, >>>> container=default, section=2], offset=97327973, >>>> length=877],offset=0,name=3699054725979545,size=877]. Routing to >>>> failure.: org.postgresql.util.PSQLException: Cannot cast to boolean: >>>> "hKaT0ytUPfgwNcjhDDtKRin42743t" >>>> org.postgresql.util.PSQLException: Cannot cast to boolean: >>>> "hKaT0ytUPfgwNcjhDDtKRin42743t" >>>> at >>>> org.postgresql.jdbc.BooleanTypeUtil.cannotCoerceException(BooleanTypeUtil.java:99) >>>> at >>>> org.postgresql.jdbc.BooleanTypeUtil.fromString(BooleanTypeUtil.java:67) >>>> at >>>> org.postgresql.jdbc.BooleanTypeUtil.castToBoolean(BooleanTypeUtil.java:43) >>>> at >>>> org.postgresql.jdbc.PgPreparedStatement.setObject(PgPreparedStatement.java:655) >>>> at >>>> org.postgresql.jdbc.PgPreparedStatement.setObject(PgPreparedStatement.java:935) >>>> at >>>> org.apache.commons.dbcp2.DelegatingPreparedStatement.setObject(DelegatingPreparedStatement.java:529) >>>> at >>>> org.apache.commons.dbcp2.DelegatingPreparedStatement.setObject(DelegatingPreparedStatement.java:529) >>>> at jdk.internal.reflect.GeneratedMethodAccessor687.invoke(Unknown >>>> Source) >>>> at >>>> java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) >>>> at java.base/java.lang.reflect.Method.invoke(Method.java:566) >>>> at >>>> org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:254) >>>> at >>>> org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler.access$100(StandardControllerServiceInvocationHandler.java:38) >>>> at >>>> org.apache.nifi.controller.service.StandardControllerServiceInvocationHandler$ProxiedReturnObjectInvocationHandler.invoke(StandardControllerServiceInvocationHandler.java:240) >>>> at com.sun.proxy.$Proxy357.setObject(Unknown Source) >>>> at >>>> org.apache.nifi.processors.standard.PutDatabaseRecord.executeDML(PutDatabaseRecord.java:736) >>>> at >>>> org.apache.nifi.processors.standard.PutDatabaseRecord.putToDatabase(PutDatabaseRecord.java:841) >>>> at >>>> org.apache.nifi.processors.standard.PutDatabaseRecord.onTrigger(PutDatabaseRecord.java:487) >>>> at >>>> org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27) >>>> at >>>> org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1173) >>>> at >>>> org.apache.nifi.controller.tasks.ConnectableTask.invoke(ConnectableTask.java:214) >>>> at >>>> org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:117) >>>> at org.apache.nifi.engine.FlowEngine$2.run(FlowEngine.java:110) >>>> at >>>> java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) >>>> at >>>> java.base/java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305) >>>> at >>>> java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305) >>>> at >>>> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) >>>> at >>>> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) >>>> at java.base/java.lang.Thread.run(Thread.java:834) >>>> >>>> >>>> Our test environment is: >>>> NiFi 1.13.2 Cluster with 3 nodes >>>> Postgres 13.2 >>>> openjdk version "11.0.10" >>>> Ubuntu 20.04.1 LTS >>>> >>>> Is this a known issue or an individual fate that makes us despair? >>>> >>>> >>
