Jim,
The content of the flow file is the body of the outgoing POST, so you
could query provenance for the PostHttp processor, find the associated
flow file(s), and (if the content is still available in the content
repository) retrieve the content. Also the resolved URL for the POST
(after
t attribute to be the flowfile content. How do I set
> that attribute to be my flowfile content?
>
> The challenge I seem to be having is that the service is not a nifi flow.
> How do i feed to it the content body?
>
> On Wed, Nov 8, 2017 at 9:41 AM, Matt Burgess <mattyb
Sally,
I don't think you want a FlowFileFilter here, as your smaller flow
files will remain in the queue while the large enough ones get
processed. Here's a script that I think does what you want it to, but
please let me know if I've misunderstood your intent:
def ffList = session.get(1000)
def
Alberto,
This came up the other day as well, the generated SQL is a Prepared
Statement, which allows the code to use the same statement but then
just set different values based on "parameters". In this case the
values for the parameters are stored in "positional" flow file
attributes for the
Alberto,
What version of NiFi are you using? As of version 1.1.0,
QueryDatabaseTable has a "Normalize Table/Column Names" property that
you can set to true, and it will replace all Avro-illegal characters
with underscores.
Regards,
Matt
On Wed, Dec 6, 2017 at 12:06 PM, Alberto Bengoa
Aruna,
The index and type for Elasticsearch are kinds of partitioning that can
help the users organize data, but definitely help in indexing and searching
data. Types are not always required, but an index is. Imagine you are
trying to store a bunch of tweets from a Twitter feed (or firehose) into
Tina,
What database are you using? Do you have the ability to call CREATE
TABLE IF NOT EXISTS? If so, you could add those to your other sql
statements and send them either to PutSQL individually (but you'll
probably want a Prioritizer on the connection between ExecuteScript ->
PutSQL or an
Leandro,
NiFi does not yet work with Java 9 as far as I know, your version was compiled
against (and is intended to run against) Java 8.
Regards,
Matt
> On Oct 28, 2017, at 3:36 PM, Leandro Lourenço
> wrote:
>
> Hi,
>
> I'm having a strange issue with
Boris,
Besides the (better) answers about building out tests using the mock
framework, here's a "just-in-time" idea that's just crazy enough to
work ;)
If you rename the .nar to a .jar, you could have an
InvokeScriptedProcessor (ISP) whose Module Directory points to the
JAR, then in the script
Boris,
Bryan found the bug, we were setting the results object to null in one
place instead of an empty collection. I've written up NIFI-5230 [1] to
cover it, and issued a PR [2] to fix it.
Regards,
Matt
[1] https://issues.apache.org/jira/browse/NIFI-5230
[2]
For large result sets you’ll want to parallelize the fetch across a multi-node
NiFi cluster. You can do this with GenerateTableFetch -> RemoteProcessGroup (to
the same cluster) -> InputPort -> ExecuteSQL. GenerateTableFetch is like
QueryDatabaseTable but it just generates the SQL to fetch
I left some comments on the PR, will do my best to get it merged if
the changes go in by tomorrow night.
Regards,
Matt
On Sat, Jun 9, 2018 at 3:41 PM Mike Thomsen wrote:
>
> Yves,
>
> Please take a look:
>
> https://github.com/apache/nifi/pull/2778
>
>
Mike,
IIRC you just need your Hadoop Configuration Resources to include a
core-site.xml that has the filesystem set to file:/// or something
similar. Others can/will certainly know better and hopefully
confirm/deny :)
For example, in the unit test PutParquetTest, the core-site.xml has
the
If "customId" is a flow file attribute (and the property supports
expression language), then you just have your braces and $ swapped,
try ${customId} instead of {$customId}
On Wed, Jun 27, 2018 at 4:15 PM Mike Thomsen wrote:
>
> > can you clarify if you mean the NiFi Processor Property "Update
Joe,
Only the first (source) processor needs to be set to Primary Node
Only. Once that happens, the flow files will only proceed down the
flow on the primary node, so step 5 will also only run on the primary
node. In order to redistribute the flow files among the cluster,
you'll want a Remote
Timothy,
I haven't seen anything that can cause this to hang, in the Groovy
source code it might seem to "hang" [1] if there's a crazy large
input; how big are your flow files going into the ExecuteScript
processor? If size is not the issue, then perhaps there's an
assumption about character
Uwe,
I think you are running into either AVRO-2065 [1] or its related issue
AVRO-1891 [2]. Hopefully they will fix it for 1.8.3 and we can
upgrade after it is released. In the meantime, try a schema with just
a union between null and long, then use QueryRecord to filter out all
the records whose
I have an example (albeit a trivial one) of this in my ExecuteScript
Cookbook post [1]. As far as a separate workaround, I can't tell from
the description what you need to do differently than ListFile. It
starts with no state, lists all the files, saves the time of the
newest file in state, then
Pat,
Are you trying to put the whole array in as a single document, or are
you trying to put each element of the array in as a separate document?
If the former, you could use ReplaceText to put the array into a JSON
object. If the latter, you can use SplitJSON to split the array into
individual
Pat,
If you match the entire text, you should be able to do something like
the following as the replacement:
{ "array": $1 }
I didn't try this, but I think it should put the array into a JSON
object. Although an array may be a valid JSON "object", I don't think
Elasticsearch accepts them as
Perhaps a quick(ish) win would be to implement a
DeadlockDetectionReportingTask, where you could specify processor IDs
(or names but that can get dicey) and it would monitor those
processors for incoming connections that all have backpressure
applied, and that the processor has not run for X
Jeremy,
Sorry to hear of your struggles with this; also I can appreciate that
it is difficult to produce representative (but "clean") sample data.
Here's what I tried, perhaps you can comment on how your data/config
is different and I can zero in on what's happening:
Input CSV data (via
Apache Hive 1.2.1, or a version from a vendor? Also are you using
Apache NiFi or a version from a vendor?
On Wed, Jan 10, 2018 at 11:07 AM, Georg Heiler
wrote:
> Hive is 1.2.1
> Joe Witt schrieb am Mi. 10. Jan. 2018 um 17:04:
>>
>> Interesting.
Georg, are you seeing the same stack trace as Jonathan? Or something
different?
On Wed, Jan 10, 2018 at 11:11 AM, Schneider, Jonathan wrote:
> HDF 3.0.0?
>
>
>
> *Jonathan Schneider*
>
> Hadoop/UNIX Administrator, STSC
>
> SCL Health
>
> 17501 W. 98
Austin,
What version of NiFi are you using? I'm wondering if you're running
into [1] (fixed in 1.3.0), or [2] (fixed in 1.4.0), or something else.
You may want to change the types to be "optional", meaning a union
between null and the intended type. So for "PracticeId", try:
{"name":
Jim,
You can use session.create() to create a new FlowFile from within your
script. You don't need a parent, or to transfer input->output, or even
write any content to the output FlowFile for your use case. After
flowFile = session.create(), you can do flowFile =
session.putAttribute(flowFile,
Austin,
Can you create a (non-)materialized view from that query? If so then
QueryDatabaseTable could work. If not, then try QueryRecord after
ExecuteSQL (after adding s.txn_time, I didn't see it in the query), I
think you can add a "max_txn_time" field to the schema and do
something like SELECT
Jim,
In this case I don't think it's as much that the modules aren't being
found, rather that the datetime module in Jython returns
java.sql.Timestamp (Java) objects, rather than Jython/Python datetime
objects, and the former do not support the methods/attributes of the
latter, including
Austin,
This one works for your sample data but doesn't extend to more fields
unless you keep repeating the pattern in the spec:
[
{
"operation": "shift",
"spec": {
"@Place1": "TestArray[0].Place",
"@Holder1": "TestArray[0].Holder",
"@Place2": "TestArray[1].Place",
Jim,
I don't think that's possible because I don't think the user-defined
properties are guaranteed to be in a particular order (insertion order
in the processor config dialog, e.g.), but someone please correct me
if I'm wrong. If true, then we wouldn't be able to have the
user-defined properties
3.2.6.4.0-91
> -DskipTests –e
>
>
> Regards,
> Mike
>
>> -Original Message-
>> From: Matt Burgess [mailto:mattyb...@apache.org]
>> Sent: Monday, February 19, 2018 2:30 PM
>> To: users@nifi.apache.org
>> Subject: Re: PutHiveStreaming NullPoi
Mike,
Joe is correct, in order for Apache NiFi to interact with HDP Hive,
the Hive client dependencies need to be swapped out, as HDP Hive 1.x
components are not 100% compatible with Apache Hive 1.x components.
This can be done (in general) while building NiFi with Maven, by using
a vendor
Jim,
Instead of "import java.util Calendar" try "from java.util import
Calendar". You have the right approach (using calendar) to get the
year, but always be aware of any timezone issues (are the input
timestamps UTC? If not you might need to make an adjustment to the
calendar for the timezone).
Tim,
What version of NiFi are you using? As of 1.1.0 [1], you can specify a
Validation Query on the DBCPConnectionPool, this is used by DBCP to
validate that a connection is "good" before offering it to the client.
For idle/timed-out connections, the validation query should fail and
DBCP should
Tim,
We can certainly expose some DBCP properties as processor properties,
we'd have to enumerate them explicitly since the user-defined ones are
used for the connection. Please feel free to write up an Improvement
Jira [1] to cover whichever properties you'd like to see added to
Márcio,
I believe you are running into this bug [1], which seems it will be
fixed in Jython 2.7.2, but that version has not been released yet
(looks like it's in alpha). When/if they release jython-shaded 2.7.2,
I will upgrade the library in the scripting bundle.
Regards,
Matt
[1]
Tina,
You could use SplitText with a very large value for Line Split Count
(larger than any of your files would contain), and you will get the
same flow file out but with an attribute called "text.line.count" that
contains the number of lines in the file.
Regards,
Matt
On Fri, Dec 22, 2017 at
Kui,
The getControllerService() method requires a controller service (CS)
identifier, not the name (because names are not necessarily unique).
To get the CS by name, you have to get the list of all CSs and match
on the name. I have an example in Groovy on my blog [1], but here is a
similar one
sible nar?
> It is a more generic solution.
>
>
> On July 26, 2018 at 17:23:31, Matt Burgess (mattyb...@apache.org) wrote:
>
> This has been biting a few users lately, not sure when it changed
> exactly, but the Hive NAR uses a version of Snappy that tries to
> extract the nati
re is a place for username and
> password for query that works as required. In PutHive3Streaming there is no
> place to put credentials.
>
> Regards,
> Mike
>
> > -Original Message-
> > From: Matt Burgess
> > Sent: Monday, July 30, 2018 2:45 PM
> >
Mike,
That error usually indicates a Thrift version mismatch, which in this
case is pretty much expected since PutHiveStreaming uses version 1.2.1
and HDP 3.0 uses 3.0.0+. As of NiFi 1.7.0 you can add the
"-Pinclude-hive3" profile in your Maven build and it will add a full
set of Hive
This has been biting a few users lately, not sure when it changed
exactly, but the Hive NAR uses a version of Snappy that tries to
extract the native Snappy library into a location pointed to by the
"java.io.tmpdir" variable, which IIRC is /tmp/. The /tmp
directory sometimes has a noexec
Mike, Ryan, Boris et al,
I'd like to wrap my head around the kinds of use cases y'all have for
provenance data in NiFi: what's good, what's bad, what we need to do
to make things better. Are there questions you want to ask of
provenance that you can't today? Do the DROP events give you what you
nk you!!
>
> On Tue, Aug 7, 2018 at 5:16 PM Matt Burgess wrote:
>>
>> Sounds good, it makes the underlying code a bit more complicated but I see
>> from y’all’s points that a “separate” processor is a better user experience.
>> I’m knee deep in it as we speak, hope to have
Bob,
Unless you already have SQL in your flow files, I always recommend
PutDatabaseRecord [1] over PutSQL. The former is basically a mashup of
ConvertJSONToSQL -> PutSQL, but takes in data in any format supported by
our record readers (CSV, Avro, XML, JSON, etc.) and takes care of all the
SQL
e done in the past is dumping data either to HDFS or local
> disk and then using efficient tools to do this job using bulk load tools,
> specific to your target platform. Sqoop can also do it to some degree but
> has a long list of limitations then it comes down to exports.
>
>
>
t;> input/output upfront and just want to run SQL query and get whatever results
>> back. It just adds an extra step that will be subject to break/support.
>>
>> Similar to Kafka processors, it is nice to have an option of record-based
>> processor vs. message oriented pr
ote:
>
> I'd really like to see the Record suffix on the processor for
> discoverability, as already mentioned.
>
> Andrew
>
>> On Tue, Aug 7, 2018, 2:16 PM Matt Burgess wrote:
>> Yeah that's definitely doable, most of the logic for writing a
>> ResultSet
sh
> it anywhere? Thanks.
>
> Vitaly Krivoy
>
>
>
> From: Matt Burgess
> Sent: Tuesday, July 17, 2018 2:15 PM
> To: users@nifi.apache.org
> Subject: Re: NiFi processor to execute a Java class
>
>
>
> In addition to Andy’s great links and references, for the
Do you have a Split processor upstream? If so, is the setting of
Support Fragmented Transactions in PutSQL set to true? That
combination will have PutSQL try to find all the flow files with the
same "fragment.id" attribute, and will only proceed if it gets
"fragment.count" of them (all the flow
I don't think you have to install Hadoop on Windows in order to get it
to work, just the winutils.exe and I guess put it wherever it's
looking for it (that might be configurable via an environment variable
or something).
There are pre-built binaries [1] for various versions of Hadoop, even
though
"type": "string",
>
>"doc": "Type inferred from '\"M\"'"
>
> }, {
>
>"name": "producer_datetime_last_chg",
>
>"type": &quo
Walter,
If you're looking to distribute database fetching among a cluster,
then GenerateTableFetch is the right choice (over QueryDatabaseTable).
As of NiFi 1.2.0 (via NIFI-2881 [1]), GenerateTableFetch accepts
incoming flow files, the capability was added in response to exactly
the use case you
Boris,
Historically the Hive JDBC drivers (at least around 1.2.x) did not
support some of the JDBC API methods called from ExecuteSQL/PutSQL,
namely setQueryTimeout(), executeBatch(), etc. Also the column names
are returned from the metadata with the table name prepended, so we'd
need special
Juan,
What version of NiFi are you using? I tried with the latest master,
using a GenerateFlowFile (where I set the attribute using the now()
expression) into JoltTransformJSON with your spec above, and it works
fine. Are you sure there's not a double-quote missing somewhere, or
something extra
I'm definitely interested in supporting a record-aware version as well
(I wrote the Jira up last year [1] but haven't gotten around to
implementing it), however I agree with Peter's comment on the Jira.
Since ExecuteSQL is an oft-touched processor, if we had two processors
that only differed in
t;
> On Fri, Aug 10, 2018 at 4:55 PM Matt Burgess wrote:
>>
>> Boris et al,
>>
>> I put up a PR [1] to add ExecuteSQLRecord and QueryDatabaseTableRecord
>> under NIFI-4517, in case anyone wants to play around with it :)
>>
>> Regards,
>> Matt
>
You can use ValidateRecord (with a CSVReader and JSONRecordSetWriter,
and another "invalid CSV Reader" for invalid records) for that, then
SplitRecord if you need it. However if you can describe your
downstream flow, perhaps we can help you avoid the need to split the
records at all (unless you
SplitRecord to create batches for
> the large input files for efficiency of processing.
>
> On Mon, Aug 13, 2018 at 6:02 PM, Matt Burgess wrote:
>>
>> You can use ValidateRecord (with a CSVReader and JSONRecordSetWriter,
>> and another "invalid CSV Reader" for
Bob,
InferAvroSchema can infer types like boolean, integer, long, float, double,
and I believe for JSON can correctly descend into arrays and nested
maps/structs/objects. Here is an example record from NiFi provenance data
that has most of those covered (except bool and float/double, but you can
Rick,
Are you using the version of the MySQL driver in NiFi that corresponds
to your 5.7.23 server? That error seems like it'd be coming from the
database rather than the processor. I have seen issues (although not
this one) with older drivers against newer DBs.
Another thing to try might be to
ch to consider flow files
> attributes + dynamic properties configured at processor level.
>
> Pierre
>
> 2018-08-08 17:06 GMT+02:00 Matt Burgess :
>>
>> Vitaly,
>>
>> I am indeed back from vacation and I started on the ExecuteClass
>> processor but neve
essor I can use
>> where I can just fill out this list myself?
>>
>> Perhaps these are very basic questions, by the way. Everybody here is
>> learning how to work with NiFi and from an admin perspective I'm trying to
>> provide some guiding on best practices for common
Dave,
At one point (perhaps it is still true), in order to have a default
value its type had to be the first type in the array. For optional
fields, the default is null so try inverting your type arrays to match
the way they're done in the input schema; namely, ["null", "string"]
rather than
Jim,
You can use UpdateRecord for this, your input schema would have "last"
and "first" in it (and I think you can have an optional "myKey" field
so you can use the same schema for the writer), and the output schema
would have all three fields in it. Then you'd set the Replacement
Value Strategy
rencing the three fields explicitly, telling it to put my new
> field(s) after one of those where ever it may be in the object, and
> indicating it should then include all other keys/values in the object?
>
> On Fri, Aug 17, 2018 at 4:24 PM, Matt Burgess wrote:
>>
>> Jim,
&
statements. I also used Oracle SQL Developer and DBeaver to connect
> using jdbc drivers. I assumed it would be similar with NiFi.
>
> On Thu, Aug 16, 2018 at 12:38 PM Matt Burgess wrote:
>>
>> Boris,
>>
>> Historically the Hive JDBC drivers (at least around 1.2
In addition to Andy’s great links and references, for the scripting stuff I
wrote a little cookbook to help with the NiFi API in the scripting processors
[1], and I play around with the scripting stuff quite a bit and keep a blog
with fun stuff I experiment with [2].
Also your use case is
There’s a PutElasticsearchHttpRecord that should give you the best of both
worlds, the Record API to convert data types and using the REST API.
> On Jul 16, 2018, at 2:59 PM, Mike Thomsen wrote:
>
> With PutElasticsearchHttp, you'll have to define the field in advance as a
> date one because
Katrina,
During one execution of the CDC processor, it will handle all
available CDC events. State is updated according to the setting of the
State Update Interval value. By default it is set to 0, which means
state will only be updated when an execution of the processor is
completed (or the
Katrina,
What is "Retrieve All Records" set to? It defaults to true, which will
ignore the Initial Binlog Filename/Position settings. If you want to
"skip ahead" to bin.58, you should clear the processor's state
(right-click on the processor and choose View State, then "Clear
state"), set
Add the following to the end of your shift spec:
"*": "&"
This (at the root) matches any key not already matched and puts it in
the output at the same location.
Regards,
Matt
On Tue, Sep 4, 2018 at 11:39 AM l vic wrote:
>
> Hi,
> I want to "flatten" the following object:
> {
> "id": 0,
>
If you must split it, you should be able to use SplitJson with a
JSONPath of "$.mylist". Note that your example is not actually JSON,
it would have to be enclosed as an object with {}.
Regards,
Matt
On Thu, Sep 6, 2018 at 2:53 PM l vic wrote:
>
> I have array:
> "mylist": [{
> "id": 10,
>
Jim,
If you only need to read the contents of the flow file and not modify
them, then you don't have to use session.write() to send the original
content out, you can just use session.read(FlowFile,
InputStreamCallback) instead.
I'm not sure why that write fails sometimes and not others, I
Alexander,
It sounds like you'd like to see the Batch Duration capability from
ExecuteProcess added to ExecuteStreamCommand, please feel free to
write a Jira case [1] for this improvement.
In the meantime, I second Boris's thought on using Groovy to launch
your script, it's much more integrated
Scott,
I'm assuming you're trying to create a single (possibly sparse) table
for all the incoming JSON records, and that you do not know the total
set of fields ahead of time. If you do know the fields ahead of time,
you can create the table ahead of time and use the JoltTransformJSON
processor
Mike,
There is a nifi-atlas-bundle in NiFi with a NAR that includes the
ReportLineageToAtlas reporting task, but IIRC it is so large that it
is not included in the default assembly. Instead there is a
"include-atlas" profile that can be activated when building the
assembly, and that should
very
> powerful, and its power should be fully available to this kind of
> processor, IMHO.
>
> What do you think?
>
> Thank you,
>
> Marcio
>
>
> On Tuesday, March 13, 2018 8:56 PM, Matt Burgess <mattyb...@apache.org>
> wrote:
>
>
> Raman,
>
> Original
Juan,
Glad to hear of your interest in this! Strangely, it seems to be a
popular feature (see the existing Jira [1]) but so far there hasn't
been a PR to address it. This has been done for QueryDatabaseTable,
and one workaround is to use QueryDatabaseTable without specifying a
Maximum Value
Phoenix.
>
> Thanks
>
> On Wed, 14 Mar 2018 at 15:50 Matt Burgess <mattyb...@apache.org> wrote:
>>
>> Juan,
>>
>> We've had to do similar things for Oracle [1], so there is precedence,
>> please feel free to create a JIRA to fix it, thanks!
>>
>
Raman,
Originally, we stored state only based on column names, which could
obviously cause problems when you have two different tables with the same
column name. However this was because the original DB Fetch processors
(QueryDatabaseTable, GenerateTableFetch, e.g.) did not accept incoming flow
Mike,
I can't reproduce this, I use the same DDL and Avro schema, with data
coming in from the SiteToSiteProvenanceReportingTask and going to
Postgres, and it works fine. What version of NiFi are you using?
Regards,
Matt
On Thu, Apr 5, 2018 at 3:05 PM, Mike Thomsen
Nick,
Here's a example slightly modified from my cookbook example (I'm not
sure that works with the array brackets, might need to use .get()
instead), this one is a full working script to log each attribute:
var flowFile = session.get()
if (flowFile != null) {
var attrs =
Jim,
Just to confirm Joe's comments, we were using 2.7.0 but then bumped it
up to 2.7.1 as there was at least one method os.getpid() that hadn't
been implemented [1]. However, 2.7.1 apparently has multithreading
issues [2] that are causing some folks headaches as well.
Regards,
Matt
[1]
@gmail.com> wrote:
>>
>> 1.5
>>
>> Thanks,
>>
>> Mike
>>
>> On Thu, Apr 5, 2018 at 3:40 PM, Matt Burgess <mattyb...@apache.org> wrote:
>>>
>>> Mike,
>>>
>>> I can't reproduce this, I use the same DDL and Avr
Dejan,
Are there any comments in your Hive "script"? If so I believe you need to
remove them [1]. When I've seen that issue, it's always been because of
comments before the SET line.
Regards,
Matt
[1] https://issues.apache.org/jira/browse/HIVE-1166
On Mon, Apr 16, 2018 at 12:29 PM, Krstic,
If you are using a version of NiFi that has ExecuteScript, you could
write a script that reads in the properties from that file and create
flow file attributes from them, then for downstream processors that
accept incoming flow files and evaluate Expression Language against
them, you'll be able to
Binary Avro format. ". Is this format different than what ConvertCsvToAvro
> writes? Because same flow is working with ValidateRecord(write it to csv)
> +ConvertCsvToAvro processor.
>
> Thanks,
> Mohit
>
> -Original Message-
> From: Matt Burgess <mattyb
Mohit,
Can you share the config for your ConvertAvroToORC processor? Also, by
"CreateHiveTable", do you mean ReplaceText (to set the content to the
hive.ddl attribute formed by ConvertAvroToORC) -> PutHiveQL (to
execute the DDL)? If not, are you using a custom processor or
ExecuteStreamCommand
Mausam,
You could use PutFile to store off the Category CSV, then you can use
LookupRecord with either a CSVRecordLookupService or a
SimpleCsvLookupService, the former is for fetching multiple fields
from the lookup, the latter is for a single value lookup. You'll also
use a CSVReader to read in
Juan,
Yes the current behavior is to move the entire input flowfile to
failure if any errors occur. Some other record-aware processors create
separate flow files for failed and successful records, but
PutElasticsearchHttpRecord does not (yet) do that. Please feel free to
write a Jira for this
Paul,
We use Apache DBCP to pool the connections, and one of its major
features is to keep connections open for reuse, to avoid the overhead
of establishing new connections (and shutting them down) quickly. In
your case you are not creating/closing them quickly but rather every 5
mins. If they
Viking,
This is indeed a bug, the code (copied from ConvertAvroToORC and its
util classes) still expects Avro objects but PutORC uses NiFi Record
objects. Please file a Jira, I will look into this immediately.
Regards,
Matt
On Sat, Oct 6, 2018 at 10:17 AM Ei i wrote:
>
> Hi,
>
> I have been
Faisal,
There has been some related work [1] but has not yet been completed
yet, and appears to have gone idle.
Regards,
Matt
[1] https://github.com/apache/nifi/pull/2507
On Wed, Oct 24, 2018 at 5:48 AM Faisal Durrani wrote:
>
> Hi,
>
> Please let me know if there are any plans to fix the
>
mValue column but how can I use
> it to schedule cron job? I know it's possible to schedule cron from UI but
> how can i do it based on the value of attribute?
> Thank you again,
> V.
>
> On Mon, Oct 29, 2018 at 12:39 PM Matt Burgess wrote:
>>
>> Victor,
>>
>>
I’m not at my computer at the moment but I wrote 2 Jiras and put up 2 PRs to
add these features, hopefully they will make the next release. I can get the
links later today.
> On Oct 31, 2018, at 10:27 AM, Shawn Weeks wrote:
>
> Currently GenerateTableFetch doesn't set any attributes to
You can use QueryDatabaseTable (QDT) for this, you'd set your
"event_time" column as the "Maximum Value Column(s)" property in the
processor. The first time QDT executes, it will fetch all the rows
(since it has not seen event_time before), then it will keep track of
the largest value of
e other parts of it
so you wouldn't need to update a single row in an external database
table?
Regards,
Matt
On Mon, Oct 29, 2018 at 12:36 PM l vic wrote:
>
> What if have only one row and update the values in it? Will QDT fetch updates?
> Thank you,
> Victor
>
>
> On Mon, Oct
Dann,
Looks like you're running into NIFI-5652 [1], I'll try to get a fix up soon.
Regards,
Matt
[1] https://issues.apache.org/jira/browse/NIFI-5652
On Tue, Nov 13, 2018 at 10:06 AM Dann wrote:
>
> It looks like to me that the LogMessage processor doesn't work when the log
> level isn't
Building off Pierre’s idea, you could set the Penalty Duration to 0 so the the
flow files go immediately to LogMessage, then send along to ExecuteScript to
“re-penalize” the file
Regards,
Matt
> On Nov 9, 2018, at 11:39 AM, Colin Williams
> wrote:
>
> Hi Pierre,
>
> Thanks for explaining.
201 - 300 of 474 matches
Mail list logo