Re: java.lang.OutOfMemoryError: Java heap space : NIFI 1.23.2

2024-03-22 Thread Matt Burgess
For completeness, this can also affect the QueryDatabaseTable processors [1]. This will be fixed in the next release(s). Regards, Matt [1] https://issues.apache.org/jira/browse/NIFI-1931 On Tue, Mar 19, 2024 at 10:56 AM wrote: > Hello Joe, > > Thanks for the response. > But I found the

Re: java.lang.OutOfMemoryError: Java heap space : NIFI 1.23.2

2024-03-19 Thread Matt Burgess
Specifically set Fetch Size to something like 1000, by default setting Fetch Size to zero will cause Postgres to fetch the entire ResultSet into memory [1]. We should probably change that default to avoid problems like this and with other drivers (for example, Oracle's default is 10 rows which is

Re: Access instance name

2024-02-13 Thread Matt Burgess
Etienne, What instance name / id are you referring to? On Tue, Feb 13, 2024 at 8:43 AM Etienne Jouvin wrote: > Hello all. > > Just simple question, is there a way to access, from controller service, > to the instance name / id event in not cluster implementation? > > Regards > > Etienne Jouvin

Re: Pattern advice - files on disk into a record field

2024-01-04 Thread Matt Burgess
I think the hard part here is taking a "raw" file like PDF bytes and creating a record in a certain format. For now I think ScriptedReader is your best bet, you can read the entire input stream in as a byte array then return a Record that contains a "bytes" field containing that data. You can

Re: Hardware requirement for NIFI instance

2024-01-04 Thread Matt Burgess
il about the output flowfile : >> >> executesql.query.duration >> 245118 >> executesql.query.executiontime >> 64122 >> executesql.query.fetchtime >> 180996 >> executesql.resultset.index >> 0 >> executesql.row.count >> 14961077 >>

Re: Hardware requirement for NIFI instance

2024-01-04 Thread Matt Burgess
If I remember correctly, the default Fetch Size for Postgresql is to get all the rows at once, which can certainly cause the problem. Perhaps try setting Fetch Size to something like 1000 or so and see if that alleviates the problem. Regards, Matt On Thu, Jan 4, 2024 at 8:48 AM Etienne Jouvin

Re: Extract from jars and nars

2023-12-31 Thread Matt Burgess
Jim, When you say you want to "avoid having to output them to a temp directory", does that include the content repo? If not you can use UnpackContent with a Packaging Type of zip. I tried on both JARs and NARs and it works. Regards, Matt On Sun, Dec 31, 2023 at 12:37 PM James McMahon wrote: >

Re: Exception reason

2023-12-26 Thread Matt Burgess
It looks like we need to call release() from some place(s) where we don't currently. HBase had the same issues [1]. We use this in the StandardMapCacheServer and ListenBeats, are you using either, neither, or both? Regards, Matt [1] https://github.com/netty/netty/issues/12549 On Tue, Dec 26,

Re: How to upgrade from 1.24.0 to 2.0.0-M1

2023-12-06 Thread Matt Burgess
Indeed it looks like someone else has run into this: https://stackoverflow.com/questions/77615582/apache-nifi-2-x-org-eclipse-jetty-http-badmessageexception-400-invalid-sni On Wed, Dec 6, 2023 at 10:05 PM Adam Taft wrote: > David, > > Any chance that the Jetty SNI related information could

Re: NiFi hanging during large sql query

2023-09-02 Thread Matt Burgess
When you said "fetchSize set low", I assume you mean non-zero, a zero will fetch all the rows at once. How did you paginate your query with ExecuteSQLRecord? I was going to suggest GenerateTableFetch in front to paginate the queries for you, but it definitely seems like we should be able to do or

Re: Files in /tmp folder

2023-08-31 Thread Matt Burgess
Can you share your conf/state-management.xml contents? On Mon, Aug 28, 2023 at 8:33 AM Williams, Van wrote: > There are files that are appearing in the /tmp folder on some of our NiFi > Linux hosts. The files all begin with 'file', and they somewhat quickly > fill up that folder (we have an

Re: executescriipt and jython logging

2023-08-22 Thread Matt Burgess
Richard, I'll look into the logging stuff, definitely strange behavior. Are you using ExecuteScript or InvokeScriptedProcessor? I noticed an intermittent failure on InvokeScriptedProcessor in 1.20 [1] but maybe it's present in more of the scripting stuff as they use the same classes to handle the

Re: Transformation: Custom column mapping with SQL Tables (Source > Target Tables)

2023-06-23 Thread Matt Burgess
destination > > On Wed, Jun 21, 2023 at 11:36 AM Matt Burgess wrote: >> >> Kyrindor, >> >> Can you provide an example of the kind of mapping you want to do? It >> sounds like UpdateRecord [1] should work to change input fields to >> output fields. For jo

Re: Transformation: Custom column mapping with SQL Tables (Source > Target Tables)

2023-06-21 Thread Matt Burgess
Kyrindor, Can you provide an example of the kind of mapping you want to do? It sounds like UpdateRecord [1] should work to change input fields to output fields. For joining we offer "enrichment" and "lookup" components that can add fields based on some other field value(s). Regards, Matt [1]

Re: Adjusting FlattenJson output

2023-06-01 Thread Matt Burgess
Jim, I tried to use Jolt for this but I found in the doc that if you try to set an empty array or map to null or the empty string it will retain the empty array or map (no idea why). Since you know the name of the fields (and I assume want to keep the schema intact) you can use

Re: How to determine groovy version

2023-05-13 Thread Matt Burgess
Jim, You can find out in Github [1] or from your installation you can do (substituting your NiFi version in the NAR name): jar -tvf lib/nifi-scripting-nar-1.16.3.nar | grep groovy Regards, Matt [1]

Re: Generalizing QueryRecord to changing inferred CSV headers

2023-04-18 Thread Matt Burgess
support as well, feel free to file a Jira for that improvement if it will help. - Matt On Tue, Apr 18, 2023 at 6:16 PM Matt Burgess wrote: > > Jim, > > QueryRecord uses Apache Calcite under the hood and is thus at the > mercy of the SQL standard (and any additional rules/dialect from &

Re: Generalizing QueryRecord to changing inferred CSV headers

2023-04-18 Thread Matt Burgess
Jim, QueryRecord uses Apache Calcite under the hood and is thus at the mercy of the SQL standard (and any additional rules/dialect from Apache Calcite) so in general you can't select "all except X" or "all except change X to Y". Does it need to be SQL executed against the individual fields? If

Re: Execute DB2 stored procedue

2023-02-28 Thread Matt Burgess
utSQL > > CALL MYPROCEDURE.PROC1('N', ?,?,?,?) > > and I need to supply sql arg attributes... like... > > sql.args.1.type = 1 > sql.args.1.value = not sure what to put here > sql.args.2.type = 4 > sql.args.2.value = not sure what to put here > etc... > > Am I on t

Re: Execute DB2 stored procedue

2023-02-27 Thread Matt Burgess
Stored procedures that take no output parameters and return ResultSets should work fine with ExecuteSQL, but for DBs that allow OUT and INOUT parameters, those won’t make it into the outgoing FlowFile (in either content or attributes). Regards, Matt > On Feb 27, 2023, at 4:19 PM, Dmitry

Re: Best practice for configuring jolt processor?

2023-02-23 Thread Matt Burgess
I'm resurrecting the PR to add a property to specify a file location for the Jolt spec. In the meantime you could maintain the spec as the value of a context variable and point to it in the Jolt Spec property. Then you can share amongst the nodes and maintain the spec in one place. Regards, Matt

Re: PutSQL fragmented transaction error

2023-02-13 Thread Matt Burgess
a transaction? Is it possible with > PutDatabaseRecord? > > Em sex., 10 de fev. de 2023 às 14:29, Matt Burgess > escreveu: >> >> I agree with Chris here about using PutDatabaseRecord instead of the >> Split and PutSQL. PutDatabaseRecord will process all records in a &

Re: PutSQL fragmented transaction error

2023-02-10 Thread Matt Burgess
I agree with Chris here about using PutDatabaseRecord instead of the Split and PutSQL. PutDatabaseRecord will process all records in a FlowFile as a transaction, so in PutDatabaseRecord you can set an AvroReader (to read the records coming out of ExecuteSQL) and the statement type (such as INSERT)

Re: How to cherry pick a specific line from a flowfile?

2023-02-09 Thread Matt Burgess
ach in Groovy to grab line N and avoid loading the entire CSV file into string variable text? On Thu, Feb 9, 2023 at 7:18 PM Matt Burgess <mattyb...@gmail.com> wrote: I’m AFK ATM but Range Sampling was added into the SampleRecord processor (https://issues.apache.org/jira/browse/NIFI-9814),

Re: How to cherry pick a specific line from a flowfile?

2023-02-09 Thread Matt Burgess
I’m AFK ATM but Range Sampling was added into the SampleRecord processor (https://issues.apache.org/jira/browse/NIFI-9814), the Jira doesn’t say which version it went into but it is definitely in 1.19.1+. If that’s available to you then you can just specify “2” as the range and it will only

Re: ECMAScript support missing from ExecuteScript

2023-01-16 Thread Matt Burgess
Did you upgrade the version of Java when you upgraded NiFi? Later versions of Java don’t include the Nashorn (ECMAScript) library, but I believe we added it explicitly, perhaps for the 1.20 release (not at computer right now) Sent from my iPhone > On Jan 16, 2023, at 6:28 PM, Vijay Chhipa

Re: Expected mergerecord performance

2022-12-20 Thread Matt Burgess
Thanks Vijay! I agree those processors should do the trick but there were things in the transformation between input and desired output that I wasn't sure of their origin. If you are setting constants you can use either a Shift or Default spec, if you are moving fields around you can use a Shift

Re: json into a json-enabled DB

2022-12-15 Thread Matt Burgess
Geoffrey, The biggest problem with JSON columns across the board is that the JDBC and java.sql.Types specs don't handle them natively, and NiFi records don't recognize JSON as a particular type, we are only interested in the overall datatype such as String since NiFi records can be in any

Re: Joining multiple tables with left join relation

2022-12-08 Thread Matt Burgess
Michał, There are some options in NiFi that should work for you, we have LookupRecord [1] which does simple enrichment based on a common field (such as id) that acts like a SQL left join, there are a series of articles [2] to explain in more detail. For more complex use cases there are

Re: PrometheusReportingTask authorisation?

2022-11-23 Thread Matt Burgess
Michael, For the authorization use case, I recommend against using the reporting task and instead using the built in endpoint for metrics (see https://issues.apache.org/jira/browse/NIFI-7273 for more details). The NiFi REST API (to include that endpoint) is subject to the authentication and

Re: PutSQL - BatchUpdateException: invalid arguments in call

2022-08-17 Thread Matt Burgess
Sergio, Your email says the flowfiles each contain a record to insert, but PutSQL takes a full SQL statement such as INSERT INTO tableName VALUES ('hello', 'world', 1). If you have a record of data rather than a SQL statement, you can use PutDatabaseRecord for that instead. If you do have SQL

Re: Crash on startup due to Output port issue

2022-08-01 Thread Matt Burgess
Benji, I've built a custom framework NAR [1] that has additional logging to identify which components, process groups, etc. are causing the issue. If you'd like, please feel free to save off your existing framework NAR and replace it with this, then share the relevant section of the log (matching

Re: Most memory-efficient way in NiFi to fetch an entire RDBMS table?

2022-06-22 Thread Matt Burgess
Mike, I recommend QueryDatabaseTableRecord with judicious choices for the following properties: Fetch Size: This should be tuned to return the most number of rows without causing network issues such as timeouts. Can be set to the same value as Max Rows Per Flow File ensuring one fetch per

Re: nifi-content-viewer with mime.type of image/webp

2022-05-23 Thread Matt Burgess
It didn't display in NiFi until NIFI-10027 [1], which has recently been merged. It will be in the upcoming 1.17.0 release (or perhaps a 1.16.3 if the current RC is not released). Regards, Matt [1] https://issues.apache.org/jira/browse/NIFI-10027 On Mon, May 23, 2022 at 11:46 AM Ryan Hendrickson

Re: NiFi to draw samples from very large raw data sets

2022-05-19 Thread Matt Burgess
If you have large FlowFiles and are trying to sample records from each, you can use SampleRecord. It has Interval Sampling, Probabilistic Sampling, and Reservoir Sampling strategies, and I have a PR [1] up to add Range Sampling [2]. Regards, Matt [1] https://github.com/apache/nifi/pull/5878 [2]

Re: Minifi and ssl config on NiFi

2022-04-17 Thread Matt Burgess
MiNiFi is actually alive and well, we just moved it into the NiFi codebase. We’re actively developing a Command-and-Control (C2) capability to remotely update the flow on the agent for example. You can configure MiNiFi agents for SSL over Site-to-Site in order to talk to secure NiFi instances.

Re: initial NiFi 1.15.3 start-up issue on mongo-uri sensitive property complaint

2022-01-26 Thread Matt Burgess
Jeremy, I can't reproduce this on the latest main branch (closest to 1.15.3). What's weird about that error message is that it says 'mongo-uri' is a sensitive property, but it is not. I set up a Parameter Context (PC) with a non-sensitive parameter "muri", set the PC on the root Process Group

Re: PutSQL in combination with ConvertJSONToSQL gives java.sql.SQLException: Invalid column type for Orcale DataType BINARY_DOUBLE

2022-01-20 Thread Matt Burgess
Sven, This is a recently discovered bug, I am still working on characterizing the issue before writing a Jira to describe it. NIFI-9169 [1] has the same cause but is a slightly different issue. So far the issue seems to be with using update key(s) with "Quote Identifiers" set to true. Setting it

Re: Penalty feature of Processor (Disable)

2021-10-25 Thread Matt Burgess
The approach in #1 is already present in a few Put processors like PutHive3QL, the property is named "Rollback on Failure" and takes a boolean value. The docs explain that if set to false, the flowfile is routed to failure, and if true will throw an exception and rollback the session. Check

Re: MiNiFi agent cannot update flow configuration

2021-09-21 Thread Matt Burgess
Tom, Which implementation of the Provenance Repository are you using? If not the VolatileProvenanceRepository, can you try that as a workaround? Also are you using the 1.14.0 version of the C2 server? Regards, Matt On Tue, Sep 21, 2021 at 3:45 PM Tomislav Novosel wrote: > > Hi to all, > > >

Re: nifi 1.11 /metrics and prometheus question

2021-08-09 Thread Matt Burgess
You might be running into NIFI-7379 [1] where the different Prometheus components are writing to the same registries. If you upgrade to a later version of NiFi you should see the correct data. Regards, Matt [1] https://issues.apache.org/jira/browse/NIFI-7379 On Mon, Aug 9, 2021 at 11:35 AM

Re: NiFi Queue Monitoring

2021-07-27 Thread Matt Burgess
ReportingTask with >> > "SELECT * FROM CONNECTION_STATUS WHERE isBackPressureEnabled = true" and >> > the new LoggingRecordSink as you suggested. Everything is working >> > flawlessly now. Thank you again! >> > >> > Scott >> > >&

Re: Persisting a logged message to the bulletin board

2021-07-22 Thread Matt Burgess
Jim, Are you doing the whole series of directories in one call to onTrigger? If so you could keep getting the current time and if you haven't switched directories then you could reissue the bulletin if the elapsed time > 5 mins, then reset the variable to determine the next elapsed time.

Re: NiFi Queue Monitoring

2021-07-21 Thread Matt Burgess
> > On Wed, Jul 21, 2021 at 3:26 PM Matt Burgess wrote: >> >> Scott et al, >> >> There are a number of options for monitoring flows, including >> backpressure and even backpressure prediction: >> >> 1) The REST API for metrics. As you point out, it

Re: NiFi Queue Monitoring

2021-07-21 Thread Matt Burgess
Scott et al, There are a number of options for monitoring flows, including backpressure and even backpressure prediction: 1) The REST API for metrics. As you point out, it's subject to the same authz/authn as any other NiFi operation and doesn't sound like it will work out for you. 2) The

Re: Expression language within scripts in ExecuteScript

2021-07-19 Thread Matt Burgess
Jim, You can apply Chris's solution to ExecuteScript itself, as you can add dynamic properties there and they will evaluate the EL for you. If you set a dynamic property "myNode" to "${ip()}", you should be able to use myNode.evaluateAttributeExpressions().getValue(). Regards, Matt On Mon, Jul

Re: Invalidating InvokeScriptedProcessors when Script Files changes

2021-07-06 Thread Matt Burgess
Dirk, We could look at adding a FileWatcher or something to InvokeScriptedProcessor, but I doubt we'd want to allow re-evaluating the script on the fly, maybe we would just set a flag indicating a change was detected and the next time the processor is started or the script would be evaluated,

Re: Issue when extracting a large dataset into CSV files

2021-05-10 Thread Matt Burgess
Vibath, What is the "Fetch Size" property set to? It looks like PostgreSQL will load all results if Fetch Size is set to zero [1]. Try setting it to 1 or something like that, whatever doesn't use too much memory but doesn't slow down the performance too much. Regards, Matt [1]

Re: Possible problem with DBCPConnectionPool 1.12.1

2021-04-19 Thread Matt Burgess
Carlos, >From the DBCP doc: If maxIdle is set too low on heavily loaded systems it is possible you will see connections being closed and almost immediately new connections being opened. This is a result of the active threads momentarily closing connections faster than they are opening them,

Re: Upsert logic with SQLServer - NIFI Putdatabase

2021-03-11 Thread Matt Burgess
Asmath, Upsert in SQL Server (without NiFi) can be difficult, and even error-prone if concurrency is needed [1]. I suspect that's why it hasn't been attempted in PutDatabaseRecord (well, via the MSSQL adapter(s)) as of yet. I haven't tried it without creating a procedure so I'm not sure if the

Re: Some SimpleRecordSchema questions

2021-03-01 Thread Matt Burgess
Geoffrey, In general you won't need to create your own DataType objects, instead you can use the RecordFieldType methods such as RecordFieldType.ARRAY.getArrayDataType(DataType elementType, boolean elementsNullable).getDataType(). So for an array of ints: myRecordFields.add(new

Re: Stopping processor after MAX number of retries

2021-03-01 Thread Matt Burgess
There's an example template on the Example Dataflow Templates page [1] called Retry_Count_Loop.xml [2], not sure what components it uses though. Regards, Matt [1] https://cwiki.apache.org/confluence/display/NIFI/Example+Dataflow+Templates [2]

Re: ScriptedLookupService

2021-02-25 Thread Matt Burgess
Geoffrey, There are two main types of LookupService implementations used by processors like LookupAttribute and LookupRecord, namely LookupService and LookupService. The former does a single lookup and uses the single returned key. LookupRecord is most often used with LookupService

Re: Issue with QueryRecord failing when data is missing

2021-02-25 Thread Matt Burgess
Jens, What is the Schema Access Strategy set to in your CSVReader? If "Infer Schema" or "Use String Fields From Header", the setting of "Treat First Line As Header" should be ignored as those two options require a header be present anyway. If you know the schema ahead of time you could set it in

Re: some questions about splits

2021-02-24 Thread Matt Burgess
Geoffrey, There's a really good blog by the man himself [1] :) I highly recommend the official blog in general, lots of great posts and many are record-oriented [2] Regards, Matt [1] https://blogs.apache.org/nifi/entry/record-oriented-data-with-nifi [2] https://blogs.apache.org/nifi/ On Wed,

Re: Incremental Fetch in NIFI

2021-02-24 Thread Matt Burgess
Khaja, There are two options in NiFi for incremental database fetch: QueryDatabaseTable and GenerateTableFetch. The former is more often used on a standalone NiFi cluster for single tables (as it does not accept an incoming connection). It generates the SQL needed to do incremental fetching, then

Re: Issue with GenerateTableFetch Processor

2021-02-10 Thread Matt Burgess
John, It should be generating multiple queries with OFFSET, I tried to reproduce in a unit test (using Derby not MySQL) and everything looked fine. I ran it once with 3 rows and a partition size of 2 and got the expected 2 flowfiles (one with 2 rows and one with 1). Then I added 6 rows and ran

Re: Dynamic Attribute Naming

2021-01-06 Thread Matt Burgess
Eric, I don't believe it's possible in NiFi per se, because you'd have to set it via a property, and properties have unique and static names so EL is not evaluated on them. However you can use Groovy with ExecuteScript to do this, check out [1] under the recipe "Add an attribute to a flow file".

Re: schema index out of range

2020-12-02 Thread Matt Burgess
Satish, Can you provide some sample data that causes this issue? Thanks, Matt On Wed, Dec 2, 2020 at 5:18 AM naga satish wrote: > > Hi all, In record readers(CSVreader) when schema strategy is set to > InferSchema, sometimes it keeps on giving error. the error states that index > of

Re: horizontal merge

2020-11-17 Thread Matt Burgess
Geoffrey, Where are the two flowfiles coming from? This use case is often handled in NiFi using LookupRecord with one of the LookupService implementations (REST, RDBMS, CSV, etc.). We don't currently have a mechanism (besides scripting) to do enrichment/lookups from flowfiles. For your script,

Re: PrometheusReportingTask Metrics

2020-11-02 Thread Matt Burgess
David, The documentation for the metrics is in the "help" section of the datapoint definition, if you hit the REST endpoint you can see the descriptions, also they are listed in code [1]. Regards, Matt [1]

Re: GetFile with putsql/executesql

2020-10-28 Thread Matt Burgess
Asmath, GetFile doesn't take an input connection, but if the attribute is going to contain a file to ingest, you can use FetchFile instead. To get an attribute from a database, take a look at LookupAttribute with a SimpleDatabaseLookupService. Depending on the query you were going to execute, you

Re: Run Nifi in IntelliJ to debug?

2020-10-26 Thread Matt Burgess
Matt Burgess wrote: > > Yes, that's a pretty common operation amongst NiFi developers. In > conf/bootstrap.conf there's a section called Enable Remote Debugging > and a commented-out line something like: > > java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,ad

Re: Run Nifi in IntelliJ to debug?

2020-10-26 Thread Matt Burgess
Yes, that's a pretty common operation amongst NiFi developers. In conf/bootstrap.conf there's a section called Enable Remote Debugging and a commented-out line something like: java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005 You can remove the comment from that

Re: Hive NAR not loading because of snappy?

2020-10-13 Thread Matt Burgess
PM, Russell Bateman wrote: > >  No, we don't even use (nor have we ever used) Hive in our flows. It's just > there and we didn't want to modify the NiFi download. Should this not even > happen if we're not using it? > >> On 10/13/20 4:24 PM, Matt Burgess wrote: >> Ug

Re: Hive NAR not loading because of snappy?

2020-10-13 Thread Matt Burgess
uot;$LD_LIBRARY_PATH:/tmp/snappy-1.0.5-libsnappyjava.so" && > /opt/nifi/bin/nifi.sh start ) > > but it's not something we want to do (in case that shared object disappears > from /tmp). > > > On 10/13/20 3:42 PM, Matt Burgess wrote: >> IIRC this is l

Re: Hive NAR not loading because of snappy?

2020-10-13 Thread Matt Burgess
IIRC this is likely a permissions issue, Xerial Snappy tries to unzip the native library to the location pointed to by “java.io.tempdir” which on *nix defaults to /tmp. Does the NiFi user have write access to that directory? If not you can change the Java temp dir or set it specifically for

Re: NiFi V1.9.2 Performance

2020-10-02 Thread Matt Burgess
out the ConsumeKafka_0_10 and ConvertRecord > processors to the ConsumeKafkaRecord_0_10 processor based on feedback in > this chain as well. > > > > With the ConvertRecord processor, we used the kafka.topic attribute as it > was available because the ConsumeKafka processor had

Re: newbie attempting first flow with InvokeHTTP and PutDatabaseRecord

2020-09-29 Thread Matt Burgess
Eric, Depending on how large the JSON content is, you could use JoltTransformJSON to "hoist" the desired data to the top level. Given this example JSON: { "json": { "data": { "value": 3 } } } The spec would be: [ { "operation": "shift", "spec": { "json": {

Re: How to split json subarrays and keep root

2020-09-28 Thread Matt Burgess
Jens, Try ForkRecord [1] with "Mode" set to "Extract" and "Include Parent Fields" set to "true", I think that does what you're looking to do. Regards, Matt [1]

Re: NiFi V1.9.2 Performance

2020-09-24 Thread Matt Burgess
"Field3": > "[&(1)].OutputField2", > > "Sub": { > > "0": { > > "SubField1": "[&(3)].OutputField4", > >

Re: NiFi V1.9.2 Performance

2020-09-24 Thread Matt Burgess
Nathan, If you have multiple JSON messages in one flow file, is it in one large array, or a top-level JSON object with an array inside? Also are you trying to transform each message or the whole thing (i.e. do you need to know about more than one message at a time)? If you have a top-level array

Re: Data performance with FlowFile Repo's RocksDB

2020-09-10 Thread Matt Burgess
You can use a JsonTreeReader set to Infer Schema and use that in JoltTransformRecord. But if your payload is one big JSON object (rather than a top-level array of JSON objects), then you only have one record and should stick to JoltTransformJson. If you do have an array, JoltTransformJson will

Re: Re[4]: access property in ScriptedRecordSetWriter?

2020-08-24 Thread Matt Burgess
rException: Cannot invoke method > get() on null object > > The only "get()" is on line 69 above, def redis_entry = > mapCacheClient.get(mn, > StringSerializer, StringDeserializer) > > Basically, that means that the mapCacheClient is null, but we can't figure >

Re: Re[2]: access property in ScriptedRecordSetWriter?

2020-08-24 Thread Matt Burgess
The formatting got a bit wonky on the code snippet you provided, but if your GroovyRecordSetWriterFactory extends AbstractControllerService, it should have access to the getProperty() method. Try without the context, just "getProperty(CACHE_CLIENT).asControllerService(DistributedMapCacheClient)"

Re: access property in ScriptedRecordSetWriter?

2020-08-21 Thread Matt Burgess
Oops copy paste error, the GroovyScriptedRecordSetWriterFactory has to extend AbstractControllerService Sent from my iPhone > On Aug 21, 2020, at 4:50 PM, David Early wrote: > >  > Matt, > > This is very cool of you, and I feel like this is close, but once again > hanging up on my

Re: access property in ScriptedRecordSetWriter?

2020-08-21 Thread Matt Burgess
Dave, Your snippet is looking good on the inside, but as you want a ScriptedRecordSetWriter you will want to create that instead of a Processor, something like this: class GroovyRecordSetWriter implements RecordSetWriter { private int recordCount = 0 private final OutputStream out

Re: access property in ScriptedRecordSetWriter?

2020-08-21 Thread Matt Burgess
Dave, For ScriptedRecordSetWriter (and all the scripted Controller Services), you provide the properties yourself, rather than (like ExecuteScript) defining dynamic properties and referring to them from the script. I have an example [1] of using Record controller services from

Re: Detect duplicate records

2020-08-15 Thread Matt Burgess
In addition to the SO answer, if you know all the fields in the record, you can use QueryRecord with SELECT DISTINCT field1,field2... FROM FLOWFILE. The SO answer might be more performant but is more complex, and QueryRecord will do the operations in-memory so it might not handle very large

Re: Get all available variables in the InvokeScriptedProcessor

2020-08-11 Thread Matt Burgess
Although this is an "unnatural" use of Groovy (and a conversation much better suited for the dev list :), it is possible to get at a map of defined variables (key and value). This counts on particular implementations of the API and that there is no SecurityManager installed in the JVM so Groovy

Re: Retry logic for rest api - NIFI

2020-07-30 Thread Matt Burgess
Asmath, InvokeHttp routes the original flowfile to a number of different relationships based on things like the status code. For example if you're looking for a 2xx code but want to retry on that for some reason, you'd use the "Original" relationship. If you want a retryable code (5xx) you can

Re: Error with FetchFTP when filename has non-ASCII charachters

2020-07-14 Thread Matt Burgess
Luca, I'm guessing the issue is the same as the one in [1] but it just wasn't fixed for FetchFTP. Please feel free to write an improvement Jira [2] to add this to FetchFTP as well. Regards, Matt [1] https://issues.apache.org/jira/browse/NIFI-4137 [2] https://issues.apache.org/jira/browse/NIFI

Re: Processor Extensibility

2020-07-07 Thread Matt Burgess
This is probably better suited for the dev list (not sure if you're subscribed but please do, BCC'ing users and moving to dev), but the implementations (components and their NARs) are not designed to be subclassed for custom extensions outside the codebase, can you describe your use case (and

Re: Hive_1_1 Processors and Controllers Missing in NiFi 1.11.4

2020-07-07 Thread Matt Burgess
m again. > > > > Thank you, > Harsha > > Sent from Outlook <http://aka.ms/weboutlook> > -- > *From:* Matt Burgess > *Sent:* Tuesday, July 7, 2020 7:05 PM > *To:* users@nifi.apache.org > *Subject:* Re: Hive_1_1 Processors and C

Re: Hive_1_1 Processors and Controllers Missing in NiFi 1.11.4

2020-07-07 Thread Matt Burgess
Harsha, There are two NARs associated with Hive components, nifi-hive-services-api-nar which has the Hive1_1ConnectionPool service (actually an interface, but that's under the hood), and the nifi-hive1_1-nar which has the processors that declare themselves as users of that interface (and the

Re: Enrichment of record data with a REST API

2020-06-29 Thread Matt Burgess
Mike, I think you can use LookupRecord with a RestLookupService to do this. If it's missing features or it otherwise doesn't work for your use case, please let us know and/or write up whatever Jiras you feel are appropriate. Regards, Matt On Mon, Jun 29, 2020 at 4:56 PM Mike Thomsen wrote: > >

Re: Route Attribute - Database down

2020-06-11 Thread Matt Burgess
Although the error attribute can help as a workaround, counting on a text value is probably not the best option (although it's pretty much all we have for now). I wrote up NIFI-7524 [1] to add a "retry" relationship to ExecuteSQL like we have for PutSQL and PutDatabaseRecord. It would route things

Re: Combine Attributes & Content

2020-05-19 Thread Matt Burgess
Dweep, Depending on how complex the content JSON is, you might be able to use ReplaceText to smuggle the attributes into the text, but this can be tricky as you need to match on the opening JSON and the rest, and then replace it with the opening JSON, the attributes, then the rest in order to

Re: groovy script in nifi 1.11 : unable to loasd FastStringService

2020-05-11 Thread Matt Burgess
ly interested in any thoughts about how to proceed (I'm looking at you Payne lol). Regards, Matt On Mon, May 11, 2020 at 6:34 PM Matt Burgess wrote: > > Chris, > > There's definitely something funky going on there, the script doesn't > get the same classloader chain that the Sc

Re: Possible memory leak in PutHive3Streaming?

2020-05-11 Thread Matt Burgess
Martin, There were two memory leaks a while back, one in NiFi code and one in Hive client code (brought in as a dependency for the Hive 3 components). NiFi has fixed their side in 1.9.0 (via NIFI-5841 [1]) and Hive has fixed their side in 3.1.1 (via HIVE-20979 [2]). Until NiFi 1.11.4, we were

Re: ExecuteSQL Unable to resolve union for value

2020-05-06 Thread Matt Burgess
Trevor, What does your table look like and what DB are you using? On Wed, May 6, 2020 at 1:26 PM Trevor Dunn wrote: > > Hi I am using ExecuteSQL processor downstream of GenerateTableFetch to pull > data from a table. However when I run the flow I get error below. I > tried a couple of

Re: Terminate process turns ExecuteScript process invalid on nifi-1.11.3

2020-04-28 Thread Matt Burgess
Carlos, This is indeed a bug, although I'm not sure which change introduced the issue. I have written up NIFI-7404 [1] to describe the problem and cover the fix. The basic issue is that the wrong thread calls the method that adds the script engines, so it can't find the ones that are packaged in

Re: How to read a schema from a ScriptedReader

2020-04-08 Thread Matt Burgess
:) Regards, Matt [1] https://issues.apache.org/jira/browse/NIFI-7343 [2] https://issues.apache.org/jira/browse/NIFI-5115 On Wed, Apr 8, 2020 at 8:19 PM Matt Burgess wrote: > > Jairo, > We should probably move this to the dev list since we're getting into > the NiFi API, but I was

Re: How to read a schema from a ScriptedReader

2020-04-08 Thread Matt Burgess
access other ControllerServices. > > From the RecordReaderFactory I have access to a context that is of type > org.apache.nifi.controller.service.StandardConfigurationContext but with this > I don't know how to access the AvroSchemaRegistry. > > Do you know how I can access the Con

Re: How to read a schema from a ScriptedReader

2020-04-08 Thread Matt Burgess
Jairo, It is possible for a ScriptedReader to use a SchemaRegistry, although it isn't currently as easy as it could be. When the Scripted Reader/Writer components were added [1], much of the reusable code for schema registry processing was in the service implementation NAR, so the aforementioned

Re: orc question

2020-03-26 Thread Matt Burgess
On Thu, Mar 26, 2020 at 12:36 PM Matt Burgess wrote: >> >> Ben, >> >> NiFi doesn't currently have components that support Hive 2, although >> some work has been done on that front [1]. However there are Hive 3 >> components including a PutORC processor that

Re: orc question

2020-03-26 Thread Matt Burgess
Ben, NiFi doesn't currently have components that support Hive 2, although some work has been done on that front [1]. However there are Hive 3 components including a PutORC processor that is kind of a mashup between the old Hive 1-based ConvertAvroToORC and PutHDFS/PutParquet, it does the

Re: NiFi Use case

2020-03-14 Thread Matt Burgess
Sam, This is a common enrichment use case and can be done using LookupRecord with a DatabaseRecordLookupService or SimpleDatabaseLookupService. You can read in one table (Customer) and then lookup the values in the purchase_order table based on the value of customer_id in each record. Does this

Re: Split JSON using an expression to define the PATH

2020-02-18 Thread Matt Burgess
Jairo, IIRC the reason we don't support Expression Language (EL) for the JSONPath expression is because the two DSLs use the same characters in different syntax, such as $. To support both, I believe the user would have to escape the JSONPath $ characters so the NiFi Expression Language lexer

Re: NiFi 1.8.0 to 1.11.1 upgrade issue

2020-02-13 Thread Matt Burgess
Looks like this change was made in 1.9.0: https://issues.apache.org/jira/browse/NIFI-5812 On Thu, Feb 13, 2020 at 11:55 AM David Gallagher wrote: > > @Bryan Bende, to add to this it looks like 'All' was an option in 1.8.0 but > is no longer an option in 1.11.1. This is why it can't be reverted,

  1   2   3   4   5   >