Thanks Bill,

I have written up a ticket here: 
https://issues.apache.org/jira/browse/KAFKA-8042

Adrian

On 05/03/2019, 15:44, "Bill Bejeck" <b...@confluent.io> wrote:

    Hi Adrian,
    
    No, it's not an expected outcome.
    
    Could you file a Jira ticket and include the information requested by
    Guozhang (code and configs) and we can try to reproduce the error?
    
    Thanks,
    Bill
    
    On Tue, Mar 5, 2019 at 10:14 AM Adrian McCague <adrian.mcca...@zopa.com>
    wrote:
    
    > Drilling down further:
    >
    > bash-4.2# pwd
    > /data/fooapp/0_7
    > bash-4.2# for dir in $(find . -maxdepth 1 -type d); do echo "${dir}:
    > $(find ${dir} -type f -name 'MANIFEST-*' -printf x | wc -c)"; done
    > .: 8058
    > ./KSTREAM-JOINOTHER-0000000025-store: 851
    > ./KSTREAM-JOINOTHER-0000000040-store: 819
    > ./KSTREAM-JOINTHIS-0000000024-store: 851
    > ./KSTREAM-JOINTHIS-0000000029-store: 836
    > ./KSTREAM-JOINOTHER-0000000035-store: 819
    > ./KSTREAM-JOINOTHER-0000000030-store: 819
    > ./KSTREAM-JOINOTHER-0000000045-store: 745
    > ./KSTREAM-JOINTHIS-0000000039-store: 819
    > ./KSTREAM-JOINTHIS-0000000044-store: 685
    > ./KSTREAM-JOINTHIS-0000000034-store: 819
    >
    > There are many (x800 as above) of these segment files:
    > ./KSTREAM-JOINOTHER-0000000025-store.1551466290000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551559020000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551492690000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551548790000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551698610000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551530640000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551484440000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551556710000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551686730000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551595650000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551757350000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551685740000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551635250000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551652410000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551466620000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551781770000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551587400000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551681450000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551662310000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551721710000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551750750000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551630960000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551615120000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551792330000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551462660000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551536910000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551592350000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551527340000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551606870000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551744150000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551508200000
    > ./KSTREAM-JOINOTHER-0000000025-store.1551486420000
    >
    > I highly suspect this PR: https://github.com/apache/kafka/pull/5253
    >
    > Is this an expected outcome of this change? If not I'll file a bug with
    > the included information.
    >
    > My concern right now is that whilst I can reduce the MANIFEST
    > pre-allocation size, the number of segment stores appears to be unbounded
    > so under some circumstances this may not suffice.
    >
    > Thanks
    > Adrian
    >
    > On 01/03/2019, 23:05, "Adrian McCague" <adrian.mcca...@zopa.com> wrote:
    >
    >     Hi Guozhang, thanks for your response.
    >
    >     I have done some further investigations.
    >
    >     The difference I see between the two versions is the following, in 1.1
    > this is the stat of the rocksdb MANIFEST files of one of the partitions:
    >
    >     root@fooapp-6c4649dd68-wzrxk:/data# stat
    > fooapp/2_5/rocksdb/foo-store/MANIFEST-000007
    >       File: fooapp/2_5/rocksdb/foo-store/MANIFEST-000007
    >       Size: 159         Blocks: 16         IO Block: 4096   regular file
    >     Device: d3h/211d    Inode: 44831568    Links: 1
    >     Access: (0644/-rw-r--r--)  Uid: (    0/    root)   Gid: (    0/
    > root)
    >     Access: 2019-03-01 19:56:46.000000000 +0000
    >     Modify: 2019-03-01 19:56:46.000000000 +0000
    >     Change: 2019-03-01 19:56:46.000000000 +0000
    >      Birth: -
    >
    >     And ls -ls
    >     8 -rw-r--r--. 1 root root 275 Mar  1 19:57 MANIFEST-000007
    >
    >     Then in 2.1 this is the equivalent manifest file:
    >
    >     root@fooapp-7678bbcfb7-wtjjn:/data# stat
    > fooapp/2_5/rocksdb/foo-store/MANIFEST-000011
    >       File: fooapp /2_5/rocksdb/foo-store/MANIFEST-000011
    >       Size: 160         Blocks: 8200       IO Block: 4096   regular file
    >     Device: 10006eh/1048686d    Inode: 185032005   Links: 1
    >     Access: (0644/-rw-r--r--)  Uid: (    0/    root)   Gid: (    0/
    > root)
    >     Access: 2019-03-01 19:32:42.000000000 +0000
    >     Modify: 2019-03-01 19:32:42.000000000 +0000
    >     Change: 2019-03-01 19:32:42.000000000 +0000
    >      Birth: -
    >
    >     And ls -ls
    >     4100 -rw-r--r--. 1 root root     160 Mar  1 19:32 MANIFEST-000011
    >
    >     I believe this is what is resulting in the actual disk usage - many of
    > these manifest files using 4MB on the filesystem each quickly adding up
    > despite only being 160 bytes.
    >
    > 
https://github.com/facebook/rocksdb/blob/v5.14.2/include/rocksdb/options.h#L589
    > implies this should always have been the case, and they weren't
    > preallocating (properly?) but now they are.
    >     I can see in the OPTIONS file that the newer version has introduced
    > the manifest_preallocation_size=4194304 configuration whereas it's absent
    > in the 5.7.3 file.
    >
    >     At rest this application has 508 MANIFEST-* files (it has a lot of
    > individual state stores) so they will account for 2GB alone (significantly
    > greater than the total state dir size). Though this is manageable.
    >
    >     In 1.1 during start-up there can be as many as 670 of the MANIFEST
    > files, in 2.1 the number of MANIFEST files grows exceptionally fast to 
peak
    > as high as 8811 !! in one test run before dropping back down, it's 
probably
    > been more in some cases but this was a clean start with a decent initial
    > rebalance delay.
    >
    >     Both versions are running in docker, identical configurations. We have
    > not touched the defaults set for RocksDB by Kafka Streams.
    >
    >     So it looks like it might be possible to tune down the manifest
    > preallocation size somewhat in RocksDB directly given none of our manifest
    > files are required to be this size, the largest appears to be 4.7KB actual
    > disk required, many are much much smaller. My next line of enquiry then is
    > why the number of MANIFEST files on restore goes so extremely high so fast
    > in this new version.
    >
    >     4.7KB * 8811 files is a more manageable size so either addressing the
    > manifest size or the number of manifests will bring this into manageable
    > territory for us. Though I'd like to tackle both.
    >
    >     rocksdb.config.setter is I expect the way forward on the manifest
    > preallocation size. I don't know if anyone can give an indication of what 
a
    > typical MANIFEST size would be or how it scales relative to say
    > partition/segment size?
    >
    >     Are you able to shed some light on why this newer version is creating
    > so many MANIFEST files? I can look into it further but I suspect either
    > each instance is creating stores for more partitions than it will actually
    > handle or there are duplicate / redundant / cleaned up stores/MANIFEST
    > files hanging around.
    >
    >     Thanks
    >     Adrian
    >
    >     On 01/03/2019, 18:08, "Guozhang Wang" <wangg...@gmail.com> wrote:
    >
    >         Hello Adrian,
    >
    >         What you described did sounds wired to me. I'm not aware of any
    > regressions
    >         on rocksDB disk usage from 1.1 to 2.1.
    >
    >         Could you file a JIRA ticket with more details like state dir
    > snapshots,
    >         your code snippet and configs etc so we can find a way to
    > reproduce it?
    >
    >
    >         Guozhang
    >
    >         On Fri, Mar 1, 2019 at 5:54 AM Adrian McCague <
    > adrian.mcca...@zopa.com>
    >         wrote:
    >
    >         > Hi,
    >         >
    >         > We are in the process of attempting to upgrade from Kafka
    > Streams 1.1.0 to
    >         > 2.1.0(-cp1) however we get some wildly different behaviour with
    > regards to
    >         > disk usage between these two versions.
    >         >
    >         > An update that uses existing state data exhibits the same
    > behaviour as
    >         > starting with a completely clean state data directories.
    >         >
    >         > With 1.1.0 the same topologies restoring the same state stores
    > will at
    >         > most use 275MB.
    >         > With 2.1.0 they will quickly take up to 60GB – I extended the
    > volumes to
    >         > see just how far it would go, already magnitudes larger than
    > before.
    >         >
    >         > Only remotely interesting information in the logs are the
    > following
    >         > exceptions which other than the explicit disk space exception I
    > presume is
    >         > related to out of disk space:
    >         >
    >         > org.apache.kafka.streams.errors.ProcessorStateException: task
    > [0_7]
    >         > Exception caught while trying to restore state from
    >         > foo-v4-KSTREAM-JOINTHIS-0000000012-store-changelog-7
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.ProcessorStateManager.updateStandbyStates(ProcessorStateManager.java:185)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StandbyTask.update(StandbyTask.java:188)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StreamThread.maybeUpdateStandbyTasks(StreamThread.java:1114)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:895)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:777)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:747)
    >         > Caused by: java.lang.NullPointerException: null
    >         > java.lang.NullPointerException: null
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.state.internals.RocksDBStore.toggleDbForBulkLoading(RocksDBStore.java:229)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore.getWriteBatches(RocksDBSegmentedBytesStore.java:230)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore.restoreAllInternal(RocksDBSegmentedBytesStore.java:208)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore$RocksDBSegmentsBatchingRestoreCallback.restoreAll(RocksDBSegmentedBytesStore.java:262)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StateRestoreCallbackAdapter.lambda$adapt$0(StateRestoreCallbackAdapter.java:42)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.CompositeRestoreListener.restoreBatch(CompositeRestoreListener.java:89)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StateRestorer.restore(StateRestorer.java:83)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StoreChangelogReader.processNext(StoreChangelogReader.java:310)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StoreChangelogReader.restore(StoreChangelogReader.java:92)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.TaskManager.updateNewAndRestoringTasks(TaskManager.java:321)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:839)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:777)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:747)
    >         >
    >         >
    >         > org.apache.kafka.streams.errors.ProcessorStateException: Error
    > opening
    >         > store KSTREAM-JOINTHIS-0000000018-store.1551343860000 at 
location
    >         >
    > 
/data/foo/0_1/KSTREAM-JOINTHIS-0000000018-store/KSTREAM-JOINTHIS-0000000018-store.1551343860000
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.state.internals.RocksDBStore.openDB(RocksDBStore.java:158)
    >         >                 at
    >         >
    > org.apache.kafka.streams.state.internals.Segment.openDB(Segment.java:45)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.state.internals.Segments.getOrCreateSegment(Segments.java:101)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.state.internals.Segments.getOrCreateSegmentIfLive(Segments.java:81)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore.getWriteBatches(RocksDBSegmentedBytesStore.java:224)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore.restoreAllInternal(RocksDBSegmentedBytesStore.java:208)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore$RocksDBSegmentsBatchingRestoreCallback.restoreAll(RocksDBSegmentedBytesStore.java:262)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StateRestoreCallbackAdapter.lambda$adapt$0(StateRestoreCallbackAdapter.java:42)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.CompositeRestoreListener.restoreBatch(CompositeRestoreListener.java:89)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StateRestorer.restore(StateRestorer.java:83)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StoreChangelogReader.processNext(StoreChangelogReader.java:310)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StoreChangelogReader.restore(StoreChangelogReader.java:92)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.TaskManager.updateNewAndRestoringTasks(TaskManager.java:321)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:839)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:777)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:747)
    >         > Caused by: org.rocksdb.RocksDBException: While appending to 
file:
    >         >
    > 
/data/foo/0_1/KSTREAM-JOINTHIS-0000000018-store/KSTREAM-JOINTHIS-0000000018-store.1551343860000/MANIFEST-000001:
    >         > No space left on device
    >         >                 at org.rocksdb.RocksDB.open(Native Method)
    >         >                 at org.rocksdb.RocksDB.open(RocksDB.java:231)
    >         >                 at
    >         >
    > 
org.apache.kafka.streams.state.internals.RocksDBStore.openDB(RocksDBStore.java:156)
    >         >                 ... 15 common frames omitted
    >         >
    >         > It feels likely that this is some serious amount of
    > preallocation that is
    >         > happening that didn’t use to?
    >         > I have eliminated
    >         > https://bugs.java.com/bugdatabase/view_bug.do?bug_id=JDK-8202261
    > being an
    >         > issue – we see this behaviour with 8u141, 8u181 and 8u202
    >         >
    >         > Any help or avenues of investigation are welcome.
    >         >
    >         > Thanks
    >         > Adrian
    >         >
    >         > Disclaimer
    >         >
    >         > This e-mail has been sent to you by Zopa.
    >         >
    >         > Zopa Limited is authorised and regulated by the Financial 
Conduct
    >         > Authority,
    >         > and entered on the Financial Services Register under firm
    > registration
    >         > number
    >         > 718925. Zopa Bank Limited is authorised by the Prudential
    > Regulation
    >         > Authority
    >         > and regulated by the Financial Conduct Authority and the
    > Prudential
    >         > Regulation
    >         > Authority, and entered on the Financial Services Register
    > (800542).
    >         >
    >         > Registered Office: Zopa Limited (05197592) and Zopa Bank Limited
    >         > (10627575) are both incorporated in England & Wales and have
    > their
    >         > registered office at: 1st Floor, Cottons Centre, Tooley Street,
    > London,
    >         > SE1 2QG.
    >         >
    >         >
    >
    >         --
    >         -- Guozhang
    >
    >
    >
    >
    >
    

Reply via email to