flink-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From <Bernd.Winterst...@Dev.Helaba.de>
Subject AW: number of files in checkpoint directory grows endlessly
Date Thu, 06 Dec 2018 11:53:06 GMT
Seems that some file deletion is disabled by default. There are some log entries in the file

Von: Andrey Zagrebin [mailto:andrey@data-artisans.com]
Gesendet: Donnerstag, 6. Dezember 2018 12:07
An: Winterstein, Bernd
Cc: Yun Tang; Kostas Kloudas; user; Stefan Richter; Till Rohrmann; Stephan Ewen
Betreff: Re: number of files in checkpoint directory grows endlessly

Hi Bernd,

Thanks for sharing the code.

I understand your TTL requirement. It definitely makes sense for your application.
My recommendation is still to try running job with original backend without TtlDb modification
to narrow down the problem and understand where these small files come from. They might look
like some meta information which seems to accumulate over time.

Could you also investigate temporary local directories in task executors?
By default, they are usually in System.getProperty("java.io.tmpdir") (e.g. /tmp in Linux)
or there is also a config option for them [1].
Do you see there explosion of similar files? They can be named differently. The interesting
would be whether the smaller files end with ‘.sst’ or not.

Best,
Andrey

[1] https://ci.apache.org/projects/flink/flink-docs-release-1.6/ops/config.html#io-tmp-dirs


On 6 Dec 2018, at 09:16, <Bernd.Winterstein@Dev.Helaba.de<mailto:Bernd.Winterstein@Dev.Helaba.de>>
<Bernd.Winterstein@Dev.Helaba.de<mailto:Bernd.Winterstein@Dev.Helaba.de>> wrote:

Hi Andrey
We need the TTL feature, because we store about 10 million state entries per 24 hours. Each
entry has to expire after 24 hours. We couldn’t do the expiry with timers due to bad checkpoint
performance, so I switched to TtlDB.

It seems that per minute files are only available for the last incremental period. After this
time the files have five minute distance.

As for the chk-* directories, there is only one directory left.

I attached the code for the options and the KeyedStateBackend and also one of the shared files.

Many thanks for your help.


Bernd

cat /appdata/rtts/ice/shared/jobmanager/checkpoints/stp/a73a132912c8efaaab4a8e6331bdcf47/shared/6d64a7cc-8cdf-4f51-aa2f-bb7c8e7d4ce3

¦>¦¦}d¦¦/20181205-HELADEF1TSK-TX-9ffXfg1DZCHEe2JGqpC6dDJ
        ¦¦$rocksdb.block.based.table.index.type*rocksdb.block.based.table.prefix.filtering0-rocksdb.block.based.table.whole.key.filtering1rocksdb.column.family.id#rocksdb.column.family.name_timer_state/processing_user-timersrocksdb.comparatorleveldb.BytewiseComparatorrocksdb.compressionSnappyrocksdb.creation.time¦¦¦¦rocksdb.data.sizeQrocksdb.deleted.keysrocksdb.filter.policyrocksdb.BuiltinBloomFilterrocksdb.filter.sizerocksdb.fixed.key.lengthrocksdb.format.versionrocksdb.index.sizeocksdb.merge.operands
                                                                                         
                                                                                         
                                                                   rocksdb.merge.operatorMerge
By TTLrocksdb.num.data.blocksrocksdb.num.entriesrocksdb.prefix.extractor.namenullptrocksdb.property.collectors[]rocksdb.raw.key.sizeArocksdb.raw.value.size+Y¦¦¦1Nc{¦¦¦¦*Oj¦¦¦¦¦V!filter.rocksdb.BuiltinBloomFilterQrocksdb.propertiesh¦&¦¦2?
   }¦¦¦¦¦¦¦L¦¦¦JW¦¦¦$uG¦



Von: Andrey Zagrebin [mailto:andrey@data-artisans.com]
Gesendet: Mittwoch, 5. Dezember 2018 15:25
An: Winterstein, Bernd
Cc: Yun Tang; Kostas Kloudas; user; Stefan Richter; Till Rohrmann; stephan@data-artisans.com<mailto:stephan@data-artisans.com>
Betreff: Re: number of files in checkpoint directory grows endlessly

Hi Bernd,

Thanks for sharing the code.

Could you try the same job without TtlDb, with original RocksDb state backend?
Even if data does not expire every day, the overall size will grow but the number of files
should not because of compaction.

Do you set some more specific db/column options in this option factory StpRocksDbOptions?
Could you share the source code of it and modified RocksDBKeyedStateBackend, especially createColumnFamilyWithTtl?

Do you always have only 1-2 last 'chk-*’ directories at a time or there are more not cleaned
up checkpoint directories?

You have 1 minute checkpoint interval but the list of files does not have files for some minutes
consecutively. Do the files miss for some minutes/checkpoints, like 10:10 or 10:05, or checkpoints
just take long including CheckpointMinPause?

Could you check whether those ‘1121’ files identical? look inside or post one of them
here?

Best,
Andrey

On 4 Dec 2018, at 14:10, <Bernd.Winterstein@Dev.Helaba.de<mailto:Bernd.Winterstein@Dev.Helaba.de>>
<Bernd.Winterstein@Dev.Helaba.de<mailto:Bernd.Winterstein@Dev.Helaba.de>> wrote:

All calls to createColumnFamily were exchanged by createColumnFamilyWithTtl



     private <N, S> Tuple2<ColumnFamilyHandle, RegisteredKeyValueStateBackendMetaInfo<N,
S>> tryRegisterKvStateInformation(
                StateDescriptor<?, S> stateDesc,
                TypeSerializer<N> namespaceSerializer,
                @Nullable StateSnapshotTransformer<S> snapshotTransformer) throws StateMigrationException
{

          Tuple2<ColumnFamilyHandle, RegisteredStateMetaInfoBase> stateInfo =
                kvStateInformation.get(stateDesc.getName());

          RegisteredKeyValueStateBackendMetaInfo<N, S> newMetaInfo;
          if (stateInfo != null) {

                StateMetaInfoSnapshot restoredMetaInfoSnapshot = restoredKvStateMetaInfos.get(stateDesc.getName());

                Preconditions.checkState(
                     restoredMetaInfoSnapshot != null,
                     "Requested to check compatibility of a restored RegisteredKeyedBackendStateMetaInfo,"
+
                          " but its corresponding restored snapshot cannot be found.");

                newMetaInfo = RegisteredKeyValueStateBackendMetaInfo.resolveKvStateCompatibility(
                     restoredMetaInfoSnapshot,
                     namespaceSerializer,
                     stateDesc,
                     snapshotTransformer);

                stateInfo.f1 = newMetaInfo;
          } else {
                String stateName = stateDesc.getName();

                newMetaInfo = new RegisteredKeyValueStateBackendMetaInfo<>(
                     stateDesc.getType(),
                     stateName,
                     namespaceSerializer,
                     stateDesc.getSerializer(),
                     snapshotTransformer);

                ColumnFamilyHandle columnFamily = createColumnFamilyWithTtl(stateName);

                stateInfo = Tuple2.of(columnFamily, newMetaInfo);
                kvStateInformation.put(stateDesc.getName(), stateInfo);
          }

          return Tuple2.of(stateInfo.f0, newMetaInfo);
     }

Von: Yun Tang [mailto:myasuka@live.com]
Gesendet: Dienstag, 4. Dezember 2018 13:55
An: Winterstein, Bernd; andrey@data-artisans.com<mailto:andrey@data-artisans.com>
Cc: k.kloudas@data-artisans.com<mailto:k.kloudas@data-artisans.com>; user@flink.apache.org<mailto:user@flink.apache.org>;
s.richter@data-artisans.com<mailto:s.richter@data-artisans.com>; till@data-artisans.com<mailto:till@data-artisans.com>;
stephan@data-artisans.com<mailto:stephan@data-artisans.com>
Betreff: Re: number of files in checkpoint directory grows endlessly

Hi Bernd,

RocksDBStateBackend would not use default column family, but put state's entries into its
newly-created column family by default. Would you please check whether having used db.createColumnFamilyWithTtl
method instead ofdb.createColumnFamily within RocksDBKeyedStateBackend#tryRegisterKvStateInformation?

In my opinion, the default method of db.createColumnFamily would set ttl as 0, which means
infinity, for this column family. You could view RocksDB's implementation here<https://github.com/facebook/rocksdb/blob/359e48333d5aade023970d692a61434e7bf82fe8/utilities/ttl/db_ttl_impl.cc#L121>.

Best
Yun Tang
________________________________
From: Bernd.Winterstein@Dev.Helaba.de<mailto:Bernd.Winterstein@Dev.Helaba.de> <Bernd.Winterstein@Dev.Helaba.de<mailto:Bernd.Winterstein@Dev.Helaba.de>>
Sent: Tuesday, December 4, 2018 17:40
To: andrey@data-artisans.com<mailto:andrey@data-artisans.com>
Cc: k.kloudas@data-artisans.com<mailto:k.kloudas@data-artisans.com>; user@flink.apache.org<mailto:user@flink.apache.org>;
s.richter@data-artisans.com<mailto:s.richter@data-artisans.com>; till@data-artisans.com<mailto:till@data-artisans.com>;
stephan@data-artisans.com<mailto:stephan@data-artisans.com>
Subject: AW: number of files in checkpoint directory grows endlessly

Sorry for the late answer. I haven’t been in the office.

The logs show no problems.
The files that remain in the shared subfolder are almost all 1121 bytes. Except the files
from the latest checkpoint (30 files for all operators)
For each historic checkpoint six files remain (parallelism is 6)

checkpoints/stp/2b160fc9e5eba47d1906d04f36b399bf/shared/

-rw-r--r--. 1 flink ice     11318 Dec  4 10:23 046b16e5-0edc-4ca5-9757-d89aa86f5d5c
-rw-r--r--. 1 flink ice    308383 Dec  4 10:23 dab0f801-8907-4b10-ae8b-5f5f71c28524
-rw-r--r--. 1 flink ice    101035 Dec  4 10:23 e4915253-7025-4e36-8671-58a371f202ff
-rw-r--r--. 1 flink ice     11318 Dec  4 10:23 613a9fd1-4545-4785-82ef-92f8c4818bc0
-rw-r--r--. 1 flink ice    308270 Dec  4 10:23 23771709-03ef-4417-acd2-1c27c8b0785e
-rw-r--r--. 1 flink ice     11318 Dec  4 10:23 59643ae2-57b8-4d9e-9f2e-977545db9238
-rw-r--r--. 1 flink ice    102444 Dec  4 10:23 f9a70bb7-d4f3-4d94-af31-8e94276001b1
-rw-r--r--. 1 flink ice    308346 Dec  4 10:23 bafc1a20-e5a4-4b06-93fe-db00f3f3913a
-rw-r--r--. 1 flink ice     96035 Dec  4 10:23 a8b3aa75-fa44-4fc2-ab49-437d49410acd
-rw-r--r--. 1 flink ice     11318 Dec  4 10:23 961f1472-b61c-4c04-b544-4549c3ce4038
-rw-r--r--. 1 flink ice    308387 Dec  4 10:23 7245dcfe-62e8-42a3-94a9-0698bdf9fb4d
-rw-r--r--. 1 flink ice     99209 Dec  4 10:23 5ad87ff2-604c-40d9-9262-76ac6369453d
-rw-r--r--. 1 flink ice     11318 Dec  4 10:23 ccaae280-837a-4bc9-95cd-3612c4cd435c
-rw-r--r--. 1 flink ice    308451 Dec  4 10:23 69b650dd-7969-4891-b973-9bd168e2f40e
-rw-r--r--. 1 flink ice    105638 Dec  4 10:23 16f3092d-76eb-4fd7-87b4-2eb28ca4696c
-rw-r--r--. 1 flink ice     11318 Dec  4 10:23 eccb19a0-4570-4e49-be97-707248690fe8
-rw-r--r--. 1 flink ice    308513 Dec  4 10:23 47259326-cb62-4abb-ad0b-5e2deda97685
-rw-r--r--. 1 flink ice    109918 Dec  4 10:23 aebc4390-3467-4596-a5fa-0d0a6dc74f54
-rw-r--r--. 1 flink ice 259444946 Dec  4 10:22 3e97ea93-4bc9-4404-aeca-eed31e96a14b
-rw-r--r--. 1 flink ice 247501755 Dec  4 10:22 b23d2c3a-94eb-45f0-bd29-457d8796624d
-rw-r--r--. 1 flink ice 247754788 Dec  4 10:22 4eb66b02-6758-4cff-9de2-fb7399b2ac0b
-rw-r--r--. 1 flink ice 247281033 Dec  4 10:22 2aeeb9ad-2714-481c-a7a4-04148f72671d
-rw-r--r--. 1 flink ice 247345955 Dec  4 10:22 5ccf700a-bd83-4e02-93a5-db46ca71e47a
-rw-r--r--. 1 flink ice 259312070 Dec  4 10:22 3bebe32d-0ad3-4f4e-a3aa-21719fa62c87
-rw-r--r--. 1 flink ice     97551 Dec  4 10:22 5a4f8a3a-0f26-46e7-883e-d6fafd733183
-rw-r--r--. 1 flink ice    104198 Dec  4 10:22 cdbb4913-7dd0-4614-8b81-276a4cdf62cc
-rw-r--r--. 1 flink ice    101466 Dec  4 10:22 ce7f0fea-8cd3-4827-9ef1-ceba569c2989
-rw-r--r--. 1 flink ice    108561 Dec  4 10:22 5bd3f681-c131-4c41-9fdc-6a39b9954aa7
-rw-r--r--. 1 flink ice     98649 Dec  4 10:22 d5d8eb16-3bd8-4695-91a0-9d9089ca9510
-rw-r--r--. 1 flink ice    102071 Dec  4 10:22 f8e34ef1-60d6-4c0a-954b-64c8a0320834
-rw-r--r--. 1 flink ice      1121 Dec  4 10:21 8fda9911-f63e-45a6-b95a-5c93fe99d0fd
-rw-r--r--. 1 flink ice      1121 Dec  4 10:21 82545c1b-69d6-499b-a9fd-62e227b820c6
-rw-r--r--. 1 flink ice      1121 Dec  4 10:21 f9fa3bba-c92d-4dda-b16e-0ba417edf5d2
-rw-r--r--. 1 flink ice      1121 Dec  4 10:21 844fa51d-bb74-4bec-ab15-e52d37703d24
-rw-r--r--. 1 flink ice      1121 Dec  4 10:21 2115654a-4544-41cc-bbee-a36d0d80d8eb
-rw-r--r--. 1 flink ice      1121 Dec  4 10:21 acfc1566-5f14-47d7-ae54-7aa1dfb3859c
-rw-r--r--. 1 flink ice      1121 Dec  4 10:16 b0144120-cce0-4b4d-9f8c-1564b9abedd9
-rw-r--r--. 1 flink ice      1121 Dec  4 10:16 8ab4ddab-3665-4307-a581-ab413e1e2080
-rw-r--r--. 1 flink ice      1121 Dec  4 10:16 0f8c4b1a-df5d-47f7-b960-e671cfc3c666
-rw-r--r--. 1 flink ice      1121 Dec  4 10:16 40baf147-400e-455f-aea3-074355a77031
-rw-r--r--. 1 flink ice      1121 Dec  4 10:16 47d2deca-1703-4dd3-9fea-e027087d553e
-rw-r--r--. 1 flink ice      1121 Dec  4 10:16 aa336ce0-3689-4b7d-a472-b0a3ed2f5eb9
-rw-r--r--. 1 flink ice      1121 Dec  4 10:11 ee15f1e0-d23c-4add-86b4-e4ab51bb2a20
-rw-r--r--. 1 flink ice      1121 Dec  4 10:11 f440b5cf-8f62-4532-a886-a2cedc9a043e
-rw-r--r--. 1 flink ice      1121 Dec  4 10:11 de423c46-4288-464b-97cb-6f7764b88dfd
-rw-r--r--. 1 flink ice      1121 Dec  4 10:11 273a15cb-8c9f-4412-b5d2-68397ba461c9
-rw-r--r--. 1 flink ice      1121 Dec  4 10:11 bb38b011-070d-4c21-b04a-4e923f85de86
-rw-r--r--. 1 flink ice      1121 Dec  4 10:11 969abc07-d313-4d79-8119-6e1f3886be48
-rw-r--r--. 1 flink ice      1121 Dec  4 10:06 eb0b2591-653c-47bd-a6b2-9f6634ff4f0a
-rw-r--r--. 1 flink ice      1121 Dec  4 10:06 20b7e49a-ace5-4ef7-987f-0d328f47c56f
-rw-r--r--. 1 flink ice      1121 Dec  4 10:06 a25c2bd9-7fe9-4558-b9dd-30b525a0b435
-rw-r--r--. 1 flink ice      1121 Dec  4 10:06 dcd0852f-58dc-467e-93db-5700cd4f606e
-rw-r--r--. 1 flink ice      1121 Dec  4 10:06 400e5038-2913-4aea-932d-92f508bd38f7
-rw-r--r--. 1 flink ice      1121 Dec  4 10:06 10ce727b-9389-4911-b0d4-1b342dd3232c
-rw-r--r--. 1 flink ice      1121 Dec  4 10:01 daec0dcb-384a-4d86-a423-7e2b0482b70e
-rw-r--r--. 1 flink ice      1121 Dec  4 10:01 c787d58a-4bd5-4d9a-a4d8-47d9618552ff
-rw-r--r--. 1 flink ice      1121 Dec  4 10:01 b2c1383a-8452-4ec6-9064-a5e8f56e6f21
-rw-r--r--. 1 flink ice      1121 Dec  4 10:01 65c0c908-b604-4ac9-a72f-4bb87676df11
-rw-r--r--. 1 flink ice      1121 Dec  4 10:01 42d59072-95ec-42d5-81ff-b9447cb39fd0
-rw-r--r--. 1 flink ice      1121 Dec  4 10:01 b40c2577-e604-482d-86a1-ddb785e3b799

The ttldb stream backend is mostly a copy of the current flink-statebackend-rocksb which uses
the TtlDB instead of the RockDB class with the following configuration:


    protected void buildDefaultStateBackend(StreamExecutionEnvironment executionEnvironment)
throws IOException {
        FsStateBackend fsStateBackend = new FsStateBackend(configuration.getCheckpointDirectory(),
true);
        RocksDBStateBackend rocksDBStateBackend = new de.helaba.rtts.ice.rocksdb.ttl.RocksDBStateBackend(fsStateBackend,
TernaryBoolean.TRUE, configuration.getStateTimeToLive());
        rocksDBStateBackend.setPredefinedOptions(PredefinedOptions.SPINNING_DISK_OPTIMIZED_HIGH_MEM);
        rocksDBStateBackend.setOptions(new StpRocksDbOptions());
        executionEnvironment.setStateBackend(rocksDBStateBackend);
    }

    protected void configureCheckpointing(StreamExecutionEnvironment executionEnvironment)
{
        CheckpointConfig checkpointConfig = executionEnvironment.getCheckpointConfig();
        checkpointConfig.setCheckpointInterval(configuration.getCheckpointInterval());
        checkpointConfig.setMinPauseBetweenCheckpoints(configuration.getCheckpointMinPause());
        checkpointConfig.enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION);
        checkpointConfig.setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE);
        executionEnvironment.getConfig().setUseSnapshotCompression(true);
    }



Von: Andrey Zagrebin [mailto:andrey@data-artisans.com]
Gesendet: Donnerstag, 29. November 2018 15:38
An: Winterstein, Bernd
Cc: Kostas Kloudas; user; Stefan Richter; Till Rohrmann; Stephan Ewen
Betreff: Re: number of files in checkpoint directory grows endlessly

Could you share the logs to check possible failures to subsume or remove previous checkpoints?
What is the sizes of the files? It can help to understand how compaction goes.
Could you also provide more details how you setup TtlDb with Flink?

Best,
Andrey


On 29 Nov 2018, at 11:34, Andrey Zagrebin <andrey@data-artisans.com<mailto:andrey@data-artisans.com>>
wrote:

Compaction merges SST files in background using native threads. While merging it filters out
removed and expired data. In general, the idea is that there are enough resources for compaction
to keep up with the DB update rate and reduce storage. It can be quite IO intensive. Compaction
has a lot of tuning knobs and statistics to monitor the process [1] which are usually out
of the scope of Flink depending on state access pattern of the application. You can create
and set RocksDBStateBackend for you application in Flink and configure it with custom RocksDb/column
specific options.

[1] https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide
[2] https://github.com/facebook/rocksdb/wiki/Compaction


On 29 Nov 2018, at 11:20, <Bernd.Winterstein@Dev.Helaba.de<mailto:Bernd.Winterstein@Dev.Helaba.de>>
<Bernd.Winterstein@Dev.Helaba.de<mailto:Bernd.Winterstein@Dev.Helaba.de>> wrote:


We use TtlDB because the state contents should expire automatically after 24 hours. Therefore
we only changed the state backend to use TtlDb instead of RocksDB with a fixed retention time.

We have a slow IO because we only have SAN volumes available. Can you further clarify the
problem with slow compaction.

Regards,

Bernd


-----Ursprüngliche Nachricht-----
Von: Andrey Zagrebin [mailto:andrey@data-artisans.com]
Gesendet: Donnerstag, 29. November 2018 11:01
An: Winterstein, Bernd
Cc: Kostas Kloudas; user; s.richter@data-artisans.com<mailto:s.richter@data-artisans.com>;
till@data-artisans.com<mailto:till@data-artisans.com>; stephan@data-artisans.com<mailto:stephan@data-artisans.com>
Betreff: Re: number of files in checkpoint directory grows endlessly

If you use incremental checkpoints, state backend stores raw RocksDB SST files which represent
all state data. Each checkpoint adds SST files with new updates which are not present in previous
checkpoint, basically their difference.

One of the following could be happening:
- old keys are not explicitly deleted or expire (depending on how TtlDb is used)
- compaction is too slow to drop older SST files for the latest checkpoint so that they can
be deleted with the previous checkpoints



On 29 Nov 2018, at 10:48, <Bernd.Winterstein@Dev.Helaba.de<mailto:Bernd.Winterstein@Dev.Helaba.de>>
<Bernd.Winterstein@Dev.Helaba.de<mailto:Bernd.Winterstein@Dev.Helaba.de>> wrote:

Hi
We use Flink 1..6.2. As for the checkpoint directory there is only one chk-xxx directory.
Therefore if would expect only one checkpoint remains.
The value of 'state.checkpoints.num-retained’ is not set explicitly.

The problem is not the number of checkpoints but the number of files in the "shared" directory
next to the chk-xxx directory.


-----Ursprüngliche Nachricht-----
Von: Andrey Zagrebin [mailto:andrey@data-artisans.com]
Gesendet: Donnerstag, 29. November 2018 10:39
An: Kostas Kloudas
Cc: Winterstein, Bernd; user; Stefan Richter; Till Rohrmann; Stephan
Ewen
Betreff: Re: number of files in checkpoint directory grows endlessly

Hi Bernd,

Did you change 'state.checkpoints.num-retained’ in flink-conf.yaml? By default, only one
checkpoint should be retained.

Which version of Flink do you use?
Can you check Job Master logs whether you see there warning like this:
`Fail to subsume the old checkpoint`?

Best,
Andrey


On 29 Nov 2018, at 10:18, Kostas Kloudas <k.kloudas@data-artisans.com<mailto:k.kloudas@data-artisans.com>>
wrote:

Hi Bernd,

I think the Till, Stefan or Stephan (cc'ed) are the best to answer your question.

Cheers,
Kostas

________________________________


Landesbank Hessen-Thueringen Girozentrale Anstalt des oeffentlichen
Rechts
Sitz: Frankfurt am Main / Erfurt
Amtsgericht Frankfurt am Main, HRA 29821 / Amtsgericht Jena, HRA
102181

Bitte nutzen Sie die E-Mail-Verbindung mit uns ausschliesslich zum Informationsaustausch.
Wir koennen auf diesem Wege keine rechtsgeschaeftlichen Erklaerungen (Auftraege etc.) entgegennehmen.

Der Inhalt dieser Nachricht ist vertraulich und nur fuer den angegebenen Empfaenger bestimmt.
Jede Form der Kenntnisnahme oder Weitergabe durch Dritte ist unzulaessig. Sollte diese Nachricht
nicht fur Sie bestimmt sein, so bitten wir Sie, sich mit uns per E-Mail oder telefonisch in
Verbindung zu setzen.

Please use your E-mail connection with us exclusively for the exchange of information. We
do not accept legally binding declarations (orders, etc.) by this means of communication.

The contents of this message is confidential and intended only for the
recipient indicated. Taking notice of this message or disclosure by third parties is not permitted.
In the event that this message is not intended for you, please contact us via E-mail or phone.

________________________________


Landesbank Hessen-Thueringen Girozentrale
Anstalt des oeffentlichen Rechts
Sitz: Frankfurt am Main / Erfurt
Amtsgericht Frankfurt am Main, HRA 29821 / Amtsgericht Jena, HRA 102181

Bitte nutzen Sie die E-Mail-Verbindung mit uns ausschliesslich zum Informationsaustausch.
Wir koennen auf diesem Wege keine rechtsgeschaeftlichen Erklaerungen (Auftraege etc.) entgegennehmen.

Der Inhalt dieser Nachricht ist vertraulich und nur fuer den angegebenen Empfaenger bestimmt.
Jede Form der Kenntnisnahme oder Weitergabe durch Dritte ist unzulaessig. Sollte diese Nachricht
nicht fur Sie bestimmt sein, so bitten wir Sie, sich mit uns per E-Mail oder telefonisch in
Verbindung zu setzen.

Please use your E-mail connection with us exclusively for the exchange of information. We
do not accept legally binding declarations (orders, etc.) by this means of communication.

The contents of this message is confidential and intended only for the recipient indicated.
Taking notice of this message or disclosure by third parties is not
permitted. In the event that this message is not intended for you, please contact us via E-mail
or phone.

<StpRocksDbOptions.java><RocksDBKeyedStateBackend.java><shared_file.bin.txt>

Mime
View raw message