Update: session.remove(newFiles) does not work. I filed
https://issues.apache.org/jira/browse/NIFI-3205

On Thu, Dec 15, 2016 at 11:05 AM, Alan Jackoway <al...@cloudera.com> wrote:

> I am getting the successfully checkpointed message.
>
> I think I figured this out. Now we have to decide whether it's an issue in
> nifi or an issue in our code.
>
> This flow has a process that takes large zip files, unzips them, and does
> some processing of the files. I noticed that the disk space thing seems to
> go up fastest when there is a large file that is failing in the middle of
> one of these steps. I then suspected that something about the way we were
> creating new flow files out of the zips was the problem.
>
> I simulated what we were doing with the following processor in a new nifi
> 1.1. The processor takes an input file, copies it 5 times (to simulate
> unzip / process), then throws a runtime exception. I then wired a
> GenerateFlowFile of 100KB to it. I noticed the following characteristics:
> * Each time it ran, the size of the content repository went up exactly
> 500KB.
> * When I restarted the nifi, I got the messages about unknown files in the
> FileSystemRepository.
>
> So basically what this boils down to is: who is responsible to remove
> files from a session when a failure occurs? Should we be doing that (I will
> test next that calling session.remove before the error fixes the problem)
> or should the session keep track of new flow files that it created. We
> assumed the session would do so because the session yells at us if we fail
> to give a transport relationship for one of the files.
>
> Thanks for all the help with this. I think we are closing in on the point
> where I have either a fix or a bug filed or both.
>
> Test processor I used:
> // Copyright 2016 (c) Cloudera
> package com.cloudera.edh.nifi.processors.bundles;
>
> import com.google.common.collect.Lists;
>
> import java.io.IOException;
> import java.io.InputStream;
> import java.io.OutputStream;
> import java.util.List;
>
> import org.apache.nifi.annotation.behavior.InputRequirement;
> import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
> import org.apache.nifi.flowfile.FlowFile;
> import org.apache.nifi.processor.AbstractProcessor;
> import org.apache.nifi.processor.ProcessContext;
> import org.apache.nifi.processor.ProcessSession;
> import org.apache.nifi.processor.exception.ProcessException;
> import org.apache.nifi.processor.io.InputStreamCallback;
> import org.apache.nifi.processor.io.OutputStreamCallback;
> import org.apache.nifi.stream.io.StreamUtils;
>
> /**
>  * Makes 5 copies of an incoming file, then fails and rolls back.
>  */
> @InputRequirement(value = Requirement.INPUT_REQUIRED)
> public class CopyAndFail extends AbstractProcessor {
>   @Override
>   public void onTrigger(ProcessContext context, ProcessSession session)
>       throws ProcessException {
>     FlowFile inputFile = session.get();
>     if (inputFile == null) {
>       context.yield();
>       return;
>     }
>     final List<FlowFile> newFiles = Lists.newArrayList();
>
>     // Copy the file 5 times (simulates us opening a zip file and
> unpacking its contents)
>     for (int i = 0; i < 5; i++) {
>       session.read(inputFile, new InputStreamCallback() {
>         @Override
>         public void process(InputStream inputStream) throws IOException {
>           FlowFile ff = session.create(inputFile);
>           ff = session.write(ff, new OutputStreamCallback() {
>             @Override
>             public void process(final OutputStream out) throws IOException
> {
>               StreamUtils.copy(inputStream, out);
>             }
>           });
>           newFiles.add(ff);
>         }
>       });
>     }
>
>     // THIS IS WHERE I WILL PUT session.remove TO VERIFY THAT WORKS
>
>     // Simulate an error handling some file in the zip after unpacking the
> rest
>     throw new RuntimeException();
>   }
> }
>
>
> On Wed, Dec 14, 2016 at 9:23 PM, Mark Payne <marka...@hotmail.com> wrote:
>
>> I'd be very curious to see if changing the limits addresses the issue.
>> The OOME can certainly be an issue, as well. Once that gets thrown
>> anywhere
>> in the JVM, it's hard to vouch for the stability of the JVM at all.
>>
>> Seeing the claimant count drop to 0 then back up to 1, 2, and down to 1,
>> 0 again
>> is pretty common. The fact that you didn't see it marked as destructible
>> is interesting.
>> Around that same time, are you seeing log messages indicating that the
>> FlowFile repo
>> is checkpointing? Would have the words "Successfully checkpointed
>> FlowFile Repository"
>> That should happen every 2 minutes, approximately.
>>
>>
>> On Dec 14, 2016, at 8:56 PM, Alan Jackoway <al...@cloudera.com<mailto:ala
>> n...@cloudera.com>> wrote:
>>
>> I agree the limits sound low and will address that tomorrow.
>>
>> I'm not seeing FileNotFound or NoSuchFile.
>>
>> Here's an example file:
>> grep 1481763927251 logs/nifi-app.log
>> 2016-12-14 17:05:27,277 DEBUG [Timer-Driven Process Thread-36]
>> o.a.n.c.r.c.StandardResourceClaimManager Incrementing claimant count for
>> StandardResourceClaim[id=1481763927251-1, container=default, section=1]
>> to 1
>> 2016-12-14 17:05:27,357 DEBUG [Timer-Driven Process Thread-2]
>> o.a.n.c.r.c.StandardResourceClaimManager Incrementing claimant count for
>> StandardResourceClaim[id=1481763927251-1, container=default, section=1]
>> to 2
>> 2016-12-14 17:05:27,684 DEBUG [Timer-Driven Process Thread-36]
>> o.a.n.c.r.c.StandardResourceClaimManager Decrementing claimant count for
>> StandardResourceClaim[id=1481763927251-1, container=default, section=1]
>> to 1
>> 2016-12-14 17:05:27,732 DEBUG [Timer-Driven Process Thread-2]
>> o.a.n.c.r.c.StandardResourceClaimManager Decrementing claimant count for
>> StandardResourceClaim[id=1481763927251-1, container=default, section=1]
>> to 0
>> 2016-12-14 17:05:27,909 DEBUG [Timer-Driven Process Thread-14]
>> o.a.n.c.r.c.StandardResourceClaimManager Incrementing claimant count for
>> StandardResourceClaim[id=1481763927251-1, container=default, section=1]
>> to 1
>> 2016-12-14 17:05:27,945 DEBUG [Timer-Driven Process Thread-14]
>> o.a.n.c.r.c.StandardResourceClaimManager Incrementing claimant count for
>> StandardResourceClaim[id=1481763927251-1, container=default, section=1]
>> to 2
>> 2016-12-14 17:14:26,556 DEBUG [Timer-Driven Process Thread-14]
>> o.a.n.c.r.c.StandardResourceClaimManager Decrementing claimant count for
>> StandardResourceClaim[id=1481763927251-1, container=default, section=1]
>> to 1
>> 2016-12-14 17:14:26,556 DEBUG [Timer-Driven Process Thread-14]
>> o.a.n.c.r.c.StandardResourceClaimManager Decrementing claimant count for
>> StandardResourceClaim[id=1481763927251-1, container=default, section=1]
>> to 0
>>
>> This nifi-app.log covers a period when the nifi only handled two sets of
>> files for a total of maybe 10GB uncompressed. Content repository went over
>> 100GB in that time. I checked a few content repository files, and they all
>> had similar patterns - claims hit 0 twice - once around 17:05 and once
>> around 17:14, then nothing. I brought down the nifi around 17:30.
>>
>> During that time, we did have a processor hitting OutOfMemory while
>> unpacking a 1GB file. I'm adjusting the heap to try to make that succeed
>> in
>> case that was related.
>>
>> On Wed, Dec 14, 2016 at 8:32 PM, Mark Payne <marka...@hotmail.com<mailto:
>> marka...@hotmail.com>> wrote:
>>
>> OK, so these are generally the default values for most linux systems.
>> These are a little low,
>> though for what NiFi recommends and often needs. With these settings, you
>> can easily run
>> out of open file handles. When this happens, trying to access a file will
>> return a FileNotFoundException
>> even though the file exists and permissions all look good. As a result,
>> NiFi may be failing to
>> delete the data simply because it can't get an open file handle.
>>
>> The admin guide [1] explains the best practices for configuring these
>> settings. Generally, after updating
>> these settings, I think you have to logout of the machine and login again
>> for the changes to take effect.
>> Would recommend you update these settings and also search logs for
>> "FileNotFound" as well as
>> "NoSuchFile" and see if that hits anywhere.
>>
>>
>> [1] http://nifi.apache.org/docs/nifi-docs/html/administration-
>> guide.html#configuration-best-practices
>>
>>
>> On Dec 14, 2016, at 8:25 PM, Alan Jackoway <al...@cloudera.com<mailto:ala
>> n...@cloudera.com><mailto:ala
>> n...@cloudera.com<mailto:n...@cloudera.com>>> wrote:
>>
>> We haven't let the disk hit 100% in a while, but it's been crossing 90%.
>> We
>> haven't seen the "Unable to checkpoint" message in the last 24 hours.
>>
>> $ ulimit -Hn
>> 4096
>> $ ulimit -Sn
>> 1024
>>
>> I will work on tracking a specific file next.
>>
>>
>> On Wed, Dec 14, 2016 at 8:17 PM, Alan Jackoway <al...@cloudera.com
>> <mailto:al...@cloudera.com><mailto:
>> al...@cloudera.com<mailto:al...@cloudera.com>>> wrote:
>>
>> At first I thought that the drained messages always said 0, but that's not
>> right. What should the total number of claims drained be? The number of
>> flowfiles that made it through the system? If so, I think our number is
>> low:
>>
>> $ grep "StandardResourceClaimManager Drained" nifi-app_2016-12-14*  | grep
>> -v "Drained 0" | awk '{sum += $9} END {print sum}'
>> 25296
>>
>> I'm not sure how to get the count of flowfiles that moved through, but I
>> suspect that's low by an order of magnitude. That instance of nifi has
>> handled 150k files in the last 6 hours, most of which went through a
>> number
>> of processors and transformations.
>>
>> Should the number of drained claims correspond to the number of flow files
>> that moved through the system?
>> Alan
>>
>> On Wed, Dec 14, 2016 at 6:59 PM, Alan Jackoway <al...@cloudera.com
>> <mailto:al...@cloudera.com><mailto:
>> al...@cloudera.com<mailto:al...@cloudera.com>>> wrote:
>>
>> Some updates:
>> * We fixed the issue with missing transfer relationships, and this did
>> not go away.
>> * We saw this a few minutes ago when the queue was at 0.
>>
>> What should I be looking for in the logs to figure out the issue?
>>
>> Thanks,
>> Alan
>>
>> On Mon, Dec 12, 2016 at 12:45 PM, Alan Jackoway <al...@cloudera.com
>> <mailto:al...@cloudera.com>
>> <mailto:al...@cloudera.com>>
>> wrote:
>>
>> In case this is interesting, I think this started getting bad when we
>> started hitting an error where some of our files were not given a transfer
>> relationship. Maybe some combination of not giving flow files a
>> relationship and the subsequent penalization is causing the problem.
>>
>> On Mon, Dec 12, 2016 at 12:16 PM, Alan Jackoway <al...@cloudera.com
>> <mailto:al...@cloudera.com>
>> <mailto:al...@cloudera.com>>
>> wrote:
>>
>> Everything is at the default locations for these nifis.
>>
>> On one of the two machines, I did find log messages like you suggested:
>> 2016-12-11 08:00:59,389 ERROR [pool-10-thread-1]
>> o.a.n.c.r.WriteAheadFlowFileRepository Unable to checkpoint FlowFile
>> Repository due to java.io.FileNotFoundException:
>> ./flowfile_repository/partition-14/3169.journal (No space left on
>> device)
>>
>> I added the logger, which apparently takes effect right away. What am I
>> looking for in this logs? I see a lot of stuff like:
>> 2016-12-12 07:19:03,560 DEBUG [Timer-Driven Process Thread-24]
>> o.a.n.c.r.c.StandardResourceClaimManager Decrementing claimant count
>> for StandardResourceClaim[id=1481555893660-3174, container=default,
>> section=102] to 0
>> 2016-12-12 07:19:03,561 DEBUG [Timer-Driven Process Thread-31]
>> o.a.n.c.r.c.StandardResourceClaimManager Decrementing claimant count
>> for StandardResourceClaim[id=1481555922818-3275, container=default,
>> section=203] to 191
>> 2016-12-12 07:19:03,605 DEBUG [Timer-Driven Process Thread-8]
>> o.a.n.c.r.c.StandardResourceClaimManager Decrementing claimant count
>> for StandardResourceClaim[id=1481555880393-3151, container=default,
>> section=79] to 142
>> 2016-12-12 07:19:03,624 DEBUG [Timer-Driven Process Thread-38]
>> o.a.n.c.r.c.StandardResourceClaimManager Decrementing claimant count
>> for StandardResourceClaim[id=1481555872053-3146, container=default,
>> section=74] to 441
>> 2016-12-12 07:19:03,625 DEBUG [Timer-Driven Process Thread-25]
>> o.a.n.c.r.c.StandardResourceClaimManager Decrementing claimant count
>> for StandardResourceClaim[id=1481555893954-3178, container=default,
>> section=106] to 2
>> 2016-12-12 07:19:03,647 DEBUG [Timer-Driven Process Thread-24]
>> o.a.n.c.r.c.StandardResourceClaimManager Decrementing claimant count
>> for StandardResourceClaim[id=1481555893696-3175, container=default,
>> section=103] to 1
>> 2016-12-12 07:19:03,705 DEBUG [FileSystemRepository Workers Thread-1]
>> o.a.n.c.r.c.StandardResourceClaimManager Drained 0 destructable claims
>> to []
>>
>> What's puzzling to me is that both of these machines have > 100GB of
>> free space, and I have never seen the queued size go above 20GB. It seems
>> to me like it gets into a state where nothing is deleted long before it
>> runs out of disk space.
>>
>> Thanks,
>> Alan
>>
>> On Mon, Dec 12, 2016 at 9:13 AM, Mark Payne <marka...@hotmail.com<mailto:
>> marka...@hotmail.com><mailto:m
>> arka...@hotmail.com<mailto:arka...@hotmail.com>>>
>> wrote:
>>
>> Alan,
>>
>> Thanks for the thread-dump and the in-depth analysis!
>>
>> So in terms of the two tasks there, here's a quick explanation of what
>> each does:
>> ArchiveOrDestroyDestructableClaims - When a Resource Claim (which
>> maps to a file on disk) is no longer referenced
>> by any FlowFile, it can be either archived or destroyed (depending on
>> whether the property in nifi.properties has archiving
>> enabled).
>> DestroyExpiredArchiveClaims - When archiving is enabled, the Resource
>> Claims that are archived have to eventually
>> age off. This task is responsible for ensuring that this happens.
>>
>> As you mentioned, in the Executor, if the Runnable fails it will stop
>> running forever, and if the thread gets stuck, another will
>> not be launched. Neither of these appears to be the case. I say this
>> because both of those Runnables are wrapped entirely
>> within a try { ... } catch (Throwable t) {...}. So the method will
>> never end Exceptionally. Also, the thread dump shows all of the
>> threads created by that Thread Pool (those whose names begin with
>> "FileSystemRepository Workers Thread-") in WAITING
>> or TIMED_WAITING state. This means that they are sitting in the
>> Executor waiting to be scheduled to do something else,
>> so they aren't stuck in any kind of infinite loop or anything like
>> that.
>>
>> Now, with all of that being said, I have a theory as to what could
>> perhaps be happening :)
>>
>> From the configuration that you listed below, it shows that the
>> content repository is located at ./content_repository, which is
>> the default. Is the FlowFile Repository also located at the default
>> location of ./flowfile_repository? The reason that I ask is this:
>>
>> When I said above that a Resource Claim is marked destructible when no
>> more FlowFiles reference it, that was a bit of a
>> simplification. A more detailed explanation is this: when the FlowFile
>> Repository is checkpointed (this happens every 2 minutes
>> by default), its Write-Ahead Log is "rolled over" (or "checkpointed"
>> or "compacted" or however you like to refer to it). When this
>> happens, we do an fsync() to ensure that the data is stored safely on
>> disk. Only then do we actually mark a claim as destructible.
>> This is done in order to ensure that if there is a power outage and a
>> FlowFile Repository update wasn't completely flushed to disk,
>> that we can recover. For instance, if the content of a FlowFile
>> changes from Resource Claim A to Resource Claim B and as a result
>> we delete Resource Claim A and then lose power, it's possible that the
>> FlowFile Repository didn't flush that update to disk; as a result,
>> on restart, we may still have that FlowFile pointing to Resource Claim
>> A which is now deleted, so we would end up having data loss.
>> This method of only deleting Resource Claims after the FlowFile
>> Repository has been fsync'ed means that we know on restart that
>> Resource Claim A won't still be referenced.
>>
>> So that was probably a very wordy, verbose description of what happens
>> but I'm trying to make sure that I explain things adequately.
>> So with that background... if you are storing your FlowFile Repository
>> on the same volume as your Content Repository, the following
>> could happen:
>>
>> At some point in time, enough data is queued up in your flow for you
>> to run out of disk space. As a result, the FlowFile Repository is
>> unable to be compacted. Since this is not happening, it will not mark
>> any of the Resource Claims as destructible. This would mean that
>> the Content Repository does not get cleaned up. So now you've got a
>> full Content Repository and it's unable to clean up after itself, because
>> no Resource Claims are getting marked as destructible.
>>
>> So to prove or disprove this theory, there are a few things that you
>> can look at:
>>
>> Do you see the following anywhere in your logs: Unable to checkpoint
>> FlowFile Repository
>>
>> If you add the following to your conf/logback.xml:
>> <logger name="org.apache.nifi.controller.repository.claim.
>> StandardResourceClaimManager"
>> level="DEBUG" />
>> Then that should allow you to see a DEBUG-level log message every time
>> that a Resource Claim is marked destructible and every time
>> that the Content Repository requests the collection of Destructible
>> Claims ("Drained 100 destructable claims" for instance)
>>
>> Any of the logs related to those statements should be very valuable in
>> determining what's going on.
>>
>> Thanks again for all of the detailed analysis. Hopefully we can get
>> this all squared away and taken care of quickly!
>>
>> -Mark
>>
>>
>> On Dec 11, 2016, at 1:21 PM, Alan Jackoway <al...@cloudera.com<mailto:ala
>> n...@cloudera.com><mailto:ala
>> n...@cloudera.com<mailto:n...@cloudera.com>><mailto:
>> al...@cloudera.com<mailto:al...@cloudera.com><mailto:al...@cloudera.com>>>
>> wrote:
>>
>> Here is what I have figured out so far.
>>
>> The cleanups are scheduled at https://github.com/apache/nifi
>> /blob/master/nifi-nar-bundles/nifi-framework-bundle/nifi-fra
>> mework/nifi-framework-core/src/main/java/org/apache/nifi/con
>> troller/repository/FileSystemRepository.java#L232
>>
>> I'm not totally sure which one of those is the one that should be
>> cleaning things up. It's either ArchiveOrDestroyDestructableClaims or
>> DestroyExpiredArchiveClaims, both of which are in that class, and both of
>> which are scheduled with scheduleWithFixedDelay. Based on docs at
>> https://docs.oracle.com/javase/7/docs/api/java/util/concurre
>> nt/ScheduledThreadPoolExecutor.html#scheduleWithFixedDelay(j
>> ava.lang.Runnable,%20long,%20long,%20java.util.concurrent.TimeUnit)
>> if those methods fail once, they will stop running forever. Also if the
>> thread got stuck it wouldn't launch a new one.
>>
>> I then hoped I would go into the logs, see a failure, and use it to
>> figure out the issue.
>>
>> What I'm seeing instead is things like this, which comes from
>> BinDestructableClaims:
>> 2016-12-10 23:08:50,117 INFO [Cleanup Archive for default]
>> o.a.n.c.repository.FileSystemRepository Deleted 159 files from
>> archive for Container default; oldest Archive Date is now Sat Dec 10
>> 22:09:53 PST 2016; container cleanup took 34266 millis
>> that are somewhat frequent (as often as once per second, which is the
>> scheduling frequency). Then, eventually, they just stop. Unfortunately
>> there isn't an error message I can find that's killing these.
>>
>> At nifi startup, I see messages like this, which come from something
>> (not sure what yet) calling the cleanup() method on FileSystemRepository:
>> 2016-12-11 09:15:38,973 INFO [main] o.a.n.c.repository.
>> FileSystemRepository
>> Found unknown file /home/cops/edh-bundle-extracto
>> r/content_repository/0/1481467667784-2048 (1749645 bytes) in File
>> System Repository; removing file
>> I never see those after the initial cleanup that happens on restart.
>>
>> I attached a thread dump. I noticed at the top that there is a cleanup
>> thread parked. I took 10 more thread dumps after this and in every one of
>> them the cleanup thread was parked. That thread looks like it corresponds
>> to DestroyExpiredArchiveClaims, so I think it's incidental. I believe that
>> if the cleanup task I need were running, it would be in one of the
>> FileSystemRepository Workers. However, in all of my thread dumps, these
>> were always all parked.
>>
>> Attached one of the thread dumps.
>>
>> Thanks,
>> Alan
>>
>>
>> On Sun, Dec 11, 2016 at 12:17 PM, Mark Payne <marka...@hotmail.com
>> <mailto:marka...@hotmail.com>> wrote:
>> Alan,
>>
>>
>> It's possible that you've run into some sort of bug that is preventing
>>
>> it from cleaning up the Content  Repository properly. While it's stuck
>>
>> in this state, could you capture a thread dump (bin/nifi.sh dump
>> thread-dump.txt)?
>>
>> That would help us determine if there is something going on that is
>>
>> preventing the cleanup from happening.
>>
>>
>> Thanks
>>
>> -Mark
>>
>>
>> ________________________________
>> From: Alan Jackoway <al...@cloudera.com<mailto:al...@cloudera.com>>
>> Sent: Sunday, December 11, 2016 11:11 AM
>> To: dev@nifi.apache.org<mailto:dev@nifi.apache.org>
>> Subject: Re: Content Repository Cleanup
>>
>> This just filled up again even
>> with nifi.content.repository.archive.enabled=false.
>>
>> On the node that is still alive, our queued flowfiles are 91 / 16.47
>> GB,
>> but the content repository directory is using 646 GB.
>>
>> Is there a property I can set to make it clean things up more
>> frequently? I
>> expected that once I turned archive enabled off, it would delete things
>> from the content repository as soon as the flow files weren't queued
>> anywhere. So far the only way I have found to reliably get nifi to
>> clear
>> out the content repository is to restart it.
>>
>> Our version string is the following, if that interests you:
>> 11/26/2016 04:39:37 PST
>> Tagged nifi-1.1.0-RC2
>> From ${buildRevision} on branch ${buildBranch}
>>
>> Maybe we will go to the released 1.1 and see if that helps. Until then
>> I'll
>> be restarting a lot and digging into the code to figure out where this
>> cleanup is supposed to happen. Any pointers on code/configs for that
>> would
>> be appreciated.
>>
>> Thanks,
>> Alan
>>
>> On Sun, Dec 11, 2016 at 8:51 AM, Joe Gresock <jgres...@gmail.com
>> <mailto:jgres...@gmail.com>> wrote:
>>
>> No, in my scenario a server restart would not affect the content
>> repository
>> size.
>>
>> On Sun, Dec 11, 2016 at 8:46 AM, Alan Jackoway <al...@cloudera.com
>> <mailto:al...@cloudera.com>> wrote:
>>
>> If we were in the situation Joe G described, should we expect that
>> when
>> we
>> kill and restart nifi it would clean everything up? That behavior
>> has
>> been
>> consistent every time - when the disk hits 100%, we kill nifi,
>> delete
>> enough old content files to bring it back up, and before it bring
>> the UI
>> up
>> it deletes things to get within the archive policy again. That
>> sounds
>> less
>> like the files are stuck and more like it failed trying.
>>
>> For now I just turned off archiving, since we don't really need it
>> for
>> this use case.
>>
>> I attached a jstack from last night's failure, which looks pretty
>> boring
>> to me.
>>
>> On Sun, Dec 11, 2016 at 1:37 AM, Alan Jackoway <al...@cloudera.com
>> <mailto:al...@cloudera.com>>
>> wrote:
>>
>> The scenario Joe G describes is almost exactly what we are doing.
>> We
>> bring in large files and unpack them into many smaller ones. In
>> the most
>> recent iteration of this problem, I saw that we had many small
>> files
>> queued
>> up at the time trouble was happening. We will try your suggestion
>> to
>> see if
>> the situation improves.
>>
>> Thanks,
>> Alan
>>
>> On Sat, Dec 10, 2016 at 6:57 AM, Joe Gresock <jgres...@gmail.com
>> <mailto:jgres...@gmail.com>>
>> wrote:
>>
>> Not sure if your scenario is related, but one of the NiFi devs
>> recently
>> explained to me that the files in the content repository are
>> actually
>> appended together with other flow file content (please correct
>> me if
>> I'm
>> explaining it wrong).  That means if you have many small flow
>> files in
>> your
>> current backlog, and several large flow files have recently left
>> the
>> flow,
>> the large ones could still be hanging around in the content
>> repository
>> as
>> long as the small ones are still there, if they're in the same
>> appended
>> files on disk.
>>
>> This scenario recently happened to us: we had a flow with ~20
>> million
>> tiny
>> flow files queued up, and at the same time we were also
>> processing a
>> bunch
>> of 1GB files, which left the flow quickly.  The content
>> repository was
>> much
>> larger than what was actually being reported in the flow stats,
>> and our
>> disks were almost full.  On a hunch, I tried the following
>> strategy:
>> - MergeContent the tiny flow files using flow-file-v3 format (to
>> capture
>> all attributes)
>> - MergeContent 10,000 of the packaged flow files using tar
>> format for
>> easier storage on disk
>> - PutFile into a directory
>> - GetFile from the same directory, but using back pressure from
>> here on
>> out
>> (so that the flow simply wouldn't pull the same files from disk
>> until
>> it
>> was really ready for them)
>> - UnpackContent (untar them)
>> - UnpackContent (turn them back into flow files with the original
>> attributes)
>> - Then do the processing they were originally designed for
>>
>> This had the effect of very quickly reducing the size of my
>> content
>> repository to very nearly the actual size I saw reported in the
>> flow,
>> and
>> my disk usage dropped from ~95% to 50%, which is the configured
>> content
>> repository max usage percentage.  I haven't had any problems
>> since.
>>
>> Hope this helps.
>> Joe
>>
>> On Sat, Dec 10, 2016 at 12:04 AM, Joe Witt <joe.w...@gmail.com
>> <mailto:joe.w...@gmail.com>> wrote:
>>
>> Alan,
>>
>> That retention percentage only has to do with the archive of
>> data
>> which kicks in once a given chunk of content is no longer
>> reachable
>> by
>> active flowfiles in the flow.  For it to grow to 100%
>> typically would
>> mean that you have data backlogged in the flow that account
>> for that
>> much space.  If that is certainly not the case for you then we
>> need
>> to
>> dig deeper.  If you could do screenshots or share log files
>> and stack
>> dumps around this time those would all be helpful.  If the
>> screenshots
>> and such are too sensitive please just share as much as you
>> can.
>>
>> Thanks
>> Joe
>>
>> On Fri, Dec 9, 2016 at 9:55 PM, Alan Jackoway <
>> al...@cloudera.com<mailto:al...@cloudera.com>>
>> wrote:
>> One other note on this, when it came back up there were tons
>> of
>> messages
>> like this:
>>
>> 2016-12-09 18:36:36,244 INFO [main] o.a.n.c.repository.
>> FileSystemRepository
>> Found unknown file /path/to/content_repository/49
>> 8/1481329796415-87538
>> (1071114 bytes) in File System Repository; archiving file
>>
>> I haven't dug into what that means.
>> Alan
>>
>> On Fri, Dec 9, 2016 at 9:53 PM, Alan Jackoway <
>> al...@cloudera.com<mailto:al...@cloudera.com>>
>> wrote:
>>
>> Hello,
>>
>> We have a node on which nifi content repository keeps
>> growing to
>> use
>> 100%
>> of the disk. It's a relatively high-volume process. It
>> chewed
>> through
>> more
>> than 100GB in the three hours between when we first saw it
>> hit
>> 100%
>> of
>> the
>> disk and when we just cleaned it up again.
>>
>> We are running nifi 1.1 for this. Our nifi.properties
>> looked like
>> this:
>>
>> nifi.content.repository.implementation=org.apache.
>> nifi.controller.repository.FileSystemRepository
>> nifi.content.claim.max.appendable.size=10 MB
>> nifi.content.claim.max.flow.files=100
>> nifi.content.repository.direct
>> ory.default=./content_repository
>> nifi.content.repository.archive.max.retention.period=12
>> hours
>> nifi.content.repository.archive.max.usage.percentage=50%
>> nifi.content.repository.archive.enabled=true
>> nifi.content.repository.always.sync=false
>>
>> I just bumped retention period down to 2 hours, but should
>> max
>> usage
>> percentage protect us from using 100% of the disk?
>>
>> Unfortunately we didn't get jstacks on either failure. If
>> it hits
>> 100%
>> again I will make sure to get that.
>>
>> Thanks,
>> Alan
>>
>>
>>
>>
>>
>> --
>> I know what it is to be in need, and I know what it is to have
>> plenty.  I
>> have learned the secret of being content in any and every
>> situation,
>> whether well fed or hungry, whether living in plenty or in
>> want.  I can
>> do
>> all this through him who gives me strength.    *-Philippians
>> 4:12-13*
>>
>>
>>
>>
>>
>>
>> --
>> I know what it is to be in need, and I know what it is to have
>> plenty.  I
>> have learned the secret of being content in any and every situation,
>> whether well fed or hungry, whether living in plenty or in want.  I
>> can do
>> all this through him who gives me strength.    *-Philippians 4:12-13*
>>
>>
>> <thread-dump.txt>
>>
>>
>>
>>
>>
>>
>>
>>
>>
>>
>

Reply via email to