Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix lost message issues 12221 #12223

Merged
merged 10 commits into from
Sep 30, 2021
Merged

Conversation

baomingyu
Copy link
Contributor

@baomingyu baomingyu commented Sep 28, 2021

Fixes #12221

(or if this PR is one task of a github issue, please add Master Issue: #<xyz> to link to the master issue.)

Master Issue: #12221

Motivation

There are three way to create new ledger when one ledger is full.
first: after add last entry for full ledger, it will handle create ledger processing.
second: check ledger is or not full, when ledger is full and appendingAddEntries is not empty, it will handle create ledger processing.
third: check ledger is or not full, when ledger is full, it will handle create ledger processing.

So we need check it is or not creating or has created when concurrent create new ledger. avoid to create three new ledgers one time.

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.

Lgtm

}
}

synchronized boolean isNeededCreateNewLedgerAfterCloseLedger() {
Copy link
Contributor

Choose a reason for hiding this comment

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

It seems that there is no need to add synchronized here
The following methods have synchronized, isNeededCreateNewLedgerAfterCloseLedger() is called inside them:

createLedgerAfterClosed()
ledgerClosed(final LedgerHandle lh)
internalAsyncAddEntry(OpAddEntry addOperation)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yes, now it is not neccessary to add synchronized before isNeededCreateNewLedgerAfterCloseLedger method. and add synchronized now avoid it is used by others.

Copy link
Contributor

Choose a reason for hiding this comment

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

We don't know the logic of subsequent calls to this method, whether the same lock (ML object) is used.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok, i change it later.

Copy link
Contributor

@315157973 315157973 left a comment

Choose a reason for hiding this comment

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

LGTM

Copy link
Contributor

@hangc0276 hangc0276 left a comment

Choose a reason for hiding this comment

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

Good job!

@codelipenghui codelipenghui added this to the 2.9.0 milestone Sep 30, 2021
@codelipenghui codelipenghui added release/2.8.2 type/bug The PR fixed a bug or issue reported a bug labels Sep 30, 2021
@codelipenghui codelipenghui merged commit eb9d9d4 into apache:master Sep 30, 2021
codelipenghui pushed a commit that referenced this pull request Oct 6, 2021
Fixes #12221

*(or if this PR is one task of a github issue, please add `Master Issue: #<xyz>` to link to the master issue.)*

Master Issue: #12221 

### Motivation
There  are three way to create new ledger when one ledger is full.
first: after add last entry for full ledger, it will handle create ledger processing.
second: check ledger is or not full, when ledger is full and appendingAddEntries is not empty, it will handle create ledger processing.
third: check ledger is or not full, when ledger is full, it will handle create ledger processing.

 So we need check it is or not creating or has created when concurrent create new ledger. avoid to create three new ledgers one  time.

(cherry picked from commit eb9d9d4)
@codelipenghui codelipenghui added the cherry-picked/branch-2.8 Archived: 2.8 is end of life label Oct 6, 2021
nicoloboschi pushed a commit to datastax/pulsar that referenced this pull request Oct 27, 2021
Fixes apache#12221

*(or if this PR is one task of a github issue, please add `Master Issue: #<xyz>` to link to the master issue.)*

Master Issue: apache#12221

### Motivation
There  are three way to create new ledger when one ledger is full.
first: after add last entry for full ledger, it will handle create ledger processing.
second: check ledger is or not full, when ledger is full and appendingAddEntries is not empty, it will handle create ledger processing.
third: check ledger is or not full, when ledger is full, it will handle create ledger processing.

 So we need check it is or not creating or has created when concurrent create new ledger. avoid to create three new ledgers one  time.

(cherry picked from commit eb9d9d4)
(cherry picked from commit e3c8386)
@wolfstudy
Copy link
Member

Found one Java-level deadlock:
=============================
"ZKC-connect-executor-0-SendThread(9.142.172.233:2181)":
  waiting to lock monitor 0x00007fa3fc033bd8 (object 0x00007fad5e804158, a org.apache.zookeeper.ZooKeeper$States),
  which is held by "PullMessageThread_167"
"PullMessageThread_167":
  waiting to lock monitor 0x00007fbe40026668 (object 0x00007fb596023c08, a java.util.concurrent.LinkedBlockingQueue),
  which is held by "ZKC-connect-executor-0-SendThread(9.142.172.233:2181)"

Java stack information for the threads listed above:
===================================================
"ZKC-connect-executor-0-SendThread(9.142.172.233:2181)":
	at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1678)
	- waiting to lock <0x00007fad5e804158> (a org.apache.zookeeper.ZooKeeper$States)
	at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1649)
	at org.apache.zookeeper.ZooKeeper.create(ZooKeeper.java:1905)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$10.zkRun(ZooKeeperClient.java:752)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$ZkRetryRunnable.run(ZooKeeperClient.java:392)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient.create(ZooKeeperClient.java:762)
	at org.apache.bookkeeper.util.ZkUtils.asyncCreateFullPathOptimistic(ZkUtils.java:75)
	at org.apache.bookkeeper.meta.ZkLedgerIdGenerator.generateLedgerIdImpl(ZkLedgerIdGenerator.java:78)
	at org.apache.bookkeeper.meta.ZkLedgerIdGenerator.generateLedgerId(ZkLedgerIdGenerator.java:73)
	at org.apache.bookkeeper.meta.LongZkLedgerIdGenerator.generateLedgerId(LongZkLedgerIdGenerator.java:301)
	at org.apache.bookkeeper.client.LedgerCreateOp.generateLedgerIdAndCreateLedger(LedgerCreateOp.java:194)
	at org.apache.bookkeeper.client.LedgerCreateOp.initiate(LedgerCreateOp.java:182)
	at org.apache.bookkeeper.client.BookKeeper.asyncCreateLedger(BookKeeper.java:860)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncCreateLedger(ManagedLedgerImpl.java:3645)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createLedgerAfterClosed(ManagedLedgerImpl.java:1596)
	- locked <0x00007fadbdfa7b90> (a org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.ledgerClosed(ManagedLedgerImpl.java:1587)
	- locked <0x00007fadbdfa7b90> (a org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl)
	at org.apache.bookkeeper.mledger.impl.OpAddEntry.closeComplete(OpAddEntry.java:236)
	at org.apache.bookkeeper.client.LedgerHandle$5.lambda$safeRun$0(LedgerHandle.java:552)
	at org.apache.bookkeeper.client.LedgerHandle$5$$Lambda$1557/731657311.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.bookkeeper.client.LedgerHandle$5.lambda$safeRun$3(LedgerHandle.java:614)
	at org.apache.bookkeeper.client.LedgerHandle$5$$Lambda$1563/295431727.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.bookkeeper.client.MetadataUpdateLoop.lambda$writeLoop$1(MetadataUpdateLoop.java:161)
	at org.apache.bookkeeper.client.MetadataUpdateLoop$$Lambda$1562/765900306.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.bookkeeper.meta.AbstractZkLedgerManager$4.processResult(AbstractZkLedgerManager.java:508)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$22$1.processResult(ZooKeeperClient.java:1094)
	at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:638)
	at org.apache.zookeeper.ClientCnxn$EventThread.queuePacket(ClientCnxn.java:541)
	- locked <0x00007fb596023c08> (a java.util.concurrent.LinkedBlockingQueue)
	at org.apache.zookeeper.ClientCnxn.finishPacket(ClientCnxn.java:781)
	at org.apache.zookeeper.ClientCnxn.conLossPacket(ClientCnxn.java:818)
	at org.apache.zookeeper.ClientCnxn.access$2600(ClientCnxn.java:106)
	at org.apache.zookeeper.ClientCnxn$SendThread.cleanup(ClientCnxn.java:1403)
	at org.apache.zookeeper.ClientCnxn$SendThread.cleanAndNotifyState(ClientCnxn.java:1331)
	at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1309)
"PullMessageThread_167":
	at org.apache.zookeeper.ClientCnxn$EventThread.queuePacket(ClientCnxn.java:537)
	- waiting to lock <0x00007fb596023c08> (a java.util.concurrent.LinkedBlockingQueue)
	at org.apache.zookeeper.ClientCnxn.finishPacket(ClientCnxn.java:781)
	at org.apache.zookeeper.ClientCnxn.conLossPacket(ClientCnxn.java:818)
	at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1680)
	- locked <0x00007fad5e804158> (a org.apache.zookeeper.ZooKeeper$States)
	at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1649)
	at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:2411)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$19.zkRun(ZooKeeperClient.java:1009)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$ZkRetryRunnable.run(ZooKeeperClient.java:392)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient.getData(ZooKeeperClient.java:1019)
	at org.apache.bookkeeper.meta.AbstractZkLedgerManager.readLedgerMetadata(AbstractZkLedgerManager.java:435)
	at org.apache.bookkeeper.meta.AbstractZkLedgerManager.readLedgerMetadata(AbstractZkLedgerManager.java:430)
	at org.apache.bookkeeper.meta.CleanupLedgerManager.readLedgerMetadata(CleanupLedgerManager.java:157)
	at org.apache.bookkeeper.client.LedgerOpenOp.initiate(LedgerOpenOp.java:114)
	at org.apache.bookkeeper.client.LedgerOpenOp$OpenBuilderImpl.open(LedgerOpenOp.java:269)
	at org.apache.bookkeeper.client.LedgerOpenOp$OpenBuilderImpl.execute(LedgerOpenOp.java:247)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.lambda$getLedgerHandle$20(ManagedLedgerImpl.java:1773)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl$$Lambda$1195/876838597.apply(Unknown Source)
	at org.apache.pulsar.common.util.collections.ConcurrentLongHashMap$Section.put(ConcurrentLongHashMap.java:287)
	at org.apache.pulsar.common.util.collections.ConcurrentLongHashMap.computeIfAbsent(ConcurrentLongHashMap.java:135)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.getLedgerHandle(ManagedLedgerImpl.java:1742)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncReadEntry(ManagedLedgerImpl.java:1839)
	at org.apache.bookkeeper.mledger.impl.OpFindNewest.find(OpFindNewest.java:147)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncFindPosition(ManagedLedgerImpl.java:1670)
	at org.streamnative.pulsar.handlers.rocketmq.utils.MessageIdUtils.getPositionForOffset(MessageIdUtils.java:183)
	at org.streamnative.pulsar.handlers.rocketmq.inner.RopServerCnx.getMessage(RopServerCnx.java:692)
	at org.streamnative.pulsar.handlers.rocketmq.inner.processor.PullMessageProcessor.processRequest(PullMessageProcessor.java:359)
	at org.streamnative.pulsar.handlers.rocketmq.inner.processor.PullMessageProcessor.processRequest(PullMessageProcessor.java:170)
	at org.streamnative.pulsar.handlers.rocketmq.inner.proxy.RopBrokerProxy$PullMessageProcessorProxy.processRequest(RopBrokerProxy.java:1098)
	at org.streamnative.pulsar.handlers.rocketmq.inner.NettyRemotingAbstract$1.run(NettyRemotingAbstract.java:202)
	at org.apache.rocketmq.remoting.netty.RequestTask.run(RequestTask.java:80)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)

Found 1 deadlock.

@wolfstudy
Copy link
Member

cc @baomingyu @315157973 PTAL

Technoboy- pushed a commit to Technoboy-/pulsar that referenced this pull request Mar 16, 2022
Fixes apache#12221

*(or if this PR is one task of a github issue, please add `Master Issue: #<xyz>` to link to the master issue.)*

Master Issue: apache#12221 

### Motivation
There  are three way to create new ledger when one ledger is full.
first: after add last entry for full ledger, it will handle create ledger processing.
second: check ledger is or not full, when ledger is full and appendingAddEntries is not empty, it will handle create ledger processing.
third: check ledger is or not full, when ledger is full, it will handle create ledger processing.

 So we need check it is or not creating or has created when concurrent create new ledger. avoid to create three new ledgers one  time.
@Technoboy- Technoboy- added the cherry-picked/branch-2.7 Archived: 2.7 is end of life label Mar 16, 2022
bharanic-dev pushed a commit to bharanic-dev/pulsar that referenced this pull request Mar 18, 2022
Fixes apache#12221

*(or if this PR is one task of a github issue, please add `Master Issue: #<xyz>` to link to the master issue.)*

Master Issue: apache#12221 

### Motivation
There  are three way to create new ledger when one ledger is full.
first: after add last entry for full ledger, it will handle create ledger processing.
second: check ledger is or not full, when ledger is full and appendingAddEntries is not empty, it will handle create ledger processing.
third: check ledger is or not full, when ledger is full, it will handle create ledger processing.

 So we need check it is or not creating or has created when concurrent create new ledger. avoid to create three new ledgers one  time.
aloyszhang pushed a commit to aloyszhang/pulsar that referenced this pull request Aug 5, 2022
Squash merge branch 'inlong-2.8.1.1-fix-lost-message' into 'inlong-2.8.1.1'
cherry pick : Fix lost message issues 12221 (apache#12223)
Shawyeok pushed a commit to Shawyeok/pulsar that referenced this pull request Sep 6, 2022
Fixes apache#12221

*(or if this PR is one task of a github issue, please add `Master Issue: #<xyz>` to link to the master issue.)*

Master Issue: apache#12221

### Motivation
There  are three way to create new ledger when one ledger is full.
first: after add last entry for full ledger, it will handle create ledger processing.
second: check ledger is or not full, when ledger is full and appendingAddEntries is not empty, it will handle create ledger processing.
third: check ledger is or not full, when ledger is full, it will handle create ledger processing.

 So we need check it is or not creating or has created when concurrent create new ledger. avoid to create three new ledgers one  time.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
cherry-picked/branch-2.7 Archived: 2.7 is end of life cherry-picked/branch-2.8 Archived: 2.8 is end of life release/2.8.2 release/2.9.0 type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging this pull request may close these issues.

SomeTimes messages may be lost, those message has be add to ledger but were invisible for all consumers
8 participants