Skip to content

Conversation

@rdhabalia
Copy link
Contributor

@rdhabalia rdhabalia commented Feb 15, 2019

Motivation

It addresses #1938, where it provides interface for entyLogMetadataMap so, entrylogMetadata can be cached into RocksDB instead main-memory.

Modification

  • introduced RocksDB entryLogMetadataMap to cache entry-log metadata.
  • it can fallback to in-memory implementation using gcPersistentEntrylogMetadataMapEnabled flag and that can be removed in future release and keep only rocksDB implementation in future.

Result

  • It will help to avoid storing entryLogMetadata into main-memory.

cc @merlimat

@rdhabalia rdhabalia changed the title [bookie-gc] add option for persistent entry-log metadata [bookie-gc] add option for persistent entry-log metadata map Feb 15, 2019
@rdhabalia rdhabalia force-pushed the entry_rocks branch 2 times, most recently from b08ea07 to 700dc42 Compare February 15, 2019 23:45
Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did a first pass.

Please take a look

return new PersistentEntryLogMetadataMap(baseDir, conf);
}
} catch (IOException e) {
LOG.error("Failed to initialize persistent-metadata-map , clean up {}", baseDir, e);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it is not safe to fallback to "inMemory" in case of errors,
if you are using the persistent memory you don't want to use the in memory one (will the bookie crash with OutOfMemory errors?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(will the bookie crash with OutOfMemory errors?

No, it's not necessary that it will throw OOM because using inMemory is same as current logic where GC stores metadata-map into hashmap. in order to solve in-memory scale issue, we can also add this patch #1938.

I think it is not safe to fallback to "inMemory" in case of errors,

I think it can be helpful in case of rocksDB corruption and user will not be impacted in such cases and GC can still continue. It will not create any issue because entryLogMetadataMap is any way transient and recovered by extracting entry-log files.

Copy link

@athanatos athanatos Feb 20, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think that's a good idea. As a general rule, rocksdb is fairly robust. If rocksdb is corrupt, likely there is a hardware problem. The user should have to manually switch the config value back to in-memory if they want to bypass the problem.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Palese address this comment

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This comment looks unresolved

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@rdhabalia please address this comment from me and from @athanatos
cc @merlimat

@jvrao
Copy link
Contributor

jvrao commented Feb 17, 2019

@reddycharan can you please review this? This is larger change and we don't use rocksDb, lot of code reorg. @sijie @merlimat do we seriously have an issue where we need to handle large number of entrylogs ? This all came out of ZK/GC issue and it will be taken care.

@jvrao
Copy link
Contributor

jvrao commented Feb 17, 2019

keep only rocksDB implementation in future.

I don't think that is a good option. Can we talk about this in the next week's meeting?
For now I am not a fan of this change.

@merlimat
Copy link
Contributor

@sijie @merlimat do we seriously have an issue where we need to handle large number of entrylogs ? This all came out of ZK/GC issue and it will be taken care.

I haven't checked this PR yet (will do soon :) ).

The main issue is that since the entryLogMaps for each entryLog are kept in memory. The amount of memory needed by a bookie is directly proportional to the amount of data stored. Your milage might vary: if you have all your data in 10 ledgers this will require less memory that if you have it in 10M ledgers, because for each entryLog we track (ledgerId -> sizeOfEntriesInThisLog)..

In any case, I believe this is a fundamental issue with the current approach, in that we need to be able to have a fixed amount of memory used, independent of the amount of data.

In many cases, with bookies holding >10TB we saw this very high mem usage, leading to issue where the heap size is not enough and bookie basically restart and goes into a loop of GC leading to OOM.

@eolivelli
Copy link
Contributor

I am ok with Rocks DB.

In other projects where I need huge CuncurrentMap I am using MapDB

@rdhabalia rdhabalia force-pushed the entry_rocks branch 2 times, most recently from e08c2af to 4ed98e8 Compare February 18, 2019 23:22
@rdhabalia
Copy link
Contributor Author

@eolivelli
addressed your all comments.

@rdhabalia rdhabalia force-pushed the entry_rocks branch 2 times, most recently from 87a0cda to fa24177 Compare February 19, 2019 01:30
private EntryLogMetadataMap createEntryLogMetadataMap(boolean gcPersistentEntrylogMetadataMapEnabled) {
final String baseDir = this.conf.getGcPersistentEntrylogMetadataMapPath() != null
? this.conf.getGcPersistentEntrylogMetadataMapPath()
: this.entryLogger.getLedgerDirsManager().getAllLedgerDirs().get(0).toString();
Copy link
Contributor

@reddycharan reddycharan Feb 20, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

since you are adding new persistence state to the bookie - 'PersistentEntrylogMetadataMapPath', have you thought through all the persistence scenarios? like any changes required to cookie, what it means for bookie formatting, backward compatibility / forward compatibility scenarios, possible duplications because of misconfigurations and strategies to overcome this...

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think bookie-formatting is the only operation that requires metadata cleanup for which I made the change. also, GC has fall-back mechanism to use InMemoryMap in case of PersistentMap is not accessible and PersistentMap is recoverable by extracting entryLog so, that should take care of forward/backward compatibility concerns.

if (entryLogMeta.isEmpty()) {
entryLogger.removeEntryLog(entryLogId);
// remove it from entrylogmetadata-map if it presents into map
removeEntryLogRecordSafely(entryLogId);

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't understand why you want to swallow the exception here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can avoid failing entire gc-task due to single record failure and gc should continue by ignoring that record because that will not cause any harm except it will not remove that specific entry/ledger in that gc-run.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did you ever see this case in production?
If we do not have a strong motivation O prefer to leave the code as before here

@athanatos
Copy link

athanatos commented Feb 20, 2019

I'd like to suggest an alternate approach. We already in fact have the ledger map for each entry log compiled at the end of each file. Suppose instead of adding rocksdb, you use some bounded auto-loading cache (evict newest entry rather than oldest to avoid thrashing the cache during scan) which falls back to simply re-reading the index for the given EntryLog? It would smoothly transition between fully in-memory and spilling to disk behaviors as needed. By introducing rocksdb, you introduce a new way for bookie state to be corrupted -- inconsistency between entry log and cached ledger map. You also introduce rocksdb itself as a source of overhead -- it's got its own writer thread and journal as well as having to do its own compaction.

@eolivelli eolivelli requested a review from merlimat February 23, 2021 17:39
Copy link
Contributor

@dlg99 dlg99 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall LGTM.
A few comments but the idea lgtm.
Now we have to figure out how to deal with case of bookie running out of disk space and not having disk and memory to handle all the metadata for compaction :)

out.writeLong(remainingSize);
out.writeLong(ledgersMap.size());
ledgersMap.forEach((ledgerId, size) -> {
try {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this try/catch should be around whole method body, i.e. out.writeShort(DEFAULT_SERIALIZATION_VERSION) as well as out.flush() can throw IOException too.
Overall, I'd either leave it at IOException and not catch anything (and pass IOException up) or catch everything and throw something like SerializationException

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

serialize method anyway throws IOException so, no need to catch IOException for out.flush(). the only reason of try/catch and throwing runtime-exception is because it's in lambda and it requires runtime-exception.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Probably this is one of the reason to avoid lamba here. Just a normal iteration should be just fine.

removeIfLedgerNotExists(entryLogMeta);
if (entryLogMeta.isEmpty()) {
entryLogger.removeEntryLog(entryLogId);
// remove it from entrylogmetadata-map if it presents into map
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: "remove it from entrylogmetadata-map if it is present in the map"

if (conf.isGcEntryLogMetadataCacheEnabled()) {
String baseDir = this.conf.getGcEntryLogMetadataCachePath();
try {
return new PersistentEntryLogMetadataMap(baseDir, conf);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are you trying to preserve data on the disk between bookie restarts?
If not (I assume) it makes sense to explicitly clean up the baseDir and avoid situations like dealing with corrupt data, recovery from crashes, from config switches between persistent and in-memory metadata map, etc

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please consider adding tests for such scenarios (bookie stop, RocksDB corruption, bookie starts) (persistent to in-memory to persistent switch, GC runs successfully before and after, collects everything, does not collect i.e. entry logs added while using in-memory). Sorry if I missed these in the existing tests, I am still going through the code

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are you trying to preserve data on the disk between bookie restarts?

yes. and we should prevent auto-cleanup because this can be also failed if by mistake multiple processes try to read rocks-db same time then it may fail intermittently. so, to avoid such scenarios, let user cleanup rocksDB cache manually if needed. generally rocksDB is robust and it should not fail in normal conditions.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we configured multi ledger directories, all the ledger directories' PersistentEntryLogMetadataMap use the same RocksDB and located in the first ledger directory.

When we do forEach operation for PersistentEntryLogMetadataMap, it will iterate all the items in RocksDB. However most of items we iterated are not belongs to current ledger storage, it will wast of time and cpu resource.

We'd better separate the PersistentEntryLogMetadataMap's RocksDB for each ledger directory. Do you have any ideas? @rdhabalia

@rdhabalia
Copy link
Contributor Author

@eolivelli @sijie can we merge this PR? we can also create separate PRs if we need any enhancement or improvement on top of it.?

@eolivelli
Copy link
Contributor

@sijie I would like to merge this feature, WDYT?

/**
* Set directory to persist Entrylog metadata if gcPersistentEntrylogMetadataMapEnabled is true.
*
* @param gcPersistentEntrylogMetadataMapPath.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@param gcEntrylogMetadataCachePath ?

if (conf.isGcEntryLogMetadataCacheEnabled()) {
String baseDir = this.conf.getGcEntryLogMetadataCachePath();
try {
return new PersistentEntryLogMetadataMap(baseDir, conf);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we configured multi ledger directories, all the ledger directories' PersistentEntryLogMetadataMap use the same RocksDB and located in the first ledger directory.

When we do forEach operation for PersistentEntryLogMetadataMap, it will iterate all the items in RocksDB. However most of items we iterated are not belongs to current ledger storage, it will wast of time and cpu resource.

We'd better separate the PersistentEntryLogMetadataMap's RocksDB for each ledger directory. Do you have any ideas? @rdhabalia

@rdhabalia
Copy link
Contributor Author

@hangc0276 sure, we can make that enhancement, but first I would like to merge this PR then we can make any enhancement as this PR is sitting for a while. we are patching our internal release to protect ourselves. so, it would be great if we can merge it first and then make any required enhancement.

@eolivelli @sijie can we merge this PR?

@eolivelli
Copy link
Contributor

It looks like this patch needs a rebase @rdhabalia

Also @sijie left 'request changes', so we have to check if his concerns have been fixed

@rdhabalia
Copy link
Contributor Author

I think I addressed all the comments. Actually, I had rebased it multiple times so, if it looks fine for two +1 then I will rebase it and then we can merge it.

try {
removeIfLedgerNotExists(meta);
// update entryMetadta to persistent-map
entryLogMetaMap.put(meta.getEntryLogId(), meta);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we can update it only when the meta is changed?

@rdhabalia
Copy link
Contributor Author

@sijie @eolivelli can you please let me know if all the changes are addressed to unblock the PR. it's hard to keep rebasing and keep addressing new comments for a year.

@zymap
Copy link
Member

zymap commented Dec 21, 2021

Hi @rdhabalia, could you please resolve the conflicts?

address comments

address comments: config + entryLogMetadata-recyclable + bookie-format-metadata-cleanup

add doc at deserialize method

rename cache paramter

fix config

address comments

address comment: remove Exception-catch + fail on rocksDB init
@rdhabalia rdhabalia force-pushed the entry_rocks branch 3 times, most recently from 41fa08a to 2eef125 Compare December 21, 2021 22:31
@zymap zymap merged commit 370d785 into apache:master Dec 22, 2021
@zymap zymap added this to the 4.15.0 milestone Jan 5, 2022
zymap pushed a commit that referenced this pull request Feb 11, 2022
…#2965)

### Motivation
When we use RocksDB backend entryMetadataMap for multi ledger directories configured, the bookie start up failed, and throw the following exception.
```
12:24:28.530 [main] ERROR org.apache.pulsar.PulsarStandaloneStarter - Failed to start pulsar service.
java.io.IOException: Error open RocksDB database
        at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.<init>(KeyValueStorageRocksDB.java:202) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.<init>(KeyValueStorageRocksDB.java:89) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.lambda$static$0(KeyValueStorageRocksDB.java:62) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.PersistentEntryLogMetadataMap.<init>(PersistentEntryLogMetadataMap.java:87) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.createEntryLogMetadataMap(GarbageCollectorThread.java:265) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.<init>(GarbageCollectorThread.java:154) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.<init>(GarbageCollectorThread.java:133) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.<init>(SingleDirectoryDbLedgerStorage.java:182) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage.newSingleDirectoryDbLedgerStorage(DbLedgerStorage.java:190) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage.initialize(DbLedgerStorage.java:150) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.BookieResources.createLedgerStorage(BookieResources.java:110) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble.buildBookie(LocalBookkeeperEnsemble.java:328) ~[org.apache.pulsar-pulsar-zookeeper-utils-2.8.1.jar:2.8.1]
        at org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble.runBookies(LocalBookkeeperEnsemble.java:391) ~[org.apache.pulsar-pulsar-zookeeper-utils-2.8.1.jar:2.8.1]
        at org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble.startStandalone(LocalBookkeeperEnsemble.java:521) ~[org.apache.pulsar-pulsar-zookeeper-utils-2.8.1.jar:2.8.1]
        at org.apache.pulsar.PulsarStandalone.start(PulsarStandalone.java:264) ~[org.apache.pulsar-pulsar-broker-2.8.1.jar:2.8.1]
        at org.apache.pulsar.PulsarStandaloneStarter.main(PulsarStandaloneStarter.java:121) [org.apache.pulsar-pulsar-broker-2.8.1.jar:2.8.1]
Caused by: org.rocksdb.RocksDBException: lock hold by current process, acquire time 1640492668 acquiring thread 123145515651072: data/standalone/bookkeeper00/entrylogIndexCache/metadata-cache/LOCK: No locks available
        at org.rocksdb.RocksDB.open(Native Method) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
        at org.rocksdb.RocksDB.open(RocksDB.java:239) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
        at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.<init>(KeyValueStorageRocksDB.java:199) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        ... 15 more
```

The reason is multi garbageCollectionThread will open the same RocksDB and own the LOCK, and then throw the above exception.

### Modification
1. Change the default GcEntryLogMetadataCachePath from `getLedgerDirNames()[0] + "/" + ENTRYLOG_INDEX_CACHE` to  `null`. If it is `null`, it will use each ledger's directory.
2. Remove the internal directory `entrylogIndexCache`. The data structure looks like: 
```
   └── current
       ├── lastMark
       ├── ledgers
       │   ├── 000003.log
       │   ├── CURRENT
       │   ├── IDENTITY
       │   ├── LOCK
       │   ├── LOG
       │   ├── MANIFEST-000001
       │   └── OPTIONS-000005
       ├── locations
       │   ├── 000003.log
       │   ├── CURRENT
       │   ├── IDENTITY
       │   ├── LOCK
       │   ├── LOG
       │   ├── MANIFEST-000001
       │   └── OPTIONS-000005
       └── metadata-cache
           ├── 000003.log
           ├── CURRENT
           ├── IDENTITY
           ├── LOCK
           ├── LOG
           ├── MANIFEST-000001
           └── OPTIONS-000005
```
3. If user configured `GcEntryLogMetadataCachePath` in `bk_server.conf`, it only support one ledger directory configured for `ledgerDirectories`. Otherwise, the best practice is to keep it default.
4. The PR is better to release with #1949
StevenLuMT pushed a commit to StevenLuMT/bookkeeper that referenced this pull request Feb 16, 2022
…apache#2965)

When we use RocksDB backend entryMetadataMap for multi ledger directories configured, the bookie start up failed, and throw the following exception.
```
12:24:28.530 [main] ERROR org.apache.pulsar.PulsarStandaloneStarter - Failed to start pulsar service.
java.io.IOException: Error open RocksDB database
        at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.<init>(KeyValueStorageRocksDB.java:202) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.<init>(KeyValueStorageRocksDB.java:89) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.lambda$static$0(KeyValueStorageRocksDB.java:62) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.PersistentEntryLogMetadataMap.<init>(PersistentEntryLogMetadataMap.java:87) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.createEntryLogMetadataMap(GarbageCollectorThread.java:265) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.<init>(GarbageCollectorThread.java:154) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.<init>(GarbageCollectorThread.java:133) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.<init>(SingleDirectoryDbLedgerStorage.java:182) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage.newSingleDirectoryDbLedgerStorage(DbLedgerStorage.java:190) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage.initialize(DbLedgerStorage.java:150) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.BookieResources.createLedgerStorage(BookieResources.java:110) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble.buildBookie(LocalBookkeeperEnsemble.java:328) ~[org.apache.pulsar-pulsar-zookeeper-utils-2.8.1.jar:2.8.1]
        at org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble.runBookies(LocalBookkeeperEnsemble.java:391) ~[org.apache.pulsar-pulsar-zookeeper-utils-2.8.1.jar:2.8.1]
        at org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble.startStandalone(LocalBookkeeperEnsemble.java:521) ~[org.apache.pulsar-pulsar-zookeeper-utils-2.8.1.jar:2.8.1]
        at org.apache.pulsar.PulsarStandalone.start(PulsarStandalone.java:264) ~[org.apache.pulsar-pulsar-broker-2.8.1.jar:2.8.1]
        at org.apache.pulsar.PulsarStandaloneStarter.main(PulsarStandaloneStarter.java:121) [org.apache.pulsar-pulsar-broker-2.8.1.jar:2.8.1]
Caused by: org.rocksdb.RocksDBException: lock hold by current process, acquire time 1640492668 acquiring thread 123145515651072: data/standalone/bookkeeper00/entrylogIndexCache/metadata-cache/LOCK: No locks available
        at org.rocksdb.RocksDB.open(Native Method) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
        at org.rocksdb.RocksDB.open(RocksDB.java:239) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
        at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.<init>(KeyValueStorageRocksDB.java:199) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        ... 15 more
```

The reason is multi garbageCollectionThread will open the same RocksDB and own the LOCK, and then throw the above exception.

1. Change the default GcEntryLogMetadataCachePath from `getLedgerDirNames()[0] + "/" + ENTRYLOG_INDEX_CACHE` to  `null`. If it is `null`, it will use each ledger's directory.
2. Remove the internal directory `entrylogIndexCache`. The data structure looks like:
```
   └── current
       ├── lastMark
       ├── ledgers
       │   ├── 000003.log
       │   ├── CURRENT
       │   ├── IDENTITY
       │   ├── LOCK
       │   ├── LOG
       │   ├── MANIFEST-000001
       │   └── OPTIONS-000005
       ├── locations
       │   ├── 000003.log
       │   ├── CURRENT
       │   ├── IDENTITY
       │   ├── LOCK
       │   ├── LOG
       │   ├── MANIFEST-000001
       │   └── OPTIONS-000005
       └── metadata-cache
           ├── 000003.log
           ├── CURRENT
           ├── IDENTITY
           ├── LOCK
           ├── LOG
           ├── MANIFEST-000001
           └── OPTIONS-000005
```
3. If user configured `GcEntryLogMetadataCachePath` in `bk_server.conf`, it only support one ledger directory configured for `ledgerDirectories`. Otherwise, the best practice is to keep it default.
4. The PR is better to release with apache#1949
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
…pache#1949)

### Motivation

It addresses apache#1938, where it provides interface for `entyLogMetadataMap` so, `entrylogMetadata` can be cached into RocksDB instead main-memory.

### Modification
- introduced RocksDB entryLogMetadataMap to cache entry-log metadata.
- it can fallback to in-memory implementation using  `gcPersistentEntrylogMetadataMapEnabled` flag and that can be removed in future release and keep only rocksDB implementation in future.

### Result
- It will help to avoid storing entryLogMetadata into main-memory.
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
…apache#2965)

### Motivation
When we use RocksDB backend entryMetadataMap for multi ledger directories configured, the bookie start up failed, and throw the following exception.
```
12:24:28.530 [main] ERROR org.apache.pulsar.PulsarStandaloneStarter - Failed to start pulsar service.
java.io.IOException: Error open RocksDB database
        at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.<init>(KeyValueStorageRocksDB.java:202) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.<init>(KeyValueStorageRocksDB.java:89) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.lambda$static$0(KeyValueStorageRocksDB.java:62) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.PersistentEntryLogMetadataMap.<init>(PersistentEntryLogMetadataMap.java:87) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.createEntryLogMetadataMap(GarbageCollectorThread.java:265) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.<init>(GarbageCollectorThread.java:154) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.GarbageCollectorThread.<init>(GarbageCollectorThread.java:133) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.<init>(SingleDirectoryDbLedgerStorage.java:182) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage.newSingleDirectoryDbLedgerStorage(DbLedgerStorage.java:190) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage.initialize(DbLedgerStorage.java:150) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.bookkeeper.bookie.BookieResources.createLedgerStorage(BookieResources.java:110) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        at org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble.buildBookie(LocalBookkeeperEnsemble.java:328) ~[org.apache.pulsar-pulsar-zookeeper-utils-2.8.1.jar:2.8.1]
        at org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble.runBookies(LocalBookkeeperEnsemble.java:391) ~[org.apache.pulsar-pulsar-zookeeper-utils-2.8.1.jar:2.8.1]
        at org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble.startStandalone(LocalBookkeeperEnsemble.java:521) ~[org.apache.pulsar-pulsar-zookeeper-utils-2.8.1.jar:2.8.1]
        at org.apache.pulsar.PulsarStandalone.start(PulsarStandalone.java:264) ~[org.apache.pulsar-pulsar-broker-2.8.1.jar:2.8.1]
        at org.apache.pulsar.PulsarStandaloneStarter.main(PulsarStandaloneStarter.java:121) [org.apache.pulsar-pulsar-broker-2.8.1.jar:2.8.1]
Caused by: org.rocksdb.RocksDBException: lock hold by current process, acquire time 1640492668 acquiring thread 123145515651072: data/standalone/bookkeeper00/entrylogIndexCache/metadata-cache/LOCK: No locks available
        at org.rocksdb.RocksDB.open(Native Method) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
        at org.rocksdb.RocksDB.open(RocksDB.java:239) ~[org.rocksdb-rocksdbjni-6.10.2.jar:?]
        at org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB.<init>(KeyValueStorageRocksDB.java:199) ~[org.apache.bookkeeper-bookkeeper-server-4.15.0-SNAPSHOT.jar:4.15.0-SNAPSHOT]
        ... 15 more
```

The reason is multi garbageCollectionThread will open the same RocksDB and own the LOCK, and then throw the above exception.

### Modification
1. Change the default GcEntryLogMetadataCachePath from `getLedgerDirNames()[0] + "/" + ENTRYLOG_INDEX_CACHE` to  `null`. If it is `null`, it will use each ledger's directory.
2. Remove the internal directory `entrylogIndexCache`. The data structure looks like: 
```
   └── current
       ├── lastMark
       ├── ledgers
       │   ├── 000003.log
       │   ├── CURRENT
       │   ├── IDENTITY
       │   ├── LOCK
       │   ├── LOG
       │   ├── MANIFEST-000001
       │   └── OPTIONS-000005
       ├── locations
       │   ├── 000003.log
       │   ├── CURRENT
       │   ├── IDENTITY
       │   ├── LOCK
       │   ├── LOG
       │   ├── MANIFEST-000001
       │   └── OPTIONS-000005
       └── metadata-cache
           ├── 000003.log
           ├── CURRENT
           ├── IDENTITY
           ├── LOCK
           ├── LOG
           ├── MANIFEST-000001
           └── OPTIONS-000005
```
3. If user configured `GcEntryLogMetadataCachePath` in `bk_server.conf`, it only support one ledger directory configured for `ledgerDirectories`. Otherwise, the best practice is to keep it default.
4. The PR is better to release with apache#1949
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.