Skip to content

Conversation

@horizonzy
Copy link
Member

@horizonzy horizonzy commented Mar 23, 2023

The fork pr: horizonzy#12

Motivation

After #13833, we change the signature of LedgerOffloaderFactory#create.

Before: no offloaderStats.

    default T create(OffloadPoliciesImpl offloadPolicies,
                     Map<String, String> userMetadata,
                     SchemaStorage schemaStorage,
                     OrderedScheduler scheduler)
            throws IOException {
        return create(offloadPolicies, userMetadata, scheduler);
    }

After: add offloaderStats.

    default T create(OffloadPoliciesImpl offloadPolicies,
                     Map<String, String> userMetadata,
                     SchemaStorage schemaStorage,
                     OrderedScheduler scheduler,
                     LedgerOffloaderStats offloaderStats)
            throws IOException {
        return create(offloadPolicies, userMetadata, scheduler, offloaderStats);
    }

And in PulsarService#createManagedLedgerOffloader line_1402, it changes the way to create an offloader.

Before: no offloaderStats

                    return offloaderFactory.create(
                        offloadPolicies,
                        ImmutableMap.of(
                            LedgerOffloader.METADATA_SOFTWARE_VERSION_KEY.toLowerCase(), PulsarVersion.getVersion(),
                            LedgerOffloader.METADATA_SOFTWARE_GITSHA_KEY.toLowerCase(), PulsarVersion.getGitSha(),
                            LedgerOffloader.METADATA_PULSAR_CLUSTER_NAME.toLowerCase(), config.getClusterName()
                        ),
                        schemaStorage,
                        getOffloaderScheduler(offloadPolicies));

After: add offloaderStats

return offloaderFactory.create(
                        offloadPolicies,
                        ImmutableMap.of(
                            LedgerOffloader.METADATA_SOFTWARE_VERSION_KEY.toLowerCase(), PulsarVersion.getVersion(),
                            LedgerOffloader.METADATA_SOFTWARE_GITSHA_KEY.toLowerCase(), PulsarVersion.getGitSha(),
                            LedgerOffloader.METADATA_PULSAR_CLUSTER_NAME.toLowerCase(), config.getClusterName()
                        ),
                        schemaStorage, getOffloaderScheduler(offloadPolicies), this.offloaderStats);

But some users may use the old version LedgerOffloaderFactory in the offloader nar, when pulsar load the offloader nar, it will use the LedgerOffloaderFactory in the offloader nar, the LedgerOffloaderFactory#create has no param offloaderStats. Then it will throw an exception as follows:

2023-03-23T23:59:42,947 - ERROR - [main:PulsarService@918] - Failed to start Pulsar service: java.lang.AbstractMethodError: Receiver class io.streamnative.tieredstorage.pulsar.PulsarOffloaderFactory does not define or inherit an implementation of the resolved method 'abstract org.apache.bookkeeper.mledger.LedgerOffloader create(org.apache.pulsar.common.policies.data.OffloadPoliciesImpl, java.util.Map, org.apache.bookkeeper.common.util.OrderedScheduler, org.apache.bookkeeper.mledger.LedgerOffloaderStats)' of interface org.apache.bookkeeper.mledger.LedgerOffloaderFactory.
org.apache.pulsar.broker.PulsarServerException: java.lang.AbstractMethodError: Receiver class io.streamnative.tieredstorage.pulsar.PulsarOffloaderFactory does not define or inherit an implementation of the resolved method 'abstract org.apache.bookkeeper.mledger.LedgerOffloader create(org.apache.pulsar.common.policies.data.OffloadPoliciesImpl, java.util.Map, org.apache.bookkeeper.common.util.OrderedScheduler, org.apache.bookkeeper.mledger.LedgerOffloaderStats)' of interface org.apache.bookkeeper.mledger.LedgerOffloaderFactory.
	at org.apache.pulsar.broker.PulsarService.createManagedLedgerOffloader(PulsarService.java:1422) ~[classes/:?]
	at org.apache.pulsar.broker.PulsarService.start(PulsarService.java:780) ~[classes/:?]
	at org.apache.pulsar.PulsarBrokerStarter$BrokerStarter.start(PulsarBrokerStarter.java:276) ~[classes/:?]
	at org.apache.pulsar.PulsarBrokerStarter.main(PulsarBrokerStarter.java:356) ~[classes/:?]

So we should make LedgerOffloaderFactory#create compatible with the previous version, and revert the origin signature to LedgerOffloaderFactory for compatibility.

After this PR, the user upgrades the managed-ledger dependency(LedgerOffloaderFactory is defined in managed-ledger). So the user offloader nar LedgerOffloaderFactory has both methods. One is with offloaderStats, another one is without offloaderStats.

If the pulsar is the old version, it loads the offloader nar, and using LedgerOffloaderFactory#create without offloadStats to create offloader. It works.

If the pulsar is the new version, it loads the offloader nar, and using LedgerOffloaderFactory#create with offloadStats to create the offloader. It also works.

Modifications

Verifying this change

  • Make sure that the change passes the CI checks.

(Please pick either of the following options)

This change is a trivial rework / code cleanup without any test coverage.

(or)

This change is already covered by existing tests, such as (please describe tests).

(or)

This change added tests and can be verified as follows:

(example:)

  • Added integration tests for end-to-end deployment with large payloads (10MB)
  • Extended integration test for recovery after broker failure

Does this pull request potentially affect one of the following parts:

If the box was checked, please highlight the changes

  • Dependencies (add or upgrade a dependency)
  • The public API
  • The schema
  • The default values of configurations
  • The threading model
  • The binary protocol
  • The REST endpoints
  • The admin CLI options
  • The metrics
  • Anything that affects deployment

Documentation

  • doc
  • doc-required
  • doc-not-needed
  • doc-complete

Matching PR in forked repository

PR in forked repository:

@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Mar 23, 2023
@horizonzy horizonzy changed the title Make LedgerOffloaderFactory compatible with old version. [fix][broker]Make LedgerOffloaderFactory compatible with old version. Mar 24, 2023
@horizonzy horizonzy closed this Mar 24, 2023
@horizonzy horizonzy reopened this Mar 24, 2023
@Technoboy- Technoboy- added this to the 3.0.0 milestone Mar 30, 2023
@Technoboy- Technoboy- closed this Mar 30, 2023
@Technoboy- Technoboy- reopened this Mar 30, 2023
Map<String, String> userMetadata,
OrderedScheduler scheduler)
throws IOException {
return create(offloadPolicies, userMetadata, scheduler, null);
Copy link
Contributor

@poorbarcode poorbarcode Mar 30, 2023

Choose a reason for hiding this comment

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

After this change, Is it possible to throw a NullPointerEx using the variable offloaderStats?

long cost = System.nanoTime() - startReadTime;
this.offloaderStats.recordReadLedgerLatency(topicName, cost, TimeUnit.NANOSECONDS);
semaphore.acquire();

Such as line-227 above or other places.

Copy link
Member Author

Choose a reason for hiding this comment

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

Thanks for your reminder. Here we shouldn't make the method default. This method won't invoke in pulsar directly. In the pulsar master branch, it only invokes the method with the param offloaderStats. This method is compatible with the nar package, let the user implements by themselves.

@horizonzy
Copy link
Member Author

@poorbarcode Could you help to review it again, thanks!

Map<String, String> userMetadata,
OrderedScheduler scheduler,
LedgerOffloaderStats offloaderStats)
throws IOException;
Copy link
Contributor

Choose a reason for hiding this comment

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

Since this method T create(offloadPolicies, userMetadata, scheduler, offloaderStats), the changes in #13833 can not guarantee forward compatibility, right?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yes.

Copy link
Contributor

Choose a reason for hiding this comment

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

There are four scenarios:

  • old nar & old broker: I think it will work.
  • old nar & new broker: It will not work, and we do not guarantee forward compatibility. So it is okay.
  • new nar & old broker: The broker will call the method without @param offloaderStats. I think it will work.
    • we should make the method without @param offloaderStats non-default.
  • new nar & new broker I think it will work.

All things are good.

Map<String, String> userMetadata,
OrderedScheduler scheduler,
LedgerOffloaderStats offloaderStats)
throws IOException;
Copy link
Contributor

Choose a reason for hiding this comment

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

There are four scenarios:

  • old nar & old broker: I think it will work.
  • old nar & new broker: It will not work, and we do not guarantee forward compatibility. So it is okay.
  • new nar & old broker: The broker will call the method without @param offloaderStats. I think it will work.
    • we should make the method without @param offloaderStats non-default.
  • new nar & new broker I think it will work.

All things are good.

@poorbarcode poorbarcode changed the title [fix][broker]Make LedgerOffloaderFactory compatible with old version. [fix][broker]Make LedgerOffloaderFactory compatible with old nar. Apr 6, 2023
@poorbarcode poorbarcode changed the title [fix][broker]Make LedgerOffloaderFactory compatible with old nar. [fix][broker]Make LedgerOffloaderFactory can load the old nar. Apr 6, 2023
@poorbarcode poorbarcode merged commit 02c838c into apache:master Apr 10, 2023
@hangc0276
Copy link
Contributor

@codelipenghui @Technoboy- @poorbarcode I think this PR needs to be cherry-picked to branch-2.11 because it's for backporting the old interface.

Technoboy- pushed a commit that referenced this pull request May 9, 2023
After #13833, we change the signature of LedgerOffloaderFactory#create. But some users may use the old version LedgerOffloaderFactory in the offloader nar, when pulsar load the offloader nar, it will use the LedgerOffloaderFactory in the offloader nar, the LedgerOffloaderFactory#create has no param offloaderStats.

Modifications: Make LedgerOffloaderFactory can load the old nar.
@Technoboy-
Copy link
Contributor

@codelipenghui @Technoboy- @poorbarcode I think this PR needs to be cherry-picked to branch-2.11 because it's for backporting the old interface.

Done

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.

4 participants