[STREAM STORAGE] Ensure progress while restoring from checkpoint.

### Motivation

Sometimes the checkpoint restoration never completes. From the logs it looks
like it is stuck waiting for entries in distributedlog. But all the entries
have already been read. This looks like a race condition that causes the
reader to believe that there are more entries.

Although this can happen for any storage container, it is catastrophic when it
happens for root storage container. In that case all api calls start to fail.

### Changes

This happens very rarely and is difficult to debug. Here we are adding a
protection around this deadlock. We will monitor the input stream to see we
are getting new data. If new data is not received within specified amount of
time, we will fail the restore process. This will fail the starting of the
storage container. After failure we will restart the storage container again.

I ran into this issue during my testing and was able confirm that we are able
to recover from this.

```
2021-08-12T11:47:39.839+0000    org.apache.distributedlog.ReadAheadEntryReader  ReadAhead for 000000000000000000/000000000000000000/000000000000000000/checkpoints/cea4e286-5224-43ac-b6e0-7321d607f98e/MANIFEST-000004:<default> is caught up and no log segments to read now

2021-08-12T11:47:39.834+0000    org.apache.distributedlog.logsegment.PerStreamLogSegmentCache   000000000000000000/000000000000000000/000000000000000000/checkpoints/cea4e286-5224-43ac-b6e0-7321d607f98e/MANIFEST-000004 added log segment (logrecs_000000000000000001 : [LogSegmentId:2534, firstTxId:99, lastTxId:99, version:VERSION_V5_SEQUENCE_ID, completionTime:1628793488256, recordCount:1, regionId:0, status:0, logSegmentSequenceNumber:1, lastEntryId:0, lastSlotId:0, inprogress:false, minActiveDLSN:DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}, startSequenceId:0]) to cache.
2021-08-12T11:47:39.834+0000    org.apache.distributedlog.ReadAheadEntryReader  Starting the readahead entry reader for 000000000000000000/000000000000000000/000000000000000000/checkpoints/cea4e286-5224-43ac-b6e0-7321d607f98e/MANIFEST-000004:<default> : number of segments: 1, top 10 segments = [[LogSegmentId:2534, firstTxId:99, lastTxId:99, version:VERSION_V5_SEQUENCE_ID, completionTime:1628793488256, recordCount:1, regionId:0, status:0, logSegmentSequenceNumber:1, lastEntryId:0, lastSlotId:0, inprogress:false, minActiveDLSN:DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}, startSequenceId:0]]
2021-08-12T11:47:39.834+0000    org.apache.distributedlog.logsegment.PerStreamLogSegmentCache   000000000000000000/000000000000000000/000000000000000000/checkpoints/cea4e286-5224-43ac-b6e0-7321d607f98e/MANIFEST-000004 added log segment (logrecs_000000000000000001 : [LogSegmentId:2534, firstTxId:99, lastTxId:99, version:VERSION_V5_SEQUENCE_ID, completionTime:1628793488256, recordCount:1, regionId:0, status:0, logSegmentSequenceNumber:1, lastEntryId:0, lastSlotId:0, inprogress:false, minActiveDLSN:DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}, startSequenceId:0]) to cache.

2021-08-12T11:47:29.516+0000    org.apache.bookkeeper.stream.storage.impl.sc.ZkStorageContainerManager  Failed to start storage container (0)
2021-08-12T11:47:29.513+0000    org.apache.bookkeeper.stream.storage.impl.sc.StorageContainerRegistryImpl       De-registered StorageContainer ('0') when failed to start

2021-08-12T11:47:29.511+0000    org.apache.bookkeeper.statelib.impl.kv.RocksdbKVStore   Timeout waiting for checkpoint restore: Checkpoint{ID='cea4e286-5224-43ac-b6e0-7321d607f98e', createdAt: 1628793488360 files: "CURRENT"\nfiles: "MANIFEST-000004"\nfiles: "OPTIONS-000008"\ntxid: "\\000\\000\\000\\000\\000\\000\\000\\000"\ncreated_at: 1628793488360\nfileInfos {\n  name: "CURRENT"\n  checksum: "0861415cada612ea5834d56e2cf1055d3e63979b69eb71d32ae9ae394d8306cd"\n}\nfileInfos {\n  name: "MANIFEST-000004"\n  checksum: "7e4ff7d93941139236f69906d42a9e387dec565966b0017b6e8b16650c6c8be5"\n}\nfileInfos {\n  name: "OPTIONS-000008"\n  checksum: "41c8a054439d7798f88992a7a2dcb96f189cce0ed7ddb4c71af3594edc6a7a14"\n}\n}

2021-08-12T11:47:29.489+0000    org.apache.bookkeeper.statelib.impl.rocksdb.checkpoint.CheckpointFile   Timeout waiting for copy: 000000000000000000/000000000000000000/000000000000000000/checkpoints/cea4e286-5224-43ac-b6e0-7321d607f98e/MANIFEST-000004 last-read 0 current-read 0 runtime(ms) 300000

2021-08-12T11:42:29.467+0000    org.apache.distributedlog.ReadAheadEntryReader  Starting the readahead entry reader for 000000000000000000/000000000000000000/000000000000000000/checkpoints/cea4e286-5224-43ac-b6e0-7321d607f98e/MANIFEST-000004:<default> : number of segments: 1, top 10 segments = [[LogSegmentId:2534, firstTxId:99, lastTxId:99, version:VERSION_V5_SEQUENCE_ID, completionTime:1628793488256, recordCount:1, regionId:0, status:0, logSegmentSequenceNumber:1, lastEntryId:0, lastSlotId:0, inprogress:false, minActiveDLSN:DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}, startSequenceId:0]]
2021-08-12T11:42:29.466+0000    org.apache.distributedlog.logsegment.PerStreamLogSegmentCache   000000000000000000/000000000000000000/000000000000000000/checkpoints/cea4e286-5224-43ac-b6e0-7321d607f98e/MANIFEST-000004 added log segment (logrecs_000000000000000001 : [LogSegmentId:2534, firstTxId:99, lastTxId:99, version:VERSION_V5_SEQUENCE_ID, completionTime:1628793488256, recordCount:1, regionId:0, status:0, logSegmentSequenceNumber:1, lastEntryId:0, lastSlotId:0, inprogress:false, minActiveDLSN:DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}, startSequenceId:0]) to cache.
2021-08-12T11:42:29.466+0000    org.apache.distributedlog.logsegment.PerStreamLogSegmentCache   000000000000000000/000000000000000000/000000000000000000/checkpoints/cea4e286-5224-43ac-b6e0-7321d607f98e/MANIFEST-000004 added log segment (logrecs_000000000000000001 : [LogSegmentId:2534, firstTxId:99, lastTxId:99, version:VERSION_V5_SEQUENCE_ID, completionTime:1628793488256, recordCount:1, regionId:0, status:0, logSegmentSequenceNumber:1, lastEntryId:0, lastSlotId:0, inprogress:false, minActiveDLSN:DLSN{logSegmentSequenceNo=1, entryId=0, slotId=0}, startSequenceId:0]) to cache.

```



Reviewers: Ivan Kelly <ivank@apache.org>, Enrico Olivelli <eolivelli@gmail.com>

This closes #2764 from sursingh/statestore-restore-timeout and squashes the following commits:

0b5cbeed4 [Surinder Singh] Change default timout to 1 Minute.
38fe39f02 [Surinder Singh] Ensure progress while restoring from checkpoint.
10 files changed
tree: 1eb0271364545411e7ee92d1786d9d4f7e4da717
  1. .github/
  2. .test-infra/
  3. bin/
  4. bookkeeper-benchmark/
  5. bookkeeper-common/
  6. bookkeeper-common-allocator/
  7. bookkeeper-dist/
  8. bookkeeper-http/
  9. bookkeeper-proto/
  10. bookkeeper-server/
  11. bookkeeper-stats/
  12. bookkeeper-stats-providers/
  13. buildtools/
  14. circe-checksum/
  15. conf/
  16. cpu-affinity/
  17. deploy/
  18. dev/
  19. docker/
  20. gradle/
  21. metadata-drivers/
  22. microbenchmarks/
  23. shaded/
  24. site/
  25. site2/
  26. stats/
  27. stream/
  28. tests/
  29. tools/
  30. .gitignore
  31. build.gradle
  32. dependencies.gradle
  33. gradle.properties
  34. gradlew
  35. gradlew.bat
  36. Jenkinsfile
  37. LICENSE
  38. NOTICE
  39. pom.xml
  40. README.md
  41. settings.gradle
  42. version.gradle
README.md

Build Status Build Status Coverage Status Maven Central

Apache BookKeeper

Apache BookKeeper is a scalable, fault tolerant and low latency storage service optimized for append-only workloads.

It is suitable for being used in following scenarios:

  • WAL (Write-Ahead-Logging), e.g. HDFS NameNode.
  • Message Store, e.g. Apache Pulsar.
  • Offset/Cursor Store, e.g. Apache Pulsar.
  • Object/Blob Store, e.g. storing state machine snapshots.

Get Started

  • Checkout the project website.
  • Concepts: Start with the basic concepts of Apache BookKeeper. This will help you to fully understand the other parts of the documentation.
  • Follow the Install guide to setup BookKeeper.

Documentation

Please visit the Documentation from the project website for more information.

Get In Touch

Report a Bug

For filing bugs, suggesting improvements, or requesting new features, help us out by opening a Github issue or opening an Apache jira.

Need Help?

Subscribe or mail the user@bookkeeper.apache.org list - Ask questions, find answers, and also help other users.

Subscribe or mail the dev@bookkeeper.apache.org list - Join development discussions, propose new ideas and connect with contributors.

Join us on Slack - This is the most immediate way to connect with Apache BookKeeper committers and contributors.

Contributing

We feel that a welcoming open community is important and welcome contributions.

Contributing Code

  1. See Developer Setup to get your local environment setup.

  2. Take a look at our open issues: JIRA Issues Github Issues.

  3. Review our coding style and follow our pull requests to learn about our conventions.

  4. Make your changes according to our contribution guide.

Improving Website and Documentation

  1. See Building the website and documentation on how to build the website and documentation.