streamnative / pulsar-archived

Apache Pulsar - distributed pub-sub messaging system
https://pulsar.apache.org
Apache License 2.0
73 stars 26 forks source link

ISSUE-13693: Broker LedgerHandle internal metadata is inconsistent with latest ledger metadata in zk #3542

Open sijie opened 2 years ago

sijie commented 2 years ago

Original Issue: apache/pulsar#13693


Describe the bug In our staging cluster, some bookie instances keep logging No ledger found while reading entry: xx from ledger: xxxx, I did some investigation and found the internal metadata of LedgerHandle(org.apache.bookkeeper.client.LedgerHandle#versionedMetadata) of specific ledgers are inconsistent with latest ledger metadata in zookeeper.

Screenshots Bookie logs: image On that bookie, it says the readEntry request is from 172.30.10.3:48336 which is a broker instance.

[arthas@2278]$ watch org.apache.bookkeeper.proto.ReadEntryProcessorV3 getReadResponse {target.ledgerId,target.entryId,target.channel,returnObj.status} returnObj.status_!=0
Press Q or Ctrl+C to abort.
Affect(class count: 2 , method count: 1) cost in 225 ms, listenerId: 1
method=org.apache.bookkeeper.proto.ReadEntryProcessorV3.getReadResponse location=AtExit
ts=2022-01-10 14:07:30; [cost=1.716101ms] result=@ArrayList[
    @Long[4976503],
    @Long[0],
    @EpollSocketChannel[[id: 0xce78ecda, L:/172.30.10.2:3181 - R:/172.30.10.3:48336]],
    @StatusCode[ENOLEDGER],
]

On broker instance 172.30.10.3:8080, it says ensembles are 172.30.10.5:3181, 172.30.10.2:3181:

[arthas@2045]$ watch org.apache.bookkeeper.client.LedgerHandle asyncReadEntries {target.ledgerId,target.versionedMetadata} target.ledgerId==4976503
Press Q or Ctrl+C to abort.
Affect(class count: 3 , method count: 1) cost in 289 ms, listenerId: 2
method=org.apache.bookkeeper.client.LedgerHandle.asyncReadEntries location=AtExit
ts=2022-01-10 15:27:59; [cost=0.094076ms] result=@ArrayList[
    @Long[4976503],
    @Versioned[Versioned(value=LedgerMetadata{formatVersion=3, ensembleSize=2, writeQuorumSize=2, ackQuorumSize=2, state=CLOSED, length=300, lastEntryId=0, digestType=CRC32, password=base64:, ensembles={0=[172.30.10.5:3181, 172.30.10.2:3181]}, customMetadata={pulsar/compactedTopic=base64:cGVyc2lzdGVudDovL2NhbXB1cy9ic2l0ZS9fX2NoYW5nZV9ldmVudHMtcGFydGl0aW9uLTE=, component=base64:Y29tcGFjdGVkLWxlZGdlcg==, application=base64:cHVsc2Fy, pulsar/compactedTo=base64:CNnerwIQARgBMAA=}}, version=1)],
]

But the latest ledger metadata shows ensembles are 172.30.10.4:3181, 172.30.10.3:3181:

# curl -sS 'http://172.30.10.2:8000/api/v1/ledger/metadata?ledger_id=4976503'
{
  "4976503" : {
    "ledgerId" : 4976503,
    "metadataFormatVersion" : 3,
    "ensembleSize" : 2,
    "writeQuorumSize" : 2,
    "ackQuorumSize" : 2,
    "state" : "CLOSED",
    "length" : 300,
    "lastEntryId" : 0,
    "ctime" : 1640845770828,
    "digestType" : "CRC32",
    "password" : "",
    "customMetadata" : {
      "pulsar/compactedTopic" : "cGVyc2lzdGVudDovL2NhbXB1cy9ic2l0ZS9fX2NoYW5nZV9ldmVudHMtcGFydGl0aW9uLTE=",
      "component" : "Y29tcGFjdGVkLWxlZGdlcg==",
      "application" : "cHVsc2Fy",
      "pulsar/compactedTo" : "CNnerwIQARgBMAA="
    },
    "closed" : true,
    "allEnsembles" : {
      "0" : [ {
        "id" : "172.30.10.4:3181"
      }, {
        "id" : "172.30.10.3:3181"
      } ]
    },
    "ctoken" : 0
  }
}

So the broker keep sent readEntry requests to wrong bookies which are not in ledger ensemble list.

Desktop (please complete the following information):

Additional context Bookie auto-recovery is enabled.

Machine 172.30.10.2 has crashed once at 2022-01-03 17:40:08 +08:00, and bookie instance started at 2022-01-03 19:35:08 +08:00, No ledger found log start shows after 172.30.10.2 crashed.

Possible related https://github.com/apache/pulsar/issues/7214

github-actions[bot] commented 2 years ago

The issue had no activity for 30 days, mark with Stale label.