diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index aa33d7ae197d1..1642b54337fc0 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -1498,12 +1498,16 @@ jobs: with: action: wait - # This job is required for pulls to be merged. + # This job is required for pulls to be merged. This job is referenced by name in .asf.yaml file in the + # protected_branches section for master branch required_status_checks. # It depends on all other jobs in this workflow. - # It cleans up the binaries in the same job in order to not spin up another runner for basically doing nothing. + # This job also cleans up the binaries at the end of the workflow. pulsar-ci-checks-completed: name: "Pulsar CI checks completed" - if: ${{ always() && needs.preconditions.result == 'success' }} + # run always, but skip for other repositories than apache/pulsar when a scheduled workflow is cancelled + # this is to allow the workflow scheduled jobs to show as cancelled instead of failed since scheduled + # jobs are not enabled for other than apache/pulsar repository. + if: ${{ always() && !(cancelled() && github.repository != 'apache/pulsar' && github.event_name == 'schedule') }} runs-on: ubuntu-22.04 timeout-minutes: 10 needs: [ @@ -1521,10 +1525,11 @@ jobs: ] steps: - name: Check that all required jobs were completed successfully - if: ${{ needs.preconditions.outputs.docs_only != 'true' }} + if: ${{ needs.preconditions.result != 'success' || needs.preconditions.outputs.docs_only != 'true' }} run: | if [[ ! ( \ - "${{ needs.unit-tests.result }}" == "success" \ + "${{ needs.preconditions.result }}" == "success" \ + && "${{ needs.unit-tests.result }}" == "success" \ && "${{ needs.integration-tests.result }}" == "success" \ && "${{ needs.system-tests.result }}" == "success" \ && "${{ needs.macos-build.result }}" == "success" \ diff --git a/buildtools/pom.xml b/buildtools/pom.xml index cd4d02af3d7b4..58f99e9ea86b5 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -41,7 +41,7 @@ 1.8 3.1.0 2.23.1 - 1.7.32 + 2.0.13 7.7.1 3.11 4.1 @@ -100,6 +100,12 @@ org.testng testng ${testng.version} + + + org.slf4j + * + + org.apache.logging.log4j @@ -111,7 +117,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl org.slf4j diff --git a/conf/broker.conf b/conf/broker.conf index fd6bba0f45d2c..d97e3a5ef89ad 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -88,6 +88,16 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Number of threads to config Netty Acceptor. Default is 1 numAcceptorThreads= @@ -1660,10 +1670,10 @@ s3ManagedLedgerOffloadBucket= # For Amazon S3 ledger offload, Alternative endpoint to connect to (useful for testing) s3ManagedLedgerOffloadServiceEndpoint= -# For Amazon S3 ledger offload, Max block size in bytes. (64MB by default, 5MB minimum) +# For Amazon S3 ledger offload, Max block size in bytes. (64MiB by default, 5MiB minimum) s3ManagedLedgerOffloadMaxBlockSizeInBytes=67108864 -# For Amazon S3 ledger offload, Read buffer size in bytes (1MB by default) +# For Amazon S3 ledger offload, Read buffer size in bytes (1MiB by default) s3ManagedLedgerOffloadReadBufferSizeInBytes=1048576 # For Google Cloud Storage ledger offload, region where offload bucket is located. @@ -1673,10 +1683,11 @@ gcsManagedLedgerOffloadRegion= # For Google Cloud Storage ledger offload, Bucket to place offloaded ledger into gcsManagedLedgerOffloadBucket= -# For Google Cloud Storage ledger offload, Max block size in bytes. (64MB by default, 5MB minimum) -gcsManagedLedgerOffloadMaxBlockSizeInBytes=67108864 +# For Google Cloud Storage ledger offload, Max block size in bytes. (128MiB by default, 5MiB minimum) +# Since JClouds limits the maximum number of blocks to 32, the maximum size of a ledger is 32 times the block size. +gcsManagedLedgerOffloadMaxBlockSizeInBytes=134217728 -# For Google Cloud Storage ledger offload, Read buffer size in bytes (1MB by default) +# For Google Cloud Storage ledger offload, Read buffer size in bytes (1MiB by default) gcsManagedLedgerOffloadReadBufferSizeInBytes=1048576 # For Google Cloud Storage, path to json file containing service account credentials. diff --git a/conf/functions_worker.yml b/conf/functions_worker.yml index 3871c74a88778..6f995576ebd64 100644 --- a/conf/functions_worker.yml +++ b/conf/functions_worker.yml @@ -27,6 +27,16 @@ workerHostname: localhost workerPort: 6750 workerPortTls: 6751 +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled: false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor: false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses: null + # The Configuration metadata store url # Examples: # * zk:my-zk-1:2181,my-zk-2:2181,my-zk-3:2181 diff --git a/conf/proxy.conf b/conf/proxy.conf index 5a9d433f39ceb..6e6c960e8009e 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -63,6 +63,16 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Enables zero-copy transport of data across network interfaces using the splice system call. # Zero copy mode cannot be used when TLS is enabled or when proxyLogLevel is > 0. proxyZeroCopyModeEnabled=true diff --git a/conf/standalone.conf b/conf/standalone.conf index 5c94d63817a12..b04e5ccefa640 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -51,6 +51,16 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Number of threads to use for Netty IO. Default is set to 2 * Runtime.getRuntime().availableProcessors() numIOThreads= diff --git a/conf/websocket.conf b/conf/websocket.conf index 490cff2722ee5..9051f3b590c8e 100644 --- a/conf/websocket.conf +++ b/conf/websocket.conf @@ -46,6 +46,16 @@ statusFilePath= # Hostname or IP address the service binds on, default is 0.0.0.0. bindAddress=0.0.0.0 +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Name of the pulsar cluster to connect to clusterName= diff --git a/distribution/licenses/LICENSE-Reactive-gRPC.txt b/distribution/licenses/LICENSE-Reactive-gRPC.txt new file mode 100644 index 0000000000000..bc589401e7bdf --- /dev/null +++ b/distribution/licenses/LICENSE-Reactive-gRPC.txt @@ -0,0 +1,29 @@ +BSD 3-Clause License + +Copyright (c) 2019, Salesforce.com, Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +* Neither the name of the copyright holder nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 599a9755f9155..1c9ea68685308 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -180,7 +180,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 4dc6e4341672c..c5c243796b6f3 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -262,7 +262,7 @@ The Apache Software License, Version 2.0 - com.fasterxml.jackson.module-jackson-module-parameter-names-2.14.2.jar * Caffeine -- com.github.ben-manes.caffeine-caffeine-2.9.1.jar * Conscrypt -- org.conscrypt-conscrypt-openjdk-uber-2.5.2.jar - * Proto Google Common Protos -- com.google.api.grpc-proto-google-common-protos-2.9.0.jar + * Proto Google Common Protos -- com.google.api.grpc-proto-google-common-protos-2.17.0.jar * Bitbucket -- org.bitbucket.b_c-jose4j-0.9.4.jar * Gson - com.google.code.gson-gson-2.8.9.jar @@ -350,40 +350,40 @@ The Apache Software License, Version 2.0 * Log4J - org.apache.logging.log4j-log4j-api-2.23.1.jar - org.apache.logging.log4j-log4j-core-2.23.1.jar - - org.apache.logging.log4j-log4j-slf4j-impl-2.23.1.jar + - org.apache.logging.log4j-log4j-slf4j2-impl-2.23.1.jar - org.apache.logging.log4j-log4j-web-2.23.1.jar * Java Native Access JNA - net.java.dev.jna-jna-jpms-5.12.1.jar - net.java.dev.jna-jna-platform-jpms-5.12.1.jar * BookKeeper - - org.apache.bookkeeper-bookkeeper-common-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-proto-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-server-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.5.jar - - org.apache.bookkeeper-circe-checksum-4.16.5.jar - - org.apache.bookkeeper-cpu-affinity-4.16.5.jar - - org.apache.bookkeeper-statelib-4.16.5.jar - - org.apache.bookkeeper-stream-storage-api-4.16.5.jar - - org.apache.bookkeeper-stream-storage-common-4.16.5.jar - - org.apache.bookkeeper-stream-storage-java-client-4.16.5.jar - - org.apache.bookkeeper-stream-storage-java-client-base-4.16.5.jar - - org.apache.bookkeeper-stream-storage-proto-4.16.5.jar - - org.apache.bookkeeper-stream-storage-server-4.16.5.jar - - org.apache.bookkeeper-stream-storage-service-api-4.16.5.jar - - org.apache.bookkeeper-stream-storage-service-impl-4.16.5.jar - - org.apache.bookkeeper.http-http-server-4.16.5.jar - - org.apache.bookkeeper.http-vertx-http-server-4.16.5.jar - - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.5.jar - - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.5.jar - - org.apache.distributedlog-distributedlog-common-4.16.5.jar - - org.apache.distributedlog-distributedlog-core-4.16.5-tests.jar - - org.apache.distributedlog-distributedlog-core-4.16.5.jar - - org.apache.distributedlog-distributedlog-protocol-4.16.5.jar - - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.5.jar - - org.apache.bookkeeper-native-io-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-common-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-common-allocator-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-proto-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-server-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-tools-framework-4.17.0.jar + - org.apache.bookkeeper-circe-checksum-4.17.0.jar + - org.apache.bookkeeper-cpu-affinity-4.17.0.jar + - org.apache.bookkeeper-statelib-4.17.0.jar + - org.apache.bookkeeper-stream-storage-api-4.17.0.jar + - org.apache.bookkeeper-stream-storage-common-4.17.0.jar + - org.apache.bookkeeper-stream-storage-java-client-4.17.0.jar + - org.apache.bookkeeper-stream-storage-java-client-base-4.17.0.jar + - org.apache.bookkeeper-stream-storage-proto-4.17.0.jar + - org.apache.bookkeeper-stream-storage-server-4.17.0.jar + - org.apache.bookkeeper-stream-storage-service-api-4.17.0.jar + - org.apache.bookkeeper-stream-storage-service-impl-4.17.0.jar + - org.apache.bookkeeper.http-http-server-4.17.0.jar + - org.apache.bookkeeper.http-vertx-http-server-4.17.0.jar + - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.17.0.jar + - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.17.0.jar + - org.apache.distributedlog-distributedlog-common-4.17.0.jar + - org.apache.distributedlog-distributedlog-core-4.17.0-tests.jar + - org.apache.distributedlog-distributedlog-core-4.17.0.jar + - org.apache.distributedlog-distributedlog-protocol-4.17.0.jar + - org.apache.bookkeeper.stats-codahale-metrics-provider-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-slogger-api-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.17.0.jar + - org.apache.bookkeeper-native-io-4.17.0.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar @@ -430,23 +430,23 @@ The Apache Software License, Version 2.0 - org.jetbrains.kotlin-kotlin-stdlib-jdk8-1.8.20.jar - org.jetbrains-annotations-13.0.jar * gRPC - - io.grpc-grpc-all-1.55.3.jar - - io.grpc-grpc-auth-1.55.3.jar - - io.grpc-grpc-context-1.55.3.jar - - io.grpc-grpc-core-1.55.3.jar - - io.grpc-grpc-netty-1.55.3.jar - - io.grpc-grpc-protobuf-1.55.3.jar - - io.grpc-grpc-protobuf-lite-1.55.3.jar - - io.grpc-grpc-stub-1.55.3.jar - - io.grpc-grpc-alts-1.55.3.jar - - io.grpc-grpc-api-1.55.3.jar - - io.grpc-grpc-grpclb-1.55.3.jar - - io.grpc-grpc-netty-shaded-1.55.3.jar - - io.grpc-grpc-services-1.55.3.jar - - io.grpc-grpc-xds-1.55.3.jar - - io.grpc-grpc-rls-1.55.3.jar - - io.grpc-grpc-servlet-1.55.3.jar - - io.grpc-grpc-servlet-jakarta-1.55.3.jar + - io.grpc-grpc-all-1.56.0.jar + - io.grpc-grpc-auth-1.56.0.jar + - io.grpc-grpc-context-1.56.0.jar + - io.grpc-grpc-core-1.56.0.jar + - io.grpc-grpc-netty-1.56.0.jar + - io.grpc-grpc-protobuf-1.56.0.jar + - io.grpc-grpc-protobuf-lite-1.56.0.jar + - io.grpc-grpc-stub-1.56.0.jar + - io.grpc-grpc-alts-1.56.0.jar + - io.grpc-grpc-api-1.56.0.jar + - io.grpc-grpc-grpclb-1.56.0.jar + - io.grpc-grpc-netty-shaded-1.56.0.jar + - io.grpc-grpc-services-1.56.0.jar + - io.grpc-grpc-xds-1.56.0.jar + - io.grpc-grpc-rls-1.56.0.jar + - io.grpc-grpc-servlet-1.56.0.jar + - io.grpc-grpc-servlet-jakarta-1.56.0.jar * Perfmark - io.perfmark-perfmark-api-0.26.0.jar * OpenCensus @@ -481,7 +481,12 @@ The Apache Software License, Version 2.0 * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia - - io.streamnative.oxia-oxia-client-0.1.0-shaded.jar + - io.streamnative.oxia-oxia-client-0.1.6.jar + - io.streamnative.oxia-oxia-client-metrics-api-0.1.6.jar + * OpenHFT + - net.openhft-zero-allocation-hashing-0.16.jar + * Project reactor + - io.projectreactor-reactor-core-3.5.2.jar * Java JSON WebTokens - io.jsonwebtoken-jjwt-api-0.11.1.jar - io.jsonwebtoken-jjwt-impl-0.11.1.jar @@ -504,8 +509,8 @@ The Apache Software License, Version 2.0 * Google HTTP Client - com.google.http-client-google-http-client-gson-1.41.0.jar - com.google.http-client-google-http-client-1.41.0.jar - - com.google.auto.value-auto-value-annotations-1.9.jar - - com.google.re2j-re2j-1.6.jar + - com.google.auto.value-auto-value-annotations-1.10.1.jar + - com.google.re2j-re2j-1.7.jar * Jetcd - io.etcd-jetcd-api-0.7.5.jar - io.etcd-jetcd-common-0.7.5.jar @@ -548,6 +553,9 @@ BSD 3-clause "New" or "Revised" License * JSR305 -- com.google.code.findbugs-jsr305-3.0.2.jar -- ../licenses/LICENSE-JSR305.txt * JLine -- jline-jline-2.14.6.jar -- ../licenses/LICENSE-JLine.txt * JLine3 -- org.jline-jline-3.21.0.jar -- ../licenses/LICENSE-JLine.txt + * Reactive gRPC + - com.salesforce.servicelibs-reactive-grpc-common-1.2.4.jar -- ../licenses/LICENSE-Reactive-gRPC.txt + - com.salesforce.servicelibs-reactor-grpc-stub-1.2.4.jar -- ../licenses/LICENSE-Reactive-gRPC.txt BSD 2-Clause License * HdrHistogram -- org.hdrhistogram-HdrHistogram-2.1.9.jar -- ../licenses/LICENSE-HdrHistogram.txt @@ -555,8 +563,8 @@ BSD 2-Clause License MIT License * Java SemVer -- com.github.zafarkhaja-java-semver-0.9.0.jar -- ../licenses/LICENSE-SemVer.txt * SLF4J -- ../licenses/LICENSE-SLF4J.txt - - org.slf4j-slf4j-api-1.7.32.jar - - org.slf4j-jcl-over-slf4j-1.7.32.jar + - org.slf4j-slf4j-api-2.0.13.jar + - org.slf4j-jcl-over-slf4j-2.0.13.jar * The Checker Framework - org.checkerframework-checker-qual-3.33.0.jar * oshi @@ -566,8 +574,8 @@ MIT License - com.auth0-jwks-rsa-0.22.0.jar Protocol Buffers License * Protocol Buffers - - com.google.protobuf-protobuf-java-3.19.6.jar -- ../licenses/LICENSE-protobuf.txt - - com.google.protobuf-protobuf-java-util-3.19.6.jar -- ../licenses/LICENSE-protobuf.txt + - com.google.protobuf-protobuf-java-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt + - com.google.protobuf-protobuf-java-util-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 5f4fc549ccc62..144f7b1ff6d83 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -51,7 +51,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 069e61b89b55a..41b38f17dce77 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -333,6 +333,7 @@ The Apache Software License, Version 2.0 - listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar * J2ObjC Annotations -- j2objc-annotations-1.3.jar * Netty Reactive Streams -- netty-reactive-streams-2.0.6.jar + * Swagger -- swagger-annotations-1.6.2.jar * DataSketches - memory-0.8.3.jar - sketches-core-0.8.3.jar @@ -385,7 +386,7 @@ The Apache Software License, Version 2.0 * Log4J - log4j-api-2.23.1.jar - log4j-core-2.23.1.jar - - log4j-slf4j-impl-2.23.1.jar + - log4j-slf4j2-impl-2.23.1.jar - log4j-web-2.23.1.jar * OpenTelemetry - opentelemetry-api-1.34.1.jar @@ -393,9 +394,9 @@ The Apache Software License, Version 2.0 - opentelemetry-extension-incubator-1.34.1-alpha.jar * BookKeeper - - bookkeeper-common-allocator-4.16.5.jar - - cpu-affinity-4.16.5.jar - - circe-checksum-4.16.5.jar + - bookkeeper-common-allocator-4.17.0.jar + - cpu-affinity-4.17.0.jar + - circe-checksum-4.17.0.jar * AirCompressor - aircompressor-0.20.jar * AsyncHttpClient @@ -423,13 +424,13 @@ BSD 3-clause "New" or "Revised" License MIT License * SLF4J -- ../licenses/LICENSE-SLF4J.txt - - slf4j-api-1.7.32.jar + - slf4j-api-2.0.13.jar * The Checker Framework - checker-qual-3.33.0.jar Protocol Buffers License * Protocol Buffers - - protobuf-java-3.19.6.jar -- ../licenses/LICENSE-protobuf.txt + - protobuf-java-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 08c3509d58a8e..d0cc9f5e576bd 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -3817,7 +3817,7 @@ public PositionImpl getValidPositionAfterSkippedEntries(final PositionImpl posit Long nextLedgerId = ledgers.ceilingKey(skippedPosition.getLedgerId() + 1); // This means it has jumped to the last position if (nextLedgerId == null) { - if (currentLedgerEntries == 0) { + if (currentLedgerEntries == 0 && currentLedger != null) { return PositionImpl.get(currentLedger.getId(), 0); } return lastConfirmedEntry.getNext(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 1932b1d90ae5b..f16e71bf264b8 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -4697,5 +4697,66 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { && cursorReadPosition.getEntryId() == expectReadPosition.getEntryId()); } + @Test + public void testRecoverCursorWithTerminateManagedLedger() throws Exception { + String mlName = "my_test_ledger"; + String cursorName = "c1"; + + ManagedLedgerConfig config = new ManagedLedgerConfig(); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(mlName, config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(cursorName); + + // Write some data. + Position p0 = ledger.addEntry("entry-0".getBytes()); + Position p1 = ledger.addEntry("entry-1".getBytes()); + + // Read message. + List entries = c1.readEntries(2); + assertEquals(entries.size(), 2); + assertEquals(entries.get(0).getPosition(), p0); + assertEquals(entries.get(1).getPosition(), p1); + entries.forEach(Entry::release); + + // Mark delete the last message. + c1.markDelete(p1); + Position markDeletedPosition = c1.getMarkDeletedPosition(); + Assert.assertEquals(markDeletedPosition, p1); + + // Terminate the managed ledger. + Position lastPosition = ledger.terminate(); + assertEquals(lastPosition, p1); + + // Close the ledger. + ledger.close(); + + // Reopen the ledger. + ledger = (ManagedLedgerImpl) factory.open(mlName, config); + BookKeeper mockBookKeeper = mock(BookKeeper.class); + final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, new ManagedLedgerConfig(), ledger, + cursorName); + + CompletableFuture recoverFuture = new CompletableFuture<>(); + // Recover the cursor. + cursor.recover(new VoidCallback() { + @Override + public void operationComplete() { + recoverFuture.complete(null); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + recoverFuture.completeExceptionally(exception); + } + }); + + recoverFuture.join(); + assertTrue(recoverFuture.isDone()); + assertFalse(recoverFuture.isCompletedExceptionally()); + + // Verify the cursor state. + assertEquals(cursor.getMarkDeletedPosition(), markDeletedPosition); + assertEquals(cursor.getReadPosition(), markDeletedPosition.getNext()); + } + private static final Logger log = LoggerFactory.getLogger(ManagedCursorTest.class); } diff --git a/pom.xml b/pom.xml index c7fba94abd8ea..585347fb1f855 100644 --- a/pom.xml +++ b/pom.xml @@ -137,7 +137,7 @@ flexible messaging model and an intuitive client API. 1.26.0 - 4.16.5 + 4.17.0 3.9.2 1.5.0 1.10.0 @@ -153,7 +153,7 @@ flexible messaging model and an intuitive client API. 0.16.0 4.3.8 7.9.2 - 1.7.32 + 2.0.13 4.4 2.23.1 1.78 @@ -168,9 +168,9 @@ flexible messaging model and an intuitive client API. 0.5.0 1.14.12 1.17 - 3.19.6 + 3.22.3 ${protobuf3.version} - 1.55.3 + 1.56.0 1.41.0 0.26.0 ${grpc.version} @@ -248,7 +248,7 @@ flexible messaging model and an intuitive client API. 4.5.13 4.4.15 0.7.5 - 0.1.0 + 0.1.6 2.0 1.10.12 5.3.3 @@ -278,6 +278,7 @@ flexible messaging model and an intuitive client API. 1.5.4 5.4.0 2.33.2 + 1.0.3 0.6.1 @@ -351,6 +352,10 @@ flexible messaging model and an intuitive client API. org.yaml * + + org.slf4j + * + @@ -459,6 +464,10 @@ flexible messaging model and an intuitive client API. org.bouncycastle * + + log4j-slf4j-impl + org.apache.logging.log4j + slf4j-log4j12 org.slf4j @@ -1192,7 +1201,6 @@ flexible messaging model and an intuitive client API. io.streamnative.oxia oxia-client ${oxia.version} - shaded io.streamnative.oxia @@ -1582,6 +1590,10 @@ flexible messaging model and an intuitive client API. org.apache.zookeeper * + + log4j-slf4j-impl + org.apache.logging.log4j + @@ -2701,5 +2713,13 @@ flexible messaging model and an intuitive client API. false + + + oracle.releases + https://download.oracle.com/maven + + false + + diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 2b58cbc2d1178..156c83bd6960c 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -250,6 +250,22 @@ public class ServiceConfiguration implements PulsarConfiguration { + " when getting topic statistics data.") private boolean haProxyProtocolEnabled; + @FieldContext(category = CATEGORY_SERVER, + doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext( category = CATEGORY_SERVER, doc = "Number of threads to use for Netty Acceptor." diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java index 1ba353dccaa1c..975b23192f949 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java @@ -24,6 +24,8 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.function.Supplier; @@ -54,10 +56,14 @@ public class NamespaceResources extends BaseResources { private static final String NAMESPACE_BASE_PATH = "/namespace"; public NamespaceResources(MetadataStore configurationStore, int operationTimeoutSec) { + this(configurationStore, operationTimeoutSec, ForkJoinPool.commonPool()); + } + + public NamespaceResources(MetadataStore configurationStore, int operationTimeoutSec, Executor executor) { super(configurationStore, Policies.class, operationTimeoutSec); this.configurationStore = configurationStore; isolationPolicies = new IsolationPolicyResources(configurationStore, operationTimeoutSec); - partitionedTopicResources = new PartitionedTopicResources(configurationStore, operationTimeoutSec); + partitionedTopicResources = new PartitionedTopicResources(configurationStore, operationTimeoutSec, executor); } public CompletableFuture> listNamespacesAsync(String tenant) { @@ -234,9 +240,11 @@ public void setIsolationDataWithCreate(String cluster, public static class PartitionedTopicResources extends BaseResources { private static final String PARTITIONED_TOPIC_PATH = "/admin/partitioned-topics"; + private final Executor executor; - public PartitionedTopicResources(MetadataStore configurationStore, int operationTimeoutSec) { + public PartitionedTopicResources(MetadataStore configurationStore, int operationTimeoutSec, Executor executor) { super(configurationStore, PartitionedTopicMetadata.class, operationTimeoutSec); + this.executor = executor; } public CompletableFuture updatePartitionedTopicAsync(TopicName tn, Function runWithMarkDeleteAsync(TopicName topic, future.complete(deleteResult); } }); - }); + }, executor); return future; } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java index fe7ffe0bc7b43..cc64eeb52f6eb 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.resources; import java.util.Optional; +import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; import lombok.Getter; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreConfig; @@ -57,13 +59,19 @@ public class PulsarResources { public PulsarResources(MetadataStore localMetadataStore, MetadataStore configurationMetadataStore) { this(localMetadataStore, configurationMetadataStore, DEFAULT_OPERATION_TIMEOUT_SEC); } + + public PulsarResources(MetadataStore localMetadataStore, MetadataStore configurationMetadataStore, + int operationTimeoutSec) { + this(localMetadataStore, configurationMetadataStore, operationTimeoutSec, ForkJoinPool.commonPool()); + } + public PulsarResources(MetadataStore localMetadataStore, MetadataStore configurationMetadataStore, - int operationTimeoutSec) { + int operationTimeoutSec, Executor executor) { if (configurationMetadataStore != null) { tenantResources = new TenantResources(configurationMetadataStore, operationTimeoutSec); clusterResources = new ClusterResources(localMetadataStore, configurationMetadataStore, operationTimeoutSec); - namespaceResources = new NamespaceResources(configurationMetadataStore, operationTimeoutSec); + namespaceResources = new NamespaceResources(configurationMetadataStore, operationTimeoutSec, executor); resourcegroupResources = new ResourceGroupResources(configurationMetadataStore, operationTimeoutSec); } else { tenantResources = null; diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java index 0963f25c3d31f..413184764f52b 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java @@ -120,7 +120,7 @@ public CompletableFuture clearTenantPersistence(String tenant) { return store.exists(path) .thenCompose(exists -> { if (exists) { - return store.delete(path, Optional.empty()); + return store.deleteRecursive(path); } else { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java index e5daa5852b51f..fc88647eb49ea 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java @@ -18,9 +18,23 @@ */ package org.apache.pulsar.broker.web; +import java.net.InetSocketAddress; import java.util.TimeZone; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.extern.slf4j.Slf4j; +import org.eclipse.jetty.io.Connection; +import org.eclipse.jetty.io.EndPoint; +import org.eclipse.jetty.server.Connector; import org.eclipse.jetty.server.CustomRequestLog; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.Request; +import org.eclipse.jetty.server.RequestLog; +import org.eclipse.jetty.server.Response; +import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.Slf4jRequestLogWriter; +import org.eclipse.jetty.util.HostPort; +import org.eclipse.jetty.util.component.ContainerLifeCycle; /** * Class to standardize initialization of a Jetty request logger for all pulsar components. @@ -58,7 +72,184 @@ public class JettyRequestLogFactory { * Build a new Jetty request logger using the format defined in this class. * @return a request logger */ - public static CustomRequestLog createRequestLogger() { - return new CustomRequestLog(new Slf4jRequestLogWriter(), LOG_FORMAT); + public static RequestLog createRequestLogger() { + return createRequestLogger(false, null); + } + + /** + * Build a new Jetty request logger using the format defined in this class. + * @param showDetailedAddresses whether to show detailed addresses and ports in logs + * @return a request logger + */ + public static RequestLog createRequestLogger(boolean showDetailedAddresses, Server server) { + if (!showDetailedAddresses) { + return new CustomRequestLog(new Slf4jRequestLogWriter(), LOG_FORMAT); + } else { + return new OriginalClientIPRequestLog(server); + } + } + + /** + * Logs the original and real remote (client) and local (server) IP addresses + * when detailed addresses are enabled. + * Tracks the real addresses of remote and local using a registered Connection.Listener + * when detailed addresses are enabled. + * This is necessary when Proxy Protocol is used to pass the original client IP. + */ + @Slf4j + private static class OriginalClientIPRequestLog extends ContainerLifeCycle implements RequestLog { + private final ThreadLocal requestLogStringBuilder = ThreadLocal.withInitial(StringBuilder::new); + private final CustomRequestLog delegate; + private final Slf4jRequestLogWriter delegateLogWriter; + + OriginalClientIPRequestLog(Server server) { + delegate = new CustomRequestLog(this::write, LOG_FORMAT); + addBean(delegate); + delegateLogWriter = new Slf4jRequestLogWriter(); + addBean(delegateLogWriter); + if (server != null) { + for (Connector connector : server.getConnectors()) { + // adding the listener is only necessary for connectors that use ProxyConnectionFactory + if (connector.getDefaultConnectionFactory() instanceof ProxyConnectionFactory) { + connector.addBean(proxyProtocolOriginalEndpointListener); + } + } + } + } + + void write(String requestEntry) { + StringBuilder sb = requestLogStringBuilder.get(); + sb.setLength(0); + sb.append(requestEntry); + } + + @Override + public void log(Request request, Response response) { + delegate.log(request, response); + StringBuilder sb = requestLogStringBuilder.get(); + sb.append(" [R:"); + sb.append(request.getRemoteHost()); + sb.append(':'); + sb.append(request.getRemotePort()); + InetSocketAddress realRemoteAddress = lookupRealAddress(request.getHttpChannel().getRemoteAddress()); + if (realRemoteAddress != null) { + String realRemoteHost = HostPort.normalizeHost(realRemoteAddress.getHostString()); + int realRemotePort = realRemoteAddress.getPort(); + if (!realRemoteHost.equals(request.getRemoteHost()) || realRemotePort != request.getRemotePort()) { + sb.append(" via "); + sb.append(realRemoteHost); + sb.append(':'); + sb.append(realRemotePort); + } + } + sb.append("]->[L:"); + InetSocketAddress realLocalAddress = lookupRealAddress(request.getHttpChannel().getLocalAddress()); + if (realLocalAddress != null) { + String realLocalHost = HostPort.normalizeHost(realLocalAddress.getHostString()); + int realLocalPort = realLocalAddress.getPort(); + sb.append(realLocalHost); + sb.append(':'); + sb.append(realLocalPort); + if (!realLocalHost.equals(request.getLocalAddr()) || realLocalPort != request.getLocalPort()) { + sb.append(" dst "); + sb.append(request.getLocalAddr()); + sb.append(':'); + sb.append(request.getLocalPort()); + } + } else { + sb.append(request.getLocalAddr()); + sb.append(':'); + sb.append(request.getLocalPort()); + } + sb.append(']'); + try { + delegateLogWriter.write(sb.toString()); + } catch (Exception e) { + log.warn("Failed to write request log", e); + } + } + + private InetSocketAddress lookupRealAddress(InetSocketAddress socketAddress) { + if (socketAddress == null) { + return null; + } + if (proxyProtocolRealAddressMapping.isEmpty()) { + return socketAddress; + } + AddressEntry entry = proxyProtocolRealAddressMapping.get(new AddressKey(socketAddress.getHostString(), + socketAddress.getPort())); + if (entry != null) { + return entry.realAddress; + } else { + return socketAddress; + } + } + + private final Connection.Listener proxyProtocolOriginalEndpointListener = + new ProxyProtocolOriginalEndpointListener(); + + private final ConcurrentHashMap proxyProtocolRealAddressMapping = + new ConcurrentHashMap<>(); + + // Use a record as key since InetSocketAddress hash code changes if the address gets resolved + record AddressKey(String hostString, int port) { + + } + + record AddressEntry(InetSocketAddress realAddress, AtomicInteger referenceCount) { + + } + + // Tracks the real addresses of remote and local when detailed addresses are enabled. + // This is necessary when Proxy Protocol is used to pass the original client IP. + // The Proxy Protocol implementation in Jetty wraps the original endpoint with a ProxyEndPoint + // and the real endpoint information isn't available in the request object. + // This listener is added to all connectors to track the real addresses of the client and server. + class ProxyProtocolOriginalEndpointListener implements Connection.Listener { + @Override + public void onOpened(Connection connection) { + handleConnection(connection, true); + } + + @Override + public void onClosed(Connection connection) { + handleConnection(connection, false); + } + + private void handleConnection(Connection connection, boolean increment) { + if (connection.getEndPoint() instanceof ProxyConnectionFactory.ProxyEndPoint) { + ProxyConnectionFactory.ProxyEndPoint proxyEndPoint = + (ProxyConnectionFactory.ProxyEndPoint) connection.getEndPoint(); + EndPoint originalEndpoint = proxyEndPoint.unwrap(); + mapAddress(proxyEndPoint.getLocalAddress(), originalEndpoint.getLocalAddress(), increment); + mapAddress(proxyEndPoint.getRemoteAddress(), originalEndpoint.getRemoteAddress(), increment); + } + } + + private void mapAddress(InetSocketAddress current, InetSocketAddress real, boolean increment) { + // don't add the mapping if the current address is the same as the real address + if (real != null && current != null && current.equals(real)) { + return; + } + AddressKey key = new AddressKey(current.getHostString(), current.getPort()); + proxyProtocolRealAddressMapping.compute(key, (__, entry) -> { + if (entry == null) { + if (increment) { + entry = new AddressEntry(real, new AtomicInteger(1)); + } + } else { + if (increment) { + entry.referenceCount.incrementAndGet(); + } else { + if (entry.referenceCount.decrementAndGet() == 0) { + // remove the entry if the reference count drops to 0 + entry = null; + } + } + } + return entry; + }); + } + } } } diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index e15e024ea8158..3548877912199 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -176,6 +176,13 @@ test + + io.github.hakky54 + consolecaptor + ${consolecaptor.version} + test + + io.streamnative.oxia oxia-testcontainers diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 7613a13db22de..96f3653ea9966 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -444,6 +444,9 @@ public CompletableFuture closeAsync() { return closeFuture; } LOG.info("Closing PulsarService"); + if (brokerService != null) { + brokerService.unloadNamespaceBundlesGracefully(); + } state = State.Closing; // close the service in reverse order v.s. in which they are started @@ -562,6 +565,11 @@ public CompletableFuture closeAsync() { transactionBufferClient.close(); } + if (topicPoliciesService != null) { + topicPoliciesService.close(); + topicPoliciesService = null; + } + if (client != null) { client.close(); client = null; @@ -992,7 +1000,7 @@ protected ManagedLedgerStorage newManagedLedgerClientFactory() throws Exception @VisibleForTesting protected PulsarResources newPulsarResources() { PulsarResources pulsarResources = new PulsarResources(localMetadataStore, configurationMetadataStore, - config.getMetadataStoreOperationTimeoutSeconds()); + config.getMetadataStoreOperationTimeoutSeconds(), getExecutor()); pulsarResources.getClusterResources().getStore().registerListener(this::handleDeleteCluster); return pulsarResources; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index a1bfeb2142ffc..45455f16d4dc1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -23,6 +23,7 @@ import com.google.errorprone.annotations.CanIgnoreReturnValue; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -43,9 +44,11 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.broker.resources.ClusterResources; import org.apache.pulsar.broker.service.plugin.InvalidEntryFilterException; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.internal.TopicsImpl; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.naming.Constants; @@ -621,35 +624,82 @@ protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int n private void internalCreatePartitionedTopicToReplicatedClustersInBackground(int numPartitions) { getNamespaceReplicatedClustersAsync(namespaceName) - .thenAccept(clusters -> { - for (String cluster : clusters) { - if (!cluster.equals(pulsar().getConfiguration().getClusterName())) { - // this call happens in the background without async composition. completion is logged. - pulsar().getPulsarResources().getClusterResources() - .getClusterAsync(cluster) - .thenCompose(clusterDataOp -> - ((TopicsImpl) pulsar().getBrokerService() - .getClusterPulsarAdmin(cluster, - clusterDataOp).topics()) - .createPartitionedTopicAsync( - topicName.getPartitionedTopicName(), - numPartitions, - true, null)) - .whenComplete((__, ex) -> { - if (ex != null) { - log.error( - "[{}] Failed to create partitioned topic {} in cluster {}.", - clientAppId(), topicName, cluster, ex); - } else { - log.info( - "[{}] Successfully created partitioned topic {} in " - + "cluster {}", - clientAppId(), topicName, cluster); - } - }); - } + .thenAccept(clusters -> { + // this call happens in the background without async composition. completion is logged. + internalCreatePartitionedTopicToReplicatedClustersInBackground(clusters, numPartitions); + }); + } + + protected Map> internalCreatePartitionedTopicToReplicatedClustersInBackground( + Set clusters, int numPartitions) { + final String shortTopicName = topicName.getPartitionedTopicName(); + Map> tasksForAllClusters = new HashMap<>(); + for (String cluster : clusters) { + if (cluster.equals(pulsar().getConfiguration().getClusterName())) { + continue; + } + ClusterResources clusterResources = pulsar().getPulsarResources().getClusterResources(); + CompletableFuture createRemoteTopicFuture = new CompletableFuture<>(); + tasksForAllClusters.put(cluster, createRemoteTopicFuture); + clusterResources.getClusterAsync(cluster).whenComplete((clusterData, ex1) -> { + if (ex1 != null) { + // Unexpected error, such as NPE. Catch all error to avoid the "createRemoteTopicFuture" stuck. + log.error("[{}] An un-expected error occurs when trying to create partitioned topic {} in cluster" + + " {}.", clientAppId(), topicName, cluster, ex1); + createRemoteTopicFuture.completeExceptionally(new RestException(ex1)); + return; + } + // Get cluster data success. + TopicsImpl topics = + (TopicsImpl) pulsar().getBrokerService().getClusterPulsarAdmin(cluster, clusterData).topics(); + topics.createPartitionedTopicAsync(shortTopicName, numPartitions, true, null) + .whenComplete((ignore, ex2) -> { + if (ex2 == null) { + // Create success. + log.info("[{}] Successfully created partitioned topic {} in cluster {}", + clientAppId(), topicName, cluster); + createRemoteTopicFuture.complete(null); + return; + } + // Create topic on the remote cluster error. + Throwable unwrapEx2 = FutureUtil.unwrapCompletionException(ex2); + // The topic has been created before, check the partitions count is expected. + if (unwrapEx2 instanceof PulsarAdminException.ConflictException) { + topics.getPartitionedTopicMetadataAsync(shortTopicName).whenComplete((topicMeta, ex3) -> { + if (ex3 != null) { + // Unexpected error, such as NPE. Catch all error to avoid the + // "createRemoteTopicFuture" stuck. + log.error("[{}] Failed to check remote-cluster's topic metadata when creating" + + " partitioned topic {} in cluster {}.", + clientAppId(), topicName, cluster, ex3); + createRemoteTopicFuture.completeExceptionally(new RestException(ex3)); + } + // Call get partitioned metadata of remote cluster success. + if (topicMeta.partitions == numPartitions) { + log.info("[{}] Skip created partitioned topic {} in cluster {}, because that {}", + clientAppId(), topicName, cluster, unwrapEx2.getMessage()); + createRemoteTopicFuture.complete(null); + } else { + String errorMsg = String.format("[%s] There is an exists topic %s with different" + + " partitions %s on the remote cluster %s, you want to create it" + + " with partitions %s", + clientAppId(), shortTopicName, topicMeta.partitions, cluster, + numPartitions); + log.error(errorMsg); + createRemoteTopicFuture.completeExceptionally( + new RestException(Status.PRECONDITION_FAILED, errorMsg)); + } + }); + } else { + // An HTTP error was responded from the remote cluster. + log.error("[{}] Failed to create partitioned topic {} in cluster {}.", + clientAppId(), topicName, cluster, ex2); + createRemoteTopicFuture.completeExceptionally(new RestException(unwrapEx2)); } }); + }); + } + return tasksForAllClusters; } /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java index bbadc7bb3316d..5f2dccc3e9c24 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java @@ -2019,7 +2019,7 @@ protected void internalSetMaxUnackedMessagesPerConsumer(Integer maxUnackedMessag } protected void internalSetMaxSubscriptionsPerTopic(Integer maxSubscriptionsPerTopic){ - validateNamespacePolicyOperationAsync(namespaceName, PolicyName.MAX_SUBSCRIPTIONS, PolicyOperation.WRITE); + validateNamespacePolicyOperation(namespaceName, PolicyName.MAX_SUBSCRIPTIONS, PolicyOperation.WRITE); validatePoliciesReadOnlyAccess(); if (maxSubscriptionsPerTopic != null && maxSubscriptionsPerTopic < 0) { throw new RestException(Status.PRECONDITION_FAILED, @@ -2125,9 +2125,10 @@ protected CompletableFuture internalSetOffloadThresholdInSecondsAsync(long f.complete(null); }) .exceptionally(t -> { + Throwable cause = FutureUtil.unwrapCompletionException(t); log.error("[{}] Failed to update offloadThresholdInSeconds configuration for namespace {}", clientAppId(), namespaceName, t); - f.completeExceptionally(new RestException(t)); + f.completeExceptionally(new RestException(cause)); return null; }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 1f8d06571908e..682f41dcdb61f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -205,6 +205,7 @@ protected CompletableFuture> internalGetPartitionedTopicListAsync() protected CompletableFuture>> internalGetPermissionsOnTopic() { // This operation should be reading from zookeeper and it should be allowed without having admin privileges return validateAdminAccessForTenantAsync(namespaceName.getTenant()) + .thenCompose(__ -> internalCheckTopicExists(topicName)) .thenCompose(__ -> getAuthorizationService().getPermissionsAsync(topicName)); } @@ -256,9 +257,10 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse Set actions) { // This operation should be reading from zookeeper and it should be allowed without having admin privileges validateAdminAccessForTenantAsync(namespaceName.getTenant()) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 -> - grantPermissionsAsync(topicName, role, actions) - .thenAccept(unused -> asyncResponse.resume(Response.noContent().build())))) + .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + .thenCompose(__ -> internalCheckTopicExists(topicName)) + .thenCompose(unused1 -> grantPermissionsAsync(topicName, role, actions)) + .thenAccept(unused -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { Throwable realCause = FutureUtil.unwrapCompletionException(ex); log.error("[{}] Failed to get permissions for topic {}", clientAppId(), topicName, realCause); @@ -270,6 +272,7 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse protected void internalRevokePermissionsOnTopic(AsyncResponse asyncResponse, String role) { // This operation should be reading from zookeeper and it should be allowed without having admin privileges validateAdminAccessForTenantAsync(namespaceName.getTenant()) + .thenCompose(__ -> internalCheckTopicExists(topicName)) .thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, true, false) .thenCompose(metadata -> { int numPartitions = metadata.partitions; @@ -3253,12 +3256,13 @@ protected CompletableFuture internalSetBacklogQuota(BacklogQuota.BacklogQu } protected CompletableFuture internalSetReplicationClusters(List clusterIds) { + if (CollectionUtils.isEmpty(clusterIds)) { + return CompletableFuture.failedFuture(new RestException(Status.PRECONDITION_FAILED, + "ClusterIds should not be null or empty")); + } + Set replicationClusters = Sets.newHashSet(clusterIds); return validatePoliciesReadOnlyAccessAsync() .thenCompose(__ -> { - if (CollectionUtils.isEmpty(clusterIds)) { - throw new RestException(Status.PRECONDITION_FAILED, "ClusterIds should not be null or empty"); - } - Set replicationClusters = Sets.newHashSet(clusterIds); if (replicationClusters.contains("global")) { throw new RestException(Status.PRECONDITION_FAILED, "Cannot specify global in the list of replication clusters"); @@ -3273,6 +3277,20 @@ protected CompletableFuture internalSetReplicationClusters(List cl futures.add(validateClusterForTenantAsync(namespaceName.getTenant(), clusterId)); } return FutureUtil.waitForAll(futures); + }).thenCompose(__ -> { + // Sync to create partitioned topic on the remote cluster if needed. + TopicName topicNameWithoutPartition = TopicName.get(topicName.getPartitionedTopicName()); + return pulsar().getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .getPartitionedTopicMetadataAsync(topicNameWithoutPartition).thenCompose(topicMetaOp -> { + // Skip to create topic if the topic is non-partitioned, because the replicator will create + // it automatically. + if (topicMetaOp.isEmpty()) { + return CompletableFuture.completedFuture(null); + } + return FutureUtil.waitForAll( + internalCreatePartitionedTopicToReplicatedClustersInBackground(replicationClusters, + topicMetaOp.get().partitions).values()); + }); }).thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName).thenCompose(op -> { TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java index 2cce68b60cb49..0dd5d948480ab 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java @@ -31,6 +31,7 @@ import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; import org.apache.pulsar.broker.lookup.LookupResult; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.common.naming.ServiceUnitId; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.Reflections; @@ -63,7 +64,7 @@ public interface LoadManager { Optional getLeastLoaded(ServiceUnitId su) throws Exception; default CompletableFuture> findBrokerServiceUrl( - Optional topic, ServiceUnitId bundle) { + Optional topic, ServiceUnitId bundle, LookupOptions options) { throw new UnsupportedOperationException(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManager.java index b7da70d1cf1de..eabf6005b439b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManager.java @@ -60,9 +60,12 @@ public interface ExtensibleLoadManager extends Closeable { * (e.g. {@link NamespaceService#internalGetWebServiceUrl(NamespaceBundle, LookupOptions)}), * So the topic is optional. * @param serviceUnit service unit (e.g. bundle). + * @param options The lookup options. * @return The broker lookup data. */ - CompletableFuture> assign(Optional topic, ServiceUnitId serviceUnit); + CompletableFuture> assign(Optional topic, + ServiceUnitId serviceUnit, + LookupOptions options); /** * Check the incoming service unit is owned by the current broker. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index c8cf1c05756a6..41832fb60075d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -85,9 +85,11 @@ import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreException; import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreFactory; import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy; +import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategyFactory; import org.apache.pulsar.broker.loadbalance.extensions.strategy.LeastResourceUsageWithWeight; import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceAllocationPolicies; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -103,7 +105,7 @@ import org.slf4j.Logger; @Slf4j -public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { +public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerSelectionStrategyFactory { public static final String BROKER_LOAD_DATA_STORE_TOPIC = TopicName.get( TopicDomain.non_persistent.value(), @@ -251,6 +253,11 @@ public Set getOwnedServiceUnits() { return ownedServiceUnits; } + @Override + public BrokerSelectionStrategy createBrokerSelectionStrategy() { + return new LeastResourceUsageWithWeight(); + } + public enum Role { Leader, Follower @@ -266,8 +273,7 @@ public ExtensibleLoadManagerImpl() { this.brokerFilterPipeline.add(new BrokerLoadManagerClassFilter()); this.brokerFilterPipeline.add(new BrokerMaxTopicCountFilter()); this.brokerFilterPipeline.add(new BrokerVersionFilter()); - // TODO: Make brokerSelectionStrategy configurable. - this.brokerSelectionStrategy = new LeastResourceUsageWithWeight(); + this.brokerSelectionStrategy = createBrokerSelectionStrategy(); } public static boolean isLoadManagerExtensionEnabled(PulsarService pulsar) { @@ -482,7 +488,8 @@ public void initialize(PulsarService pulsar) { @Override public CompletableFuture> assign(Optional topic, - ServiceUnitId serviceUnit) { + ServiceUnitId serviceUnit, + LookupOptions options) { final String bundle = serviceUnit.toString(); @@ -496,7 +503,7 @@ public CompletableFuture> assign(Optional getHeartbeatOrSLAMonitorBrokerId(ServiceUnitId } private CompletableFuture getOrSelectOwnerAsync(ServiceUnitId serviceUnit, - String bundle) { + String bundle, + LookupOptions options) { return serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> { // If the bundle not assign yet, select and publish assign event to channel. if (broker.isEmpty()) { - return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> { + return this.selectAsync(serviceUnit, Collections.emptySet(), options).thenCompose(brokerOpt -> { if (brokerOpt.isPresent()) { assignCounter.incrementSuccess(); log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle); return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get()); } - throw new IllegalStateException( - "Failed to select the new owner broker for bundle: " + bundle); + return CompletableFuture.completedFuture(null); }); } assignCounter.incrementSkip(); @@ -534,22 +541,19 @@ private CompletableFuture> getBrokerLookupData( String bundle) { return owner.thenCompose(broker -> { if (broker.isEmpty()) { - String errorMsg = String.format( - "Failed to get or assign the owner for bundle:%s", bundle); - log.error(errorMsg); - throw new IllegalStateException(errorMsg); - } - return CompletableFuture.completedFuture(broker.get()); - }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> { - if (brokerLookupData.isEmpty()) { - String errorMsg = String.format( - "Failed to lookup broker:%s for bundle:%s, the broker has not been registered.", - broker, bundle); - log.error(errorMsg); - throw new IllegalStateException(errorMsg); + return CompletableFuture.completedFuture(Optional.empty()); } - return CompletableFuture.completedFuture(brokerLookupData); - })); + return this.getBrokerRegistry().lookupAsync(broker.get()).thenCompose(brokerLookupData -> { + if (brokerLookupData.isEmpty()) { + String errorMsg = String.format( + "Failed to lookup broker:%s for bundle:%s, the broker has not been registered.", + broker, bundle); + log.error(errorMsg); + throw new IllegalStateException(errorMsg); + } + return CompletableFuture.completedFuture(brokerLookupData); + }); + }); } /** @@ -562,7 +566,7 @@ private CompletableFuture> getBrokerLookupData( public CompletableFuture tryAcquiringOwnership(NamespaceBundle namespaceBundle) { log.info("Try acquiring ownership for bundle: {} - {}.", namespaceBundle, brokerRegistry.getBrokerId()); final String bundle = namespaceBundle.toString(); - return assign(Optional.empty(), namespaceBundle) + return assign(Optional.empty(), namespaceBundle, LookupOptions.builder().readOnly(false).build()) .thenApply(brokerLookupData -> { if (brokerLookupData.isEmpty()) { String errorMsg = String.format( @@ -595,12 +599,12 @@ private CompletableFuture> dedupeLookupRequest( } } - public CompletableFuture> selectAsync(ServiceUnitId bundle) { - return selectAsync(bundle, Collections.emptySet()); - } - public CompletableFuture> selectAsync(ServiceUnitId bundle, - Set excludeBrokerSet) { + Set excludeBrokerSet, + LookupOptions options) { + if (options.isReadOnly()) { + return CompletableFuture.completedFuture(Optional.empty()); + } BrokerRegistry brokerRegistry = getBrokerRegistry(); return brokerRegistry.getAvailableBrokerLookupDataAsync() .thenComposeAsync(availableBrokers -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java index cd1561cb70e2d..25eb27bc58d27 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java @@ -28,10 +28,11 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.loadbalance.LoadManager; import org.apache.pulsar.broker.loadbalance.ResourceUnit; -import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.lookup.LookupResult; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.common.naming.ServiceUnitId; import org.apache.pulsar.common.stats.Metrics; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; public class ExtensibleLoadManagerWrapper implements LoadManager { @@ -62,9 +63,15 @@ public boolean isCentralized() { @Override public CompletableFuture> findBrokerServiceUrl( - Optional topic, ServiceUnitId bundle) { - return loadManager.assign(topic, bundle) - .thenApply(lookupData -> lookupData.map(BrokerLookupData::toLookupResult)); + Optional topic, ServiceUnitId bundle, LookupOptions options) { + return loadManager.assign(topic, bundle, options) + .thenApply(lookupData -> lookupData.map(data -> { + try { + return data.toLookupResult(options); + } catch (PulsarServerException ex) { + throw FutureUtil.wrapToCompletionException(ex); + } + })); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index e355187af4ba2..bf6266482f8f0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -83,6 +83,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.models.Split; import org.apache.pulsar.broker.loadbalance.extensions.models.Unload; import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.client.api.CompressionType; @@ -1430,7 +1431,8 @@ private synchronized void doCleanup(String broker) { private Optional selectBroker(String serviceUnit, String inactiveBroker) { try { return loadManager.selectAsync( - LoadManagerShared.getNamespaceBundle(pulsar, serviceUnit), Set.of(inactiveBroker)) + LoadManagerShared.getNamespaceBundle(pulsar, serviceUnit), + Set.of(inactiveBroker), LookupOptions.builder().build()) .get(inFlightStateWaitingTimeInMillis, MILLISECONDS); } catch (Throwable e) { log.error("Failed to select a broker for serviceUnit:{}", serviceUnit); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java index 41f5b18e321e8..50a2b70404039 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java @@ -18,9 +18,12 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.data; +import java.net.URI; import java.util.Map; import java.util.Optional; +import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.lookup.LookupResult; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.policies.data.loadbalancer.AdvertisedListener; import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData; @@ -79,7 +82,19 @@ public long getStartTimestamp() { return this.startTimestamp; } - public LookupResult toLookupResult() { + public LookupResult toLookupResult(LookupOptions options) throws PulsarServerException { + if (options.hasAdvertisedListenerName()) { + AdvertisedListener listener = advertisedListeners.get(options.getAdvertisedListenerName()); + if (listener == null) { + throw new PulsarServerException("the broker do not have " + + options.getAdvertisedListenerName() + " listener"); + } + URI url = listener.getBrokerServiceUrl(); + URI urlTls = listener.getBrokerServiceUrlTls(); + return new LookupResult(webServiceUrl, webServiceUrlTls, + url == null ? null : url.toString(), + urlTls == null ? null : urlTls.toString(), LookupResult.Type.BrokerUrl, false); + } return new LookupResult(webServiceUrl, webServiceUrlTls, pulsarServiceUrl, pulsarServiceUrlTls, LookupResult.Type.BrokerUrl, false); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java index d916e91716223..81cf33b4a55d2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java @@ -161,12 +161,8 @@ public synchronized void init() throws IOException { } private void validateProducer() { - if (producer == null || !producer.isConnected()) { + if (producer == null) { try { - if (producer != null) { - producer.close(); - } - producer = null; startProducer(); log.info("Restarted producer on {}", topic); } catch (Exception e) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategy.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategy.java index e0a9122383c22..b240cb5b5f6a6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategy.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategy.java @@ -21,11 +21,13 @@ import java.util.Optional; import java.util.Set; import org.apache.pulsar.broker.loadbalance.extensions.LoadManagerContext; +import org.apache.pulsar.common.classification.InterfaceStability; import org.apache.pulsar.common.naming.ServiceUnitId; /** * The broker selection strategy is designed to select the broker according to different implementations. */ +@InterfaceStability.Evolving public interface BrokerSelectionStrategy { /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategyFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategyFactory.java new file mode 100644 index 0000000000000..61b9fbcfcb9e5 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategyFactory.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.loadbalance.extensions.strategy; + +import org.apache.pulsar.common.classification.InterfaceStability; + +@InterfaceStability.Stable +public interface BrokerSelectionStrategyFactory { + + BrokerSelectionStrategy createBrokerSelectionStrategy(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeight.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeight.java index 98986d84b9858..9bf16ac179532 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeight.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeight.java @@ -96,8 +96,7 @@ public Optional select( // select one of them at the end. double totalUsage = 0.0d; - // TODO: use loadBalancerDebugModeEnabled too. - boolean debugMode = log.isDebugEnabled(); + boolean debugMode = log.isDebugEnabled() || conf.isLoadBalancerDebugModeEnabled(); for (String broker : candidates) { var brokerLoadDataOptional = context.brokerLoadDataStore().get(broker); if (brokerLoadDataOptional.isEmpty()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 65081f2ea42b6..44cdd6368fe79 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -227,7 +227,7 @@ public CompletableFuture> getBrokerServiceUrlAsync(TopicN return CompletableFuture.completedFuture(optResult); } if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { - return loadManager.get().findBrokerServiceUrl(Optional.of(topic), bundle); + return loadManager.get().findBrokerServiceUrl(Optional.of(topic), bundle, options); } else { // TODO: Add unit tests cover it. return findBrokerServiceUrl(bundle, options); @@ -353,7 +353,7 @@ private CompletableFuture> internalGetWebServiceUrl(@Nullable Serv } CompletableFuture> future = ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar) - ? loadManager.get().findBrokerServiceUrl(Optional.ofNullable(topic), bundle) : + ? loadManager.get().findBrokerServiceUrl(Optional.ofNullable(topic), bundle, options) : findBrokerServiceUrl(bundle, options); return future.thenApply(lookupResult -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index 1b5b2824257b0..394fad21ae6dc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -18,16 +18,22 @@ */ package org.apache.pulsar.broker.service; +import com.google.common.annotations.VisibleForTesting; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import lombok.Getter; import org.apache.bookkeeper.mledger.Position; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.BrokerServiceException.NamingException; import org.apache.pulsar.broker.service.BrokerServiceException.TopicBusyException; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.Backoff; @@ -39,7 +45,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class AbstractReplicator { +public abstract class AbstractReplicator implements Replicator { protected final BrokerService brokerService; protected final String localTopicName; @@ -64,10 +70,31 @@ public abstract class AbstractReplicator { protected static final AtomicReferenceFieldUpdater STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(AbstractReplicator.class, State.class, "state"); - private volatile State state = State.Stopped; - - protected enum State { - Stopped, Starting, Started, Stopping + @VisibleForTesting + @Getter + protected volatile State state = State.Disconnected; + + public enum State { + /** + * This enum has two mean meanings: + * Init: replicator is just created, has not been started now. + * Disconnected: the producer was closed after {@link PersistentTopic#checkGC} called {@link #disconnect}. + */ + // The internal producer is disconnected. + Disconnected, + // Trying to create a new internal producer. + Starting, + // The internal producer has started, and tries copy data. + Started, + /** + * The producer is closing after {@link PersistentTopic#checkGC} called {@link #disconnect}. + */ + // The internal producer is trying to disconnect. + Disconnecting, + // The replicator is in terminating. + Terminating, + // The replicator is never used again. Pulsar will create a new Replicator when enable replication again. + Terminated; } public AbstractReplicator(String localCluster, Topic localTopic, String remoteCluster, String remoteTopicName, @@ -96,16 +123,16 @@ public AbstractReplicator(String localCluster, Topic localTopic, String remoteCl .sendTimeout(0, TimeUnit.SECONDS) // .maxPendingMessages(producerQueueSize) // .producerName(getProducerName()); - STATE_UPDATER.set(this, State.Stopped); + STATE_UPDATER.set(this, State.Disconnected); } protected abstract String getProducerName(); - protected abstract void readEntries(org.apache.pulsar.client.api.Producer producer); + protected abstract void setProducerAndTriggerReadEntries(org.apache.pulsar.client.api.Producer producer); protected abstract Position getReplicatorReadPosition(); - protected abstract long getNumberOfEntriesInBacklog(); + public abstract long getNumberOfEntriesInBacklog(); protected abstract void disableReplicatorRead(); @@ -113,66 +140,121 @@ public String getRemoteCluster() { return remoteCluster; } - // This method needs to be synchronized with disconnects else if there is a disconnect followed by startProducer - // the end result can be disconnect. - public synchronized void startProducer() { - if (STATE_UPDATER.get(this) == State.Stopping) { - long waitTimeMs = backOff.next(); - if (log.isDebugEnabled()) { - log.debug( - "[{}] waiting for producer to close before attempting to reconnect, retrying in {} s", - replicatorId, waitTimeMs / 1000.0); - } - // BackOff before retrying - brokerService.executor().schedule(this::checkTopicActiveAndRetryStartProducer, waitTimeMs, - TimeUnit.MILLISECONDS); - return; - } - State state = STATE_UPDATER.get(this); - if (!STATE_UPDATER.compareAndSet(this, State.Stopped, State.Starting)) { - if (state == State.Started) { - // Already running + public void startProducer() { + // Guarantee only one task call "producerBuilder.createAsync()". + Pair setStartingRes = compareSetAndGetState(State.Disconnected, State.Starting); + if (!setStartingRes.getLeft()) { + if (setStartingRes.getRight() == State.Starting) { + log.info("[{}] Skip the producer creation since other thread is doing starting, state : {}", + replicatorId, state); + } else if (setStartingRes.getRight() == State.Started) { + // Since the method "startProducer" will be called even if it is started, only print debug-level log. if (log.isDebugEnabled()) { - log.debug("[{}] Replicator was already running", replicatorId); + log.debug("[{}] Replicator was already running. state: {}", replicatorId, state); } + } else if (setStartingRes.getRight() == State.Disconnecting) { + if (log.isDebugEnabled()) { + log.debug("[{}] Rep.producer is closing, delay to retry(wait the producer close success)." + + " state: {}", replicatorId, state); + } + delayStartProducerAfterDisconnected(); } else { - log.info("[{}] Replicator already being started. Replicator state: {}", replicatorId, state); + /** {@link State.Terminating}, {@link State.Terminated}. **/ + log.info("[{}] Skip the producer creation since the replicator state is : {}", replicatorId, state); } - return; } log.info("[{}] Starting replicator", replicatorId); producerBuilder.createAsync().thenAccept(producer -> { - readEntries(producer); + setProducerAndTriggerReadEntries(producer); }).exceptionally(ex -> { - if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Stopped)) { + Pair setDisconnectedRes = compareSetAndGetState(State.Starting, State.Disconnected); + if (setDisconnectedRes.getLeft()) { long waitTimeMs = backOff.next(); log.warn("[{}] Failed to create remote producer ({}), retrying in {} s", replicatorId, ex.getMessage(), waitTimeMs / 1000.0); - // BackOff before retrying - brokerService.executor().schedule(this::checkTopicActiveAndRetryStartProducer, waitTimeMs, - TimeUnit.MILLISECONDS); + scheduleCheckTopicActiveAndStartProducer(waitTimeMs); } else { - log.warn("[{}] Failed to create remote producer. Replicator state: {}", replicatorId, - STATE_UPDATER.get(this), ex); + if (setDisconnectedRes.getRight() == State.Terminating + || setDisconnectedRes.getRight() == State.Terminated) { + log.info("[{}] Skip to create producer, because it has been terminated, state is : {}", + replicatorId, state); + } else { + /** {@link State.Disconnected}, {@link State.Starting}, {@link State.Started} **/ + // Since only one task can call "producerBuilder.createAsync()", this scenario is not expected. + // So print a warn log. + log.warn("[{}] Other thread will try to create the producer again. so skipped current one task." + + " State is : {}", + replicatorId, state); + } } return null; }); + } + /*** + * The producer is disconnecting, delay to start the producer. + * If we start a producer immediately, we will get a conflict producer(same name producer) registered error. + */ + protected void delayStartProducerAfterDisconnected() { + long waitTimeMs = backOff.next(); + if (log.isDebugEnabled()) { + log.debug( + "[{}] waiting for producer to close before attempting to reconnect, retrying in {} s", + replicatorId, waitTimeMs / 1000.0); + } + scheduleCheckTopicActiveAndStartProducer(waitTimeMs); } - protected void checkTopicActiveAndRetryStartProducer() { - isLocalTopicActive().thenAccept(isTopicActive -> { - if (isTopicActive) { - startProducer(); + protected void scheduleCheckTopicActiveAndStartProducer(final long waitTimeMs) { + brokerService.executor().schedule(() -> { + if (state == State.Terminating || state == State.Terminated) { + log.info("[{}] Skip scheduled to start the producer since the replicator state is : {}", + replicatorId, state); + return; } - }).exceptionally(ex -> { - log.warn("[{}] Stop retry to create producer due to topic load fail. Replicator state: {}", replicatorId, - STATE_UPDATER.get(this), ex); - return null; - }); + CompletableFuture> topicFuture = brokerService.getTopics().get(localTopicName); + if (topicFuture == null) { + // Topic closed. + log.info("[{}] Skip scheduled to start the producer since the topic was closed successfully." + + " And trigger a terminate.", replicatorId); + terminate(); + return; + } + topicFuture.thenAccept(optional -> { + if (optional.isEmpty()) { + // Topic closed. + log.info("[{}] Skip scheduled to start the producer since the topic was closed. And trigger a" + + " terminate.", replicatorId); + terminate(); + return; + } + if (optional.get() != localTopic) { + // Topic closed and created a new one, current replicator is outdated. + log.info("[{}] Skip scheduled to start the producer since the topic was closed. And trigger a" + + " terminate.", replicatorId); + terminate(); + return; + } + Replicator replicator = localTopic.getReplicators().get(remoteCluster); + if (replicator != AbstractReplicator.this) { + // Current replicator has been closed, and created a new one. + log.info("[{}] Skip scheduled to start the producer since a new replicator has instead current" + + " one. And trigger a terminate.", replicatorId); + terminate(); + return; + } + startProducer(); + }).exceptionally(ex -> { + log.error("[{}] [{}] Stop retry to create producer due to unknown error(topic create failed), and" + + " trigger a terminate. Replicator state: {}", + localTopicName, replicatorId, STATE_UPDATER.get(this), ex); + terminate(); + return null; + }); + }, waitTimeMs, TimeUnit.MILLISECONDS); } protected CompletableFuture isLocalTopicActive() { @@ -188,58 +270,134 @@ protected CompletableFuture isLocalTopicActive() { }, brokerService.executor()); } - protected synchronized CompletableFuture closeProducerAsync() { - if (producer == null) { - STATE_UPDATER.set(this, State.Stopped); + /** + * This method only be used by {@link PersistentTopic#checkGC} now. + */ + public CompletableFuture disconnect(boolean failIfHasBacklog, boolean closeTheStartingProducer) { + long backlog = getNumberOfEntriesInBacklog(); + if (failIfHasBacklog && backlog > 0) { + CompletableFuture disconnectFuture = new CompletableFuture<>(); + disconnectFuture.completeExceptionally(new TopicBusyException("Cannot close a replicator with backlog")); + if (log.isDebugEnabled()) { + log.debug("[{}] Replicator disconnect failed since topic has backlog", replicatorId); + } + return disconnectFuture; + } + log.info("[{}] Disconnect replicator at position {} with backlog {}", replicatorId, + getReplicatorReadPosition(), backlog); + return closeProducerAsync(closeTheStartingProducer); + } + + /** + * This method only be used by {@link PersistentTopic#checkGC} now. + */ + protected CompletableFuture closeProducerAsync(boolean closeTheStartingProducer) { + Pair setDisconnectingRes = compareSetAndGetState(State.Started, State.Disconnecting); + if (!setDisconnectingRes.getLeft()) { + if (setDisconnectingRes.getRight() == State.Starting) { + if (closeTheStartingProducer) { + /** + * Delay retry(wait for the start producer task is finish). + * Note: If the producer always start fail, the start producer task will always retry until the + * state changed to {@link State.Terminated}. + * Nit: The better solution is creating a {@link CompletableFuture} to trace the in-progress + * creation and call "inProgressCreationFuture.thenApply(closeProducer())". + */ + long waitTimeMs = backOff.next(); + brokerService.executor().schedule(() -> closeProducerAsync(true), + waitTimeMs, TimeUnit.MILLISECONDS); + } else { + log.info("[{}] Skip current producer closing since the previous producer has been closed," + + " and trying start a new one, state : {}", + replicatorId, setDisconnectingRes.getRight()); + } + } else if (setDisconnectingRes.getRight() == State.Disconnected + || setDisconnectingRes.getRight() == State.Disconnecting) { + log.info("[{}] Skip current producer closing since other thread did closing, state : {}", + replicatorId, setDisconnectingRes.getRight()); + } else if (setDisconnectingRes.getRight() == State.Terminating + || setDisconnectingRes.getRight() == State.Terminated) { + log.info("[{}] Skip current producer closing since other thread is doing termination, state : {}", + replicatorId, state); + } + log.info("[{}] Skip current termination since other thread is doing close producer or termination," + + " state : {}", replicatorId, state); return CompletableFuture.completedFuture(null); } - CompletableFuture future = producer.closeAsync(); + + // Close producer and update state. + return doCloseProducerAsync(producer, () -> { + Pair setDisconnectedRes = compareSetAndGetState(State.Disconnecting, State.Disconnected); + if (setDisconnectedRes.getLeft()) { + this.producer = null; + // deactivate further read + disableReplicatorRead(); + return; + } + if (setDisconnectedRes.getRight() == State.Terminating + || setDisconnectingRes.getRight() == State.Terminated) { + log.info("[{}] Skip setting state to terminated because it was terminated, state : {}", + replicatorId, state); + } else { + // Since only one task can call "doCloseProducerAsync(producer, action)", this scenario is not expected. + // So print a warn log. + log.warn("[{}] Other task has change the state to terminated. so skipped current one task." + + " State is : {}", + replicatorId, state); + } + }); + } + + protected CompletableFuture doCloseProducerAsync(Producer producer, Runnable actionAfterClosed) { + CompletableFuture future = + producer == null ? CompletableFuture.completedFuture(null) : producer.closeAsync(); return future.thenRun(() -> { - STATE_UPDATER.set(this, State.Stopped); - this.producer = null; - // deactivate further read - disableReplicatorRead(); + actionAfterClosed.run(); }).exceptionally(ex -> { long waitTimeMs = backOff.next(); log.warn( - "[{}] Exception: '{}' occurred while trying to close the producer." - + " retrying again in {} s", - replicatorId, ex.getMessage(), waitTimeMs / 1000.0); + "[{}] Exception: '{}' occurred while trying to close the producer. Replicator state: {}." + + " Retrying again in {} s.", + replicatorId, ex.getMessage(), state, waitTimeMs / 1000.0); // BackOff before retrying - brokerService.executor().schedule(this::closeProducerAsync, waitTimeMs, TimeUnit.MILLISECONDS); + brokerService.executor().schedule(() -> doCloseProducerAsync(producer, actionAfterClosed), + waitTimeMs, TimeUnit.MILLISECONDS); return null; }); } - - public CompletableFuture disconnect() { - return disconnect(false); + public CompletableFuture terminate() { + if (!tryChangeStatusToTerminating()) { + log.info("[{}] Skip current termination since other thread is doing termination, state : {}", replicatorId, + state); + return CompletableFuture.completedFuture(null); + } + return doCloseProducerAsync(producer, () -> { + STATE_UPDATER.set(this, State.Terminated); + this.producer = null; + // set the cursor as inactive. + disableReplicatorRead(); + // release resources. + doReleaseResources(); + }); } - public synchronized CompletableFuture disconnect(boolean failIfHasBacklog) { - if (failIfHasBacklog && getNumberOfEntriesInBacklog() > 0) { - CompletableFuture disconnectFuture = new CompletableFuture<>(); - disconnectFuture.completeExceptionally(new TopicBusyException("Cannot close a replicator with backlog")); - if (log.isDebugEnabled()) { - log.debug("[{}] Replicator disconnect failed since topic has backlog", replicatorId); - } - return disconnectFuture; - } + protected void doReleaseResources() {} - if (STATE_UPDATER.get(this) == State.Stopping) { - // Do nothing since the all "STATE_UPDATER.set(this, Stopping)" instructions are followed by - // closeProducerAsync() - // which will at some point change the state to stopped - return CompletableFuture.completedFuture(null); + protected boolean tryChangeStatusToTerminating() { + if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Terminating)){ + return true; } - - if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Stopping) - || STATE_UPDATER.compareAndSet(this, State.Started, State.Stopping)) { - log.info("[{}] Disconnect replicator at position {} with backlog {}", replicatorId, - getReplicatorReadPosition(), getNumberOfEntriesInBacklog()); + if (STATE_UPDATER.compareAndSet(this, State.Started, State.Terminating)){ + return true; } - - return closeProducerAsync(); + if (STATE_UPDATER.compareAndSet(this, State.Disconnecting, State.Terminating)){ + return true; + } + if (STATE_UPDATER.compareAndSet(this, State.Disconnected, State.Terminating)) { + return true; + } + return false; } public CompletableFuture remove() { @@ -300,4 +458,22 @@ public static CompletableFuture validatePartitionedTopicAsync(String topic public State getState() { return state; } + + protected ImmutablePair compareSetAndGetState(State expect, State update) { + State original1 = state; + if (STATE_UPDATER.compareAndSet(this, expect, update)) { + return ImmutablePair.of(true, expect); + } + State original2 = state; + // Maybe the value changed more than once even if "original1 == original2", but the probability is very small, + // so let's ignore this case for prevent using a lock. + if (original1 == original2) { + return ImmutablePair.of(false, original1); + } + return compareSetAndGetState(expect, update); + } + + public boolean isTerminated() { + return state == State.Terminating || state == State.Terminated; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 249008bad91ad..1f0cb12258e1d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -309,6 +309,7 @@ public class BrokerService implements Closeable { private Set brokerEntryPayloadProcessors; private final TopicEventsDispatcher topicEventsDispatcher = new TopicEventsDispatcher(); + private volatile boolean unloaded = false; public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws Exception { this.pulsar = pulsar; @@ -744,7 +745,7 @@ public CompletableFuture closeAndRemoveReplicationClient(String clusterNam if (ot.isPresent()) { Replicator r = ot.get().getReplicators().get(clusterName); if (r != null && r.isConnected()) { - r.disconnect(false).whenComplete((v, e) -> f.complete(null)); + r.terminate().whenComplete((v, e) -> f.complete(null)); return; } } @@ -926,9 +927,13 @@ public void unloadNamespaceBundlesGracefully() { } public void unloadNamespaceBundlesGracefully(int maxConcurrentUnload, boolean closeWithoutWaitingClientDisconnect) { + if (unloaded) { + return; + } try { log.info("Unloading namespace-bundles..."); // make broker-node unavailable from the cluster + long disableBrokerStartTime = System.nanoTime(); if (pulsar.getLoadManager() != null && pulsar.getLoadManager().get() != null) { try { pulsar.getLoadManager().get().disableBroker(); @@ -937,6 +942,10 @@ public void unloadNamespaceBundlesGracefully(int maxConcurrentUnload, boolean cl // still continue and release bundle ownership as broker's registration node doesn't exist. } } + double disableBrokerTimeSeconds = + TimeUnit.NANOSECONDS.toMillis((System.nanoTime() - disableBrokerStartTime)) + / 1000.0; + log.info("Disable broker in load manager completed in {} seconds", disableBrokerTimeSeconds); // unload all namespace-bundles gracefully long closeTopicsStartTime = System.nanoTime(); @@ -966,6 +975,8 @@ public void unloadNamespaceBundlesGracefully(int maxConcurrentUnload, boolean cl } } catch (Exception e) { log.error("Failed to disable broker from loadbalancer list {}", e.getMessage(), e); + } finally { + unloaded = true; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java index 831d6068e2097..6abe40f811d1d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java @@ -146,6 +146,10 @@ public static class TopicBusyException extends BrokerServiceException { public TopicBusyException(String msg) { super(msg); } + + public TopicBusyException(String msg, Throwable t) { + super(msg, t); + } } public static class TopicNotFoundException extends BrokerServiceException { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index 6b2028095e205..b1c3687b3a0f6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -286,16 +286,29 @@ public Future sendMessages(final List entries, EntryBatch totalChunkedMessages, redeliveryTracker, DEFAULT_CONSUMER_EPOCH); } + public Future sendMessages(final List entries, EntryBatchSizes batchSizes, + EntryBatchIndexesAcks batchIndexesAcks, + int totalMessages, long totalBytes, long totalChunkedMessages, + RedeliveryTracker redeliveryTracker, long epoch) { + return sendMessages(entries, null, batchSizes, batchIndexesAcks, totalMessages, totalBytes, + totalChunkedMessages, redeliveryTracker, epoch); + } + /** * Dispatch a list of entries to the consumer.
* It is also responsible to release entries data and recycle entries object. * * @return a SendMessageInfo object that contains the detail of what was sent to consumer */ - public Future sendMessages(final List entries, EntryBatchSizes batchSizes, + public Future sendMessages(final List entries, + final List stickyKeyHashes, + EntryBatchSizes batchSizes, EntryBatchIndexesAcks batchIndexesAcks, - int totalMessages, long totalBytes, long totalChunkedMessages, - RedeliveryTracker redeliveryTracker, long epoch) { + int totalMessages, + long totalBytes, + long totalChunkedMessages, + RedeliveryTracker redeliveryTracker, + long epoch) { this.lastConsumedTimestamp = System.currentTimeMillis(); if (entries.isEmpty() || totalMessages == 0) { @@ -323,7 +336,7 @@ public Future sendMessages(final List entries, EntryBatch // because this consumer is possible to disconnect at this time. if (pendingAcks != null) { int batchSize = batchSizes.getBatchSize(i); - int stickyKeyHash = getStickyKeyHash(entry); + int stickyKeyHash = stickyKeyHashes == null ? getStickyKeyHash(entry) : stickyKeyHashes.get(i); long[] ackSet = batchIndexesAcks == null ? null : batchIndexesAcks.getAckSet(i); if (ackSet != null) { unackedMessages -= (batchSize - BitSet.valueOf(ackSet).cardinality()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java index 482fa2cbd2300..5c314397da80e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java @@ -29,9 +29,9 @@ public interface Replicator { ReplicatorStatsImpl getStats(); - CompletableFuture disconnect(); + CompletableFuture terminate(); - CompletableFuture disconnect(boolean b); + CompletableFuture disconnect(boolean failIfHasBacklog, boolean closeTheStartingProducer); void updateRates(); @@ -51,4 +51,6 @@ default Optional getRateLimiter() { boolean isConnected(); long getNumberOfEntriesInBacklog(); + + boolean isTerminated(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index 0449e5c885cd3..6d18d6d61b08e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -32,6 +32,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nonnull; import org.apache.commons.lang3.concurrent.ConcurrentInitializer; @@ -72,6 +73,7 @@ public class SystemTopicBasedTopicPoliciesService implements TopicPoliciesServic private final PulsarService pulsarService; private final HashSet localCluster; private final String clusterName; + private final AtomicBoolean closed = new AtomicBoolean(false); private final ConcurrentInitializer namespaceEventsSystemTopicFactoryLazyInitializer = new LazyInitializer<>() { @@ -110,12 +112,18 @@ public SystemTopicBasedTopicPoliciesService(PulsarService pulsarService) { this.writerCaches = Caffeine.newBuilder() .expireAfterAccess(5, TimeUnit.MINUTES) .removalListener((namespaceName, writer, cause) -> { - ((SystemTopicClient.Writer) writer).closeAsync().exceptionally(ex -> { - log.error("[{}] Close writer error.", namespaceName, ex); - return null; - }); + try { + ((SystemTopicClient.Writer) writer).close(); + } catch (Exception e) { + log.error("[{}] Close writer error.", namespaceName, e); + } }) + .executor(pulsarService.getExecutor()) .buildAsync((namespaceName, executor) -> { + if (closed.get()) { + return CompletableFuture.failedFuture( + new BrokerServiceException(getClass().getName() + " is closed.")); + } SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(namespaceName); return systemTopicClient.newWriterAsync(); @@ -382,6 +390,10 @@ public CompletableFuture addOwnedNamespaceBundleAsync(NamespaceBundle name protected CompletableFuture> createSystemTopicClient( NamespaceName namespace) { + if (closed.get()) { + return CompletableFuture.failedFuture( + new BrokerServiceException(getClass().getName() + " is closed.")); + } try { createSystemTopicFactoryIfNeeded(); } catch (PulsarServerException ex) { @@ -430,6 +442,11 @@ public boolean test(NamespaceBundle namespaceBundle) { } private void initPolicesCache(SystemTopicClient.Reader reader, CompletableFuture future) { + if (closed.get()) { + future.completeExceptionally(new BrokerServiceException(getClass().getName() + " is closed.")); + cleanCacheAndCloseReader(reader.getSystemTopic().getTopicName().getNamespaceObject(), false); + return; + } reader.hasMoreEventsAsync().whenComplete((hasMore, ex) -> { if (ex != null) { log.error("[{}] Failed to check the move events for the system topic", @@ -511,6 +528,10 @@ private void cleanCacheAndCloseReader(@Nonnull NamespaceName namespace, boolean * #{@link SystemTopicBasedTopicPoliciesService#getTopicPoliciesAsync(TopicName)} method to block loading topic. */ private void readMorePoliciesAsync(SystemTopicClient.Reader reader) { + if (closed.get()) { + cleanCacheAndCloseReader(reader.getSystemTopic().getTopicName().getNamespaceObject(), false); + return; + } reader.readNextAsync() .thenAccept(msg -> { refreshTopicPoliciesCache(msg); @@ -628,11 +649,20 @@ private NamespaceEventsSystemTopicFactory getNamespaceEventsSystemTopicFactory() private void fetchTopicPoliciesAsyncAndCloseReader(SystemTopicClient.Reader reader, TopicName topicName, TopicPolicies policies, CompletableFuture future) { + if (closed.get()) { + future.completeExceptionally(new BrokerServiceException(getClass().getName() + " is closed.")); + reader.closeAsync().whenComplete((v, e) -> { + if (e != null) { + log.error("[{}] Close reader error.", topicName, e); + } + }); + return; + } reader.hasMoreEventsAsync().whenComplete((hasMore, ex) -> { if (ex != null) { future.completeExceptionally(ex); } - if (hasMore) { + if (hasMore != null && hasMore) { reader.readNextAsync().whenComplete((msg, e) -> { if (e != null) { future.completeExceptionally(e); @@ -656,7 +686,9 @@ private void fetchTopicPoliciesAsyncAndCloseReader(SystemTopicClient.Reader { if (e != null) { log.error("[{}] Close reader error.", topicName, e); @@ -740,4 +772,23 @@ protected AsyncLoadingCache } private static final Logger log = LoggerFactory.getLogger(SystemTopicBasedTopicPoliciesService.class); + + @Override + public void close() throws Exception { + if (closed.compareAndSet(false, true)) { + writerCaches.synchronous().invalidateAll(); + readerCaches.values().forEach(future -> { + if (future != null && !future.isCompletedExceptionally()) { + future.thenAccept(reader -> { + try { + reader.close(); + } catch (Exception e) { + log.error("Failed to close reader.", e); + } + }); + } + }); + readerCaches.clear(); + } + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java index 332d754cf97d2..bd1b90981695e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java @@ -142,8 +142,26 @@ private SystemTopicClient getTransactionBufferSystemTopicClient(NamespaceName public void close() throws Exception { for (Map.Entry> entry : clients.entrySet()) { - entry.getValue().close(); + try { + entry.getValue().close(); + } catch (Exception e) { + log.error("Failed to close system topic client for namespace {}", entry.getKey(), e); + } + } + clients.clear(); + for (Map.Entry> entry : refCountedWriterMap.entrySet()) { + CompletableFuture> future = entry.getValue().getFuture(); + if (!future.isCompletedExceptionally()) { + future.thenAccept(writer -> { + try { + writer.close(); + } catch (Exception e) { + log.error("Failed to close writer for namespace {}", entry.getKey(), e); + } + }); + } } + refCountedWriterMap.clear(); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java index aa3a6aaeff29f..41fecb3b87ed4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java @@ -38,7 +38,7 @@ * Topic policies service. */ @InterfaceStability.Evolving -public interface TopicPoliciesService { +public interface TopicPoliciesService extends AutoCloseable { TopicPoliciesService DISABLED = new TopicPoliciesServiceDisabled(); long DEFAULT_GET_TOPIC_POLICY_TIMEOUT = 30_000; @@ -239,5 +239,10 @@ public void registerListener(TopicName topicName, TopicPolicyListener listener) { //No-op } + + @Override + public void close() { + //No-op + } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java index 087c5f932008f..51509f3818a28 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java @@ -67,7 +67,7 @@ protected String getProducerName() { } @Override - protected void readEntries(Producer producer) { + protected void setProducerAndTriggerReadEntries(Producer producer) { this.producer = (ProducerImpl) producer; if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Started)) { @@ -78,8 +78,7 @@ protected void readEntries(Producer producer) { "[{}] Replicator was stopped while creating the producer." + " Closing it. Replicator state: {}", replicatorId, STATE_UPDATER.get(this)); - STATE_UPDATER.set(this, State.Stopping); - closeProducerAsync(); + doCloseProducerAsync(producer, () -> {}); return; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java index 2cad253f96ee2..fb7bd22de94a7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java @@ -126,6 +126,14 @@ protected Map> initialValue() throws Exception { } }; + private static final FastThreadLocal>> localGroupedStickyKeyHashes = + new FastThreadLocal>>() { + @Override + protected Map> initialValue() throws Exception { + return new HashMap<>(); + } + }; + @Override public void sendMessages(List entries) { if (entries.isEmpty()) { @@ -139,28 +147,38 @@ public void sendMessages(List entries) { final Map> groupedEntries = localGroupedEntries.get(); groupedEntries.clear(); + final Map> consumerStickyKeyHashesMap = localGroupedStickyKeyHashes.get(); + consumerStickyKeyHashesMap.clear(); for (Entry entry : entries) { - Consumer consumer = selector.select(peekStickyKey(entry.getDataBuffer())); + byte[] stickyKey = peekStickyKey(entry.getDataBuffer()); + int stickyKeyHash = StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey); + + Consumer consumer = selector.select(stickyKeyHash); if (consumer != null) { - groupedEntries.computeIfAbsent(consumer, k -> new ArrayList<>()).add(entry); + int startingSize = Math.max(10, entries.size() / (2 * consumerSet.size())); + groupedEntries.computeIfAbsent(consumer, k -> new ArrayList<>(startingSize)).add(entry); + consumerStickyKeyHashesMap + .computeIfAbsent(consumer, k -> new ArrayList<>(startingSize)).add(stickyKeyHash); } else { entry.release(); } } for (Map.Entry> entriesByConsumer : groupedEntries.entrySet()) { - Consumer consumer = entriesByConsumer.getKey(); - List entriesForConsumer = entriesByConsumer.getValue(); + final Consumer consumer = entriesByConsumer.getKey(); + final List entriesForConsumer = entriesByConsumer.getValue(); + final List stickyKeysForConsumer = consumerStickyKeyHashesMap.get(consumer); SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); EntryBatchSizes batchSizes = EntryBatchSizes.get(entriesForConsumer.size()); filterEntriesForConsumer(entriesForConsumer, batchSizes, sendMessageInfo, null, null, false, consumer); if (consumer.getAvailablePermits() > 0 && consumer.isWritable()) { - consumer.sendMessages(entriesForConsumer, batchSizes, null, sendMessageInfo.getTotalMessages(), + consumer.sendMessages(entriesForConsumer, stickyKeysForConsumer, batchSizes, + null, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), - getRedeliveryTracker()); + getRedeliveryTracker(), Commands.DEFAULT_CONSUMER_EPOCH); TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -sendMessageInfo.getTotalMessages()); } else { entriesForConsumer.forEach(e -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 9a3a0a7d83d50..8cb8394440f33 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -420,7 +420,7 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, boolean c CompletableFuture closeClientFuture = new CompletableFuture<>(); if (closeIfClientsConnected) { List> futures = new ArrayList<>(); - replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); + replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); producers.values().forEach(producer -> futures.add(producer.disconnect())); subscriptions.forEach((s, sub) -> futures.add(sub.close(true, Optional.empty()))); FutureUtil.waitForAll(futures).thenRun(() -> { @@ -477,7 +477,8 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, boolean c } }).exceptionally(ex -> { deleteFuture.completeExceptionally( - new TopicBusyException("Failed to close clients before deleting topic.")); + new TopicBusyException("Failed to close clients before deleting topic.", + FutureUtil.unwrapCompletionException(ex))); return null; }); } finally { @@ -523,7 +524,7 @@ public CompletableFuture close( List> futures = new ArrayList<>(); - replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); + replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); if (disconnectClients) { futures.add(ExtensibleLoadManagerImpl.getAssignedBrokerLookupData( brokerService.getPulsar(), topic).thenAccept(lookupData -> { @@ -582,7 +583,7 @@ public CompletableFuture close( public CompletableFuture stopReplProducers() { List> closeFutures = new ArrayList<>(); - replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect())); + replicators.forEach((region, replicator) -> closeFutures.add(replicator.terminate())); return FutureUtil.waitForAll(closeFutures); } @@ -663,7 +664,7 @@ CompletableFuture removeReplicator(String remoteCluster) { String name = NonPersistentReplicator.getReplicatorName(replicatorPrefix, remoteCluster); - replicators.get(remoteCluster).disconnect().thenRun(() -> { + replicators.get(remoteCluster).terminate().thenRun(() -> { log.info("[{}] Successfully removed replicator {}", name, remoteCluster); replicators.remove(remoteCluster); @@ -1032,7 +1033,7 @@ private CompletableFuture disconnectReplicators() { List> futures = new ArrayList<>(); ConcurrentOpenHashMap replicators = getReplicators(); replicators.forEach((r, replicator) -> { - futures.add(replicator.disconnect()); + futures.add(replicator.terminate()); }); return FutureUtil.waitForAll(futures); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index 754d25b8b0ab4..367d19652072d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -18,6 +18,10 @@ */ package org.apache.pulsar.broker.service.persistent; +import static org.apache.pulsar.broker.service.AbstractReplicator.State.Started; +import static org.apache.pulsar.broker.service.AbstractReplicator.State.Starting; +import static org.apache.pulsar.broker.service.AbstractReplicator.State.Terminated; +import static org.apache.pulsar.broker.service.AbstractReplicator.State.Terminating; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; @@ -26,7 +30,6 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; @@ -43,10 +46,10 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.AbstractReplicator; import org.apache.pulsar.broker.service.BrokerService; -import org.apache.pulsar.broker.service.BrokerServiceException.TopicBusyException; import org.apache.pulsar.broker.service.MessageExpirer; import org.apache.pulsar.broker.service.Replicator; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; @@ -134,30 +137,44 @@ public PersistentReplicator(String localCluster, PersistentTopic localTopic, Man } @Override - protected void readEntries(Producer producer) { - // Rewind the cursor to be sure to read again all non-acked messages sent while restarting + protected void setProducerAndTriggerReadEntries(Producer producer) { + // Rewind the cursor to be sure to read again all non-acked messages sent while restarting. cursor.rewind(); - cursor.cancelPendingReadRequest(); - HAVE_PENDING_READ_UPDATER.set(this, FALSE); - this.producer = (ProducerImpl) producer; - if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Started)) { - log.info("[{}] Created replicator producer", replicatorId); + /** + * 1. Try change state to {@link Started}. + * 2. Atoms modify multiple properties if change state success, to avoid another thread get a null value + * producer when the state is {@link Started}. + */ + Pair changeStateRes; + changeStateRes = compareSetAndGetState(Starting, Started); + if (changeStateRes.getLeft()) { + this.producer = (ProducerImpl) producer; + HAVE_PENDING_READ_UPDATER.set(this, FALSE); + // Trigger a new read. + log.info("[{}] Created replicator producer, Replicator state: {}", replicatorId, state); backOff.reset(); - // activate cursor: so, entries can be cached + // activate cursor: so, entries can be cached. this.cursor.setActive(); // read entries readMoreEntries(); } else { - log.info( - "[{}] Replicator was stopped while creating the producer." - + " Closing it. Replicator state: {}", - replicatorId, STATE_UPDATER.get(this)); - STATE_UPDATER.set(this, State.Stopping); - closeProducerAsync(); + if (changeStateRes.getRight() == Started) { + // Since only one task can call "producerBuilder.createAsync()", this scenario is not expected. + // So print a warn log. + log.warn("[{}] Replicator was already started by another thread while creating the producer." + + " Closing the producer newly created. Replicator state: {}", replicatorId, state); + } else if (changeStateRes.getRight() == Terminating || changeStateRes.getRight() == Terminated) { + log.info("[{}] Replicator was terminated, so close the producer. Replicator state: {}", + replicatorId, state); + } else { + log.error("[{}] Replicator state is not expected, so close the producer. Replicator state: {}", + replicatorId, changeStateRes.getRight()); + } + // Close the producer if change the state fail. + doCloseProducerAsync(producer, () -> {}); } - } @Override @@ -420,8 +437,8 @@ public CompletableFuture getFuture() { @Override public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - if (STATE_UPDATER.get(this) != State.Started) { - log.info("[{}] Replicator was stopped while reading entries." + if (state != Started) { + log.info("[{}] Replicator was disconnected while reading entries." + " Stop reading. Replicator state: {}", replicatorId, STATE_UPDATER.get(this)); return; @@ -433,11 +450,11 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { long waitTimeMillis = readFailureBackoff.next(); if (exception instanceof CursorAlreadyClosedException) { - log.error("[{}] Error reading entries because replicator is" + log.warn("[{}] Error reading entries because replicator is" + " already deleted and cursor is already closed {}, ({})", replicatorId, ctx, exception.getMessage(), exception); - // replicator is already deleted and cursor is already closed so, producer should also be stopped - closeProducerAsync(); + // replicator is already deleted and cursor is already closed so, producer should also be disconnected. + terminate(); return; } else if (!(exception instanceof TooManyRequestsException)) { log.error("[{}] Error reading entries at {}. Retrying to read in {}s. ({})", @@ -553,10 +570,10 @@ public void deleteFailed(ManagedLedgerException exception, Object ctx) { log.error("[{}] Failed to delete message at {}: {}", replicatorId, ctx, exception.getMessage(), exception); if (exception instanceof CursorAlreadyClosedException) { - log.error("[{}] Asynchronous ack failure because replicator is already deleted and cursor is already" + log.warn("[{}] Asynchronous ack failure because replicator is already deleted and cursor is already" + " closed {}, ({})", replicatorId, ctx, exception.getMessage(), exception); - // replicator is already deleted and cursor is already closed so, producer should also be stopped - closeProducerAsync(); + // replicator is already deleted and cursor is already closed so, producer should also be disconnected. + terminate(); return; } if (ctx instanceof PositionImpl) { @@ -675,36 +692,17 @@ protected void checkReplicatedSubscriptionMarker(Position position, MessageImpl< } } - @Override - public CompletableFuture disconnect() { - return disconnect(false); - } - - @Override - public synchronized CompletableFuture disconnect(boolean failIfHasBacklog) { - final CompletableFuture future = new CompletableFuture<>(); - - super.disconnect(failIfHasBacklog).thenRun(() -> { - dispatchRateLimiter.ifPresent(DispatchRateLimiter::close); - future.complete(null); - }).exceptionally(ex -> { - Throwable t = (ex instanceof CompletionException ? ex.getCause() : ex); - if (!(t instanceof TopicBusyException)) { - log.error("[{}] Failed to close dispatch rate limiter: {}", replicatorId, ex.getMessage()); - } - future.completeExceptionally(t); - return null; - }); - - return future; - } - @Override public boolean isConnected() { ProducerImpl producer = this.producer; return producer != null && producer.isConnected(); } + @Override + protected void doReleaseResources() { + dispatchRateLimiter.ifPresent(DispatchRateLimiter::close); + } + private static final Logger log = LoggerFactory.getLogger(PersistentReplicator.class); @VisibleForTesting diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 936091edce557..155b67778820b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -45,6 +45,7 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -833,15 +834,15 @@ public CompletableFuture startReplProducers() { public CompletableFuture stopReplProducers() { List> closeFutures = new ArrayList<>(); - replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect())); - shadowReplicators.forEach((__, replicator) -> closeFutures.add(replicator.disconnect())); + replicators.forEach((region, replicator) -> closeFutures.add(replicator.terminate())); + shadowReplicators.forEach((__, replicator) -> closeFutures.add(replicator.terminate())); return FutureUtil.waitForAll(closeFutures); } private synchronized CompletableFuture closeReplProducersIfNoBacklog() { List> closeFutures = new ArrayList<>(); - replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect(true))); - shadowReplicators.forEach((__, replicator) -> closeFutures.add(replicator.disconnect(true))); + replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect(true, true))); + shadowReplicators.forEach((__, replicator) -> closeFutures.add(replicator.disconnect(true, true))); return FutureUtil.waitForAll(closeFutures); } @@ -1423,13 +1424,20 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, List> futures = new ArrayList<>(); subscriptions.forEach((s, sub) -> futures.add(sub.close(true, Optional.empty()))); if (closeIfClientsConnected) { - replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); - shadowReplicators.forEach((__, replicator) -> futures.add(replicator.disconnect())); + replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); + shadowReplicators.forEach((__, replicator) -> futures.add(replicator.terminate())); producers.values().forEach(producer -> futures.add(producer.disconnect())); } FutureUtil.waitForAll(futures).thenRunAsync(() -> { closeClientFuture.complete(null); - }, getOrderedExecutor()).exceptionally(ex -> { + }, command -> { + try { + getOrderedExecutor().execute(command); + } catch (RejectedExecutionException e) { + // executor has been shut down, execute in current thread + command.run(); + } + }).exceptionally(ex -> { log.error("[{}] Error closing clients", topic, ex); unfenceTopicToResume(); closeClientFuture.completeExceptionally(ex); @@ -1506,7 +1514,8 @@ public void deleteLedgerComplete(Object ctx) { }).exceptionally(ex->{ unfenceTopicToResume(); deleteFuture.completeExceptionally( - new TopicBusyException("Failed to close clients before deleting topic.")); + new TopicBusyException("Failed to close clients before deleting topic.", + FutureUtil.unwrapCompletionException(ex))); return null; }); @@ -1565,8 +1574,8 @@ public CompletableFuture close( List> futures = new ArrayList<>(); futures.add(transactionBuffer.closeAsync()); - replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); - shadowReplicators.forEach((__, replicator) -> futures.add(replicator.disconnect())); + replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); + shadowReplicators.forEach((__, replicator) -> futures.add(replicator.terminate())); if (disconnectClients) { futures.add(ExtensibleLoadManagerImpl.getAssignedBrokerLookupData( brokerService.getPulsar(), topic).thenAccept(lookupData -> { @@ -1731,6 +1740,7 @@ public CompletableFuture checkReplication() { return deleteForcefully(); } + removeTerminatedReplicators(replicators); List> futures = new ArrayList<>(); // Check for missing replicators @@ -1769,6 +1779,8 @@ private CompletableFuture checkShadowReplication() { if (log.isDebugEnabled()) { log.debug("[{}] Checking shadow replication status, shadowTopics={}", topic, configuredShadowTopics); } + + removeTerminatedReplicators(shadowReplicators); List> futures = new ArrayList<>(); // Check for missing replicators @@ -1919,19 +1931,30 @@ protected CompletableFuture addReplicationCluster(String remoteCluster, Ma if (replicationClient == null) { return; } - Replicator replicator = replicators.computeIfAbsent(remoteCluster, r -> { - try { - return new GeoPersistentReplicator(PersistentTopic.this, cursor, localCluster, - remoteCluster, brokerService, (PulsarClientImpl) replicationClient); - } catch (PulsarServerException e) { - log.error("[{}] Replicator startup failed {}", topic, remoteCluster, e); + lock.readLock().lock(); + try { + if (isClosingOrDeleting) { + // Whether is "transferring" or not, do not create new replicator. + log.info("[{}] Skip to create replicator because this topic is closing." + + " remote cluster: {}. State of transferring : {}", + topic, remoteCluster, transferring); + return; } - return null; - }); - - // clean up replicator if startup is failed - if (replicator == null) { - replicators.removeNullValue(remoteCluster); + Replicator replicator = replicators.computeIfAbsent(remoteCluster, r -> { + try { + return new GeoPersistentReplicator(PersistentTopic.this, cursor, localCluster, + remoteCluster, brokerService, (PulsarClientImpl) replicationClient); + } catch (PulsarServerException e) { + log.error("[{}] Replicator startup failed {}", topic, remoteCluster, e); + } + return null; + }); + // clean up replicator if startup is failed + if (replicator == null) { + replicators.removeNullValue(remoteCluster); + } + } finally { + lock.readLock().unlock(); } }); } @@ -1942,7 +1965,7 @@ CompletableFuture removeReplicator(String remoteCluster) { String name = PersistentReplicator.getReplicatorName(replicatorPrefix, remoteCluster); - Optional.ofNullable(replicators.get(remoteCluster)).map(Replicator::disconnect) + Optional.ofNullable(replicators.get(remoteCluster)).map(Replicator::terminate) .orElse(CompletableFuture.completedFuture(null)).thenRun(() -> { ledger.asyncDeleteCursor(name, new DeleteCursorCallback() { @Override @@ -2014,7 +2037,7 @@ CompletableFuture removeShadowReplicator(String shadowTopic) { log.info("[{}] Removing shadow topic replicator to {}", topic, shadowTopic); final CompletableFuture future = new CompletableFuture<>(); String name = ShadowReplicator.getShadowReplicatorName(replicatorPrefix, shadowTopic); - shadowReplicators.get(shadowTopic).disconnect().thenRun(() -> { + shadowReplicators.get(shadowTopic).terminate().thenRun(() -> { ledger.asyncDeleteCursor(name, new DeleteCursorCallback() { @Override @@ -2898,7 +2921,7 @@ private CompletableFuture checkAndDisconnectReplicators() { ConcurrentOpenHashMap replicators = getReplicators(); replicators.forEach((r, replicator) -> { if (replicator.getNumberOfEntriesInBacklog() <= 0) { - futures.add(replicator.disconnect()); + futures.add(replicator.terminate()); } }); return FutureUtil.waitForAll(futures); @@ -2949,6 +2972,15 @@ public void checkGC() { log.debug("[{}] Global topic inactive for {} seconds, closing repl producers.", topic, maxInactiveDurationInSec); } + /** + * There is a race condition that may cause a NPE: + * - task 1: a callback of "replicator.cursor.asyncRead" will trigger a replication. + * - task 2: "closeReplProducersIfNoBacklog" called by current thread will make the variable + * "replicator.producer" to a null value. + * Race condition: task 1 will get a NPE when it tries to send messages using the variable + * "replicator.producer", because task 2 will set this variable to "null". + * TODO Create a seperated PR to fix it. + */ closeReplProducersIfNoBacklog().thenRun(() -> { if (hasRemoteProducers()) { if (log.isDebugEnabled()) { @@ -3872,9 +3904,27 @@ private void fenceTopicToCloseOrDelete() { } private void unfenceTopicToResume() { - subscriptions.values().forEach(sub -> sub.resumeAfterFence()); isFenced = false; isClosingOrDeleting = false; + subscriptions.values().forEach(sub -> sub.resumeAfterFence()); + unfenceReplicatorsToResume(); + } + + private void unfenceReplicatorsToResume() { + checkReplication(); + checkShadowReplication(); + } + + private void removeTerminatedReplicators(ConcurrentOpenHashMap replicators) { + Map terminatedReplicators = new HashMap<>(); + replicators.forEach((cluster, replicator) -> { + if (replicator.isTerminated()) { + terminatedReplicators.put(cluster, replicator); + } + }); + terminatedReplicators.entrySet().forEach(entry -> { + replicators.remove(entry.getKey(), entry.getValue()); + }); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java index 8cd68caf1ee26..6b4d08c359d42 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java @@ -24,6 +24,7 @@ import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.PooledByteBufAllocator; +import io.netty.buffer.Unpooled; import io.prometheus.client.Collector; import java.io.BufferedOutputStream; import java.io.IOException; @@ -191,8 +192,8 @@ private static class GzipByteBufferWriter { crc = new CRC32(); this.bufferSize = Math.max(Math.min(resolveChunkSize(bufAllocator), readableBytes), 8192); this.bufAllocator = bufAllocator; - this.resultBuffer = bufAllocator.compositeDirectBuffer(readableBytes / bufferSize + 1); - allocateBuffer(); + this.resultBuffer = bufAllocator.compositeDirectBuffer(readableBytes / bufferSize + 2); + allocateCompressBuffer(); } /** @@ -217,37 +218,66 @@ private void compressAndAppend(ByteBuffer nioBuffer, boolean isFirst, boolean is // write gzip header compressBuffer.put(GZIP_HEADER); } + // update the CRC32 checksum calculation nioBuffer.mark(); crc.update(nioBuffer); nioBuffer.reset(); + // pass the input buffer to the deflater deflater.setInput(nioBuffer); + // when the input buffer is the last one, set the flag to finish the deflater if (isLast) { deflater.finish(); } - while (!deflater.needsInput() && !deflater.finished()) { - int written = deflater.deflate(compressBuffer); - if (written == 0 && !compressBuffer.hasRemaining()) { - backingCompressBuffer.setIndex(0, compressBuffer.position()); - resultBuffer.addComponent(true, backingCompressBuffer); - allocateBuffer(); + int written = -1; + // the deflater may need multiple calls to deflate the input buffer + // the completion is checked by the deflater.needsInput() method for buffers that aren't the last buffer + // for the last buffer, the completion is checked by the deflater.finished() method + while (!isLast && !deflater.needsInput() || isLast && !deflater.finished()) { + // when the previous deflater.deflate call returns 0 (and needsInput/finished returns false), + // it means that the output buffer is full. + // append the compressed buffer to the result buffer and allocate a new buffer. + if (written == 0) { + if (compressBuffer.position() > 0) { + appendCompressBufferToResultBuffer(); + allocateCompressBuffer(); + } else { + // this is an unexpected case, throw an exception to prevent an infinite loop + throw new IllegalStateException( + "Deflater didn't write any bytes while the compress buffer is empty."); + } } + written = deflater.deflate(compressBuffer); } if (isLast) { - // write gzip footer, integer values are in little endian byte order - compressBuffer.order(ByteOrder.LITTLE_ENDIAN); - // write CRC32 checksum - compressBuffer.putInt((int) crc.getValue()); - // write uncompressed size - compressBuffer.putInt(deflater.getTotalIn()); - // append the last compressed buffer - backingCompressBuffer.setIndex(0, compressBuffer.position()); - resultBuffer.addComponent(true, backingCompressBuffer); + // append the last compressed buffer when it is not empty + if (compressBuffer.position() > 0) { + appendCompressBufferToResultBuffer(); + } else { + // release an unused empty buffer + backingCompressBuffer.release(); + } backingCompressBuffer = null; compressBuffer = null; + + // write gzip trailer, 2 integers (CRC32 checksum and uncompressed size) + ByteBuffer trailerBuf = ByteBuffer.allocate(2 * Integer.BYTES); + // integer values are in little endian byte order + trailerBuf.order(ByteOrder.LITTLE_ENDIAN); + // write CRC32 checksum + trailerBuf.putInt((int) crc.getValue()); + // write uncompressed size + trailerBuf.putInt(deflater.getTotalIn()); + trailerBuf.flip(); + resultBuffer.addComponent(true, Unpooled.wrappedBuffer(trailerBuf)); } } - private void allocateBuffer() { + private void appendCompressBufferToResultBuffer() { + backingCompressBuffer.setIndex(0, compressBuffer.position()); + resultBuffer.addComponent(true, backingCompressBuffer); + } + + private void allocateCompressBuffer() { backingCompressBuffer = bufAllocator.directBuffer(bufferSize); compressBuffer = backingCompressBuffer.nioBuffer(0, bufferSize); } @@ -282,7 +312,7 @@ public PrometheusMetricsGenerator(PulsarService pulsar, boolean includeTopicMetr this.clock = clock; } - private ByteBuf generate0(List metricsProviders) { + protected ByteBuf generateMetrics(List metricsProviders) { ByteBuf buf = allocateMultipartCompositeDirectBuffer(); boolean exceptionHappens = false; //Used in namespace/topic and transaction aggregators as share metric names @@ -342,7 +372,9 @@ private ByteBuf allocateMultipartCompositeDirectBuffer() { int totalLen = 0; while (totalLen < initialBufferSize) { totalLen += chunkSize; - buf.addComponent(false, byteBufAllocator.directBuffer(chunkSize)); + // increase the capacity in increments of chunkSize to preallocate the buffers + // in the composite buffer + buf.capacity(totalLen); } return buf; } @@ -492,7 +524,7 @@ public MetricsBuffer renderToBuffer(Executor executor, List bufferFuture = newMetricsBuffer.getBufferFuture(); executor.execute(() -> { try { - bufferFuture.complete(new ResponseBuffer(generate0(metricsProviders))); + bufferFuture.complete(new ResponseBuffer(generateMetrics(metricsProviders))); } catch (Exception e) { bufferFuture.completeExceptionally(e); } finally { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java index 8dc36e2917ed1..9a439268a8b4f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java @@ -31,12 +31,18 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.RequestLog; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandler; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; @@ -103,9 +109,18 @@ public WebService(PulsarService pulsar) throws PulsarServerException { Optional port = config.getWebServicePort(); HttpConfiguration httpConfig = new HttpConfiguration(); + if (config.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } httpConfig.setRequestHeaderSize(pulsar.getConfig().getHttpMaxRequestHeaderSize()); + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); if (port.isPresent()) { - httpConnector = new ServerConnector(server, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + httpConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpConnector.setPort(port.get()); httpConnector.setHost(pulsar.getBindAddress()); connectors.add(httpConnector); @@ -144,7 +159,18 @@ public WebService(PulsarService pulsar) throws PulsarServerException { config.getWebServiceTlsProtocols(), config.getTlsCertRefreshCheckDurationSec()); } - httpsConnector = new ServerConnector(server, sslCtxFactory, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + httpsConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpsConnector.setPort(tlsPort.get()); httpsConnector.setHost(pulsar.getBindAddress()); connectors.add(httpsConnector); @@ -284,7 +310,12 @@ public void addStaticResources(String basePath, String resourcePath) { public void start() throws PulsarServerException { try { RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = pulsar.getConfiguration().getWebServiceLogDetailedAddresses() != null + ? pulsar.getConfiguration().getWebServiceLogDetailedAddresses() : + (pulsar.getConfiguration().isWebServiceHaProxyProtocolEnabled() + || pulsar.getConfiguration().isWebServiceTrustXForwardedFor()); + RequestLog requestLogger = JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server); + requestLogHandler.setRequestLog(requestLogger); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/MultiBrokerTestZKBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/MultiBrokerTestZKBaseTest.java index e5b80c0af33ab..a78254df4aae0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/MultiBrokerTestZKBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/MultiBrokerTestZKBaseTest.java @@ -59,6 +59,7 @@ protected void onCleanup() { } catch (Exception e) { log.error("Error in stopping ZK server", e); } + testZKServer = null; } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java index 4a6524bf24521..941229fc3d96c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java @@ -126,15 +126,26 @@ private void createTenant(PulsarAdmin pulsarAdmin) @AfterClass(alwaysRun = true) public void shutdown() throws Exception { log.info("--- Shutting down ---"); - executor.shutdownNow(); - executor = null; + if (executor != null) { + executor.shutdownNow(); + executor = null; + } for (int i = 0; i < BROKER_COUNT; i++) { - pulsarAdmins[i].close(); - pulsarServices[i].close(); + if (pulsarAdmins[i] != null) { + pulsarAdmins[i].close(); + pulsarAdmins[i] = null; + } + if (pulsarServices[i] != null) { + pulsarServices[i].close(); + pulsarServices[i] = null; + } } - bkEnsemble.stop(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaWithAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaWithAuthTest.java index e89b4ff5e83ec..2dcb930fbe719 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaWithAuthTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaWithAuthTest.java @@ -120,6 +120,7 @@ public void testGetCreateDeleteSchema() throws Exception { .serviceHttpUrl(brokerUrl != null ? brokerUrl.toString() : brokerUrlTls.toString()) .authentication(AuthenticationToken.class.getName(), PRODUCE_TOKEN) .build(); + admin.topics().createNonPartitionedTopic(topicName); admin.topics().grantPermission(topicName, "consumer", EnumSet.of(AuthAction.consume)); admin.topics().grantPermission(topicName, "producer", EnumSet.of(AuthAction.produce)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index b28cfc98fdb07..635b2c25bc1d0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -3698,4 +3698,16 @@ public void testRetentionAndBacklogQuotaCheck() throws PulsarAdminException { }); } + + @Test + @SneakyThrows + public void testPermissions() { + String namespace = "prop-xyz/ns1/"; + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://" + namespace + random; + final String subject = UUID.randomUUID().toString(); + assertThrows(NotFoundException.class, () -> admin.topics().getPermissions(topic)); + assertThrows(NotFoundException.class, () -> admin.topics().grantPermission(topic, subject, Set.of(AuthAction.produce))); + assertThrows(NotFoundException.class, () -> admin.topics().revokePermissions(topic, subject)); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java new file mode 100644 index 0000000000000..a710a03970d06 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.admin; + +import io.jsonwebtoken.Jwts; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.TopicOperation; +import org.apache.pulsar.security.MockedPulsarStandalone; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import static org.mockito.Mockito.doReturn; + +public class AuthZTest extends MockedPulsarStandalone { + + protected PulsarAdmin superUserAdmin; + + protected PulsarAdmin tenantManagerAdmin; + + protected AuthorizationService authorizationService; + + protected AuthorizationService orignalAuthorizationService; + + protected static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); + protected static final String TENANT_ADMIN_TOKEN = Jwts.builder() + .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); + + @BeforeMethod(alwaysRun = true) + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + authorizationService = Mockito.spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + authorizationService, true); + } + + @AfterMethod(alwaysRun = true) + public void after() throws IllegalAccessException { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + orignalAuthorizationService, true); + } + + protected AtomicBoolean setAuthorizationTopicOperationChecker(String role, Object operation) { + AtomicBoolean execFlag = new AtomicBoolean(false); + if (operation instanceof TopicOperation) { + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(2); + if (role.equals(role_)) { + TopicOperation operation_ = invocationOnMock.getArgument(1); + Assert.assertEquals(operation_, operation); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowTopicOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any()); + } else if (operation instanceof NamespaceOperation) { + doReturn(true) + .when(authorizationService).isValidOriginalPrincipal(Mockito.any(), Mockito.any(), Mockito.any()); + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(2); + if (role.equals(role_)) { + TopicOperation operation_ = invocationOnMock.getArgument(1); + Assert.assertEquals(operation_, operation); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowNamespaceOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any()); + } else { + throw new IllegalArgumentException(""); + } + + + return execFlag; + } + + protected void createTopic(String topic, boolean partitioned) throws Exception { + if (partitioned) { + superUserAdmin.topics().createPartitionedTopic(topic, 2); + } else { + superUserAdmin.topics().createNonPartitionedTopic(topic); + } + } + + protected void deleteTopic(String topic, boolean partitioned) throws Exception { + if (partitioned) { + superUserAdmin.topics().deletePartitionedTopic(topic, true); + } else { + superUserAdmin.topics().delete(topic, true); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java index d5a0468f340c9..5358295b78568 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.admin; +import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.deleteNamespaceWithRetry; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -58,7 +59,6 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @Test(groups = "broker-admin") @@ -72,8 +72,6 @@ public class NamespaceAuthZTest extends MockedPulsarStandalone { private AuthorizationService authorizationService; - private AuthorizationService orignalAuthorizationService; - private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); private static final String TENANT_ADMIN_TOKEN = Jwts.builder() .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); @@ -100,6 +98,9 @@ public void setup() { .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) .build(); this.pulsarClient = super.getPulsarService().getClient(); + this.authorizationService = Mockito.spy(getPulsarService().getBrokerService().getAuthorizationService()); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + authorizationService, true); } @@ -115,19 +116,9 @@ public void cleanup() { close(); } - @BeforeMethod - public void before() throws IllegalAccessException { - orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); - authorizationService = Mockito.spy(orignalAuthorizationService); - FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", - authorizationService, true); - } - @AfterMethod - public void after() throws IllegalAccessException, PulsarAdminException { - FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", - orignalAuthorizationService, true); - superUserAdmin.namespaces().deleteNamespace("public/default", true); + public void after() throws Exception { + deleteNamespaceWithRetry("public/default", true, superUserAdmin); superUserAdmin.namespaces().createNamespace("public/default"); } @@ -1028,4 +1019,43 @@ public void testPackageAPI() throws Exception { superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); } } + + @Test + @SneakyThrows + public void testOffloadThresholdInSeconds() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getOffloadThresholdInSeconds(namespace)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setOffloadThresholdInSeconds(namespace, 10000)); + } + + @Test + @SneakyThrows + public void testMaxSubscriptionsPerTopic() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getMaxSubscriptionsPerTopic(namespace)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setMaxSubscriptionsPerTopic(namespace, 100)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeMaxSubscriptionsPerTopic(namespace)); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index c588051a0feff..55b4c6e1c6f59 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -890,12 +890,15 @@ public void testGetList() throws Exception { public void testGrantNonPartitionedTopic() { final String topicName = "non-partitioned-topic"; AsyncResponse response = mock(AsyncResponse.class); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); persistentTopics.createNonPartitionedTopic(response, testTenant, testNamespace, topicName, true, null); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); String role = "role"; Set expectActions = new HashSet<>(); expectActions.add(AuthAction.produce); response = mock(AsyncResponse.class); - ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); persistentTopics.grantPermissionsOnTopic(response, testTenant, testNamespace, topicName, role, expectActions); verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); @@ -957,12 +960,15 @@ public void testGrantPartitionedTopic() { public void testRevokeNonPartitionedTopic() { final String topicName = "non-partitioned-topic"; AsyncResponse response = mock(AsyncResponse.class); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); persistentTopics.createNonPartitionedTopic(response, testTenant, testNamespace, topicName, true, null); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); String role = "role"; Set expectActions = new HashSet<>(); expectActions.add(AuthAction.produce); response = mock(AsyncResponse.class); - ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); persistentTopics.grantPermissionsOnTopic(response, testTenant, testNamespace, topicName, role, expectActions); verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index 3c0596d531f41..ad47ac74a8980 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -20,6 +20,7 @@ package org.apache.pulsar.broker.admin; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import io.jsonwebtoken.Jwts; import lombok.Cleanup; import lombok.SneakyThrows; @@ -38,59 +39,48 @@ import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.AutoSubscriptionCreationOverride; +import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; +import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.EntryFilters; +import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; +import org.apache.pulsar.common.policies.data.OffloadPolicies; +import org.apache.pulsar.common.policies.data.PersistencePolicies; +import org.apache.pulsar.common.policies.data.PolicyName; +import org.apache.pulsar.common.policies.data.PolicyOperation; +import org.apache.pulsar.common.policies.data.PublishRate; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.TenantInfo; -import org.apache.pulsar.security.MockedPulsarStandalone; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.UUID; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.pulsar.broker.authorization.AuthorizationService; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.transaction.Transaction; -import org.apache.pulsar.common.naming.SystemTopicNames; -import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.TopicOperation; -import org.apache.pulsar.common.schema.SchemaInfo; -import org.apache.pulsar.common.schema.SchemaType; -import org.apache.pulsar.metadata.api.MetadataStoreException; import org.mockito.Mockito; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; +import static org.mockito.Mockito.doReturn; @Test(groups = "broker-admin") -public class TopicAuthZTest extends MockedPulsarStandalone { - - private PulsarAdmin superUserAdmin; - - private PulsarAdmin tenantManagerAdmin; - - private AuthorizationService authorizationService; - - private AuthorizationService orignalAuthorizationService; - - private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); - private static final String TENANT_ADMIN_TOKEN = Jwts.builder() - .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); +public class TopicAuthZTest extends AuthZTest { @SneakyThrows @BeforeClass(alwaysRun = true) public void setup() { configureTokenAuthentication(); configureDefaultAuthorization(); - enableTransaction(); start(); - createTransactionCoordinatorAssign(16); this.superUserAdmin =PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) @@ -103,16 +93,6 @@ public void setup() { .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) .build(); - superUserAdmin.tenants().createTenant("pulsar", tenantInfo); - superUserAdmin.namespaces().createNamespace("pulsar/system"); - } - - protected void createTransactionCoordinatorAssign(int numPartitionsOfTC) throws MetadataStoreException { - getPulsarService().getPulsarResources() - .getNamespaceResources() - .getPartitionedTopicResources() - .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, - new PartitionedTopicMetadata(numPartitionsOfTC)); } @SneakyThrows @@ -127,48 +107,28 @@ public void cleanup() { close(); } - @BeforeMethod - public void before() throws IllegalAccessException { - orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); - authorizationService = Mockito.spy(orignalAuthorizationService); - FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", - authorizationService, true); - } + private AtomicBoolean setAuthorizationPolicyOperationChecker(String role, Object policyName, Object operation) { + AtomicBoolean execFlag = new AtomicBoolean(false); + if (operation instanceof PolicyOperation ) { - @AfterMethod - public void after() throws IllegalAccessException { - FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", - orignalAuthorizationService, true); - } + doReturn(true) + .when(authorizationService).isValidOriginalPrincipal(Mockito.any(), Mockito.any(), Mockito.any()); - private AtomicBoolean setAuthorizationTopicOperationChecker(String role, Object operation) { - AtomicBoolean execFlag = new AtomicBoolean(false); - if (operation instanceof TopicOperation) { - Mockito.doAnswer(invocationOnMock -> { - String role_ = invocationOnMock.getArgument(2); - if (role.equals(role_)) { - TopicOperation operation_ = invocationOnMock.getArgument(1); - Assert.assertEquals(operation_, operation); - } - execFlag.set(true); - return invocationOnMock.callRealMethod(); - }).when(authorizationService).allowTopicOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), - Mockito.any(), Mockito.any()); - } else if (operation instanceof NamespaceOperation) { Mockito.doAnswer(invocationOnMock -> { - String role_ = invocationOnMock.getArgument(2); - if (role.equals(role_)) { - TopicOperation operation_ = invocationOnMock.getArgument(1); - Assert.assertEquals(operation_, operation); - } - execFlag.set(true); - return invocationOnMock.callRealMethod(); - }).when(authorizationService).allowNamespaceOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), - Mockito.any(), Mockito.any()); + String role_ = invocationOnMock.getArgument(4); + if (role.equals(role_)) { + PolicyName policyName_ = invocationOnMock.getArgument(1); + PolicyOperation operation_ = invocationOnMock.getArgument(2); + Assert.assertEquals(operation_, operation); + Assert.assertEquals(policyName_, policyName); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowTopicPolicyOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any(), Mockito.any()); } else { throw new IllegalArgumentException(""); } - return execFlag; } @@ -1213,171 +1173,8 @@ public void testExpireMessageByPosition() { deleteTopic(topic, false); } - public enum OperationAuthType { - Lookup, - Produce, - Consume, - AdminOrSuperUser, - NOAuth - } - private final String testTopic = "persistent://public/default/" + UUID.randomUUID().toString(); - @FunctionalInterface - public interface ThrowingBiConsumer { - void accept(T t) throws PulsarAdminException; - } - @DataProvider(name = "authFunction") - public Object[][] authFunction () throws Exception { - String sub = "my-sub"; - createTopic(testTopic, false); - @Cleanup final PulsarClient pulsarClient = PulsarClient.builder() - .serviceUrl(getPulsarService().getBrokerServiceUrl()) - .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) - .enableTransaction(true) - .build(); - @Cleanup final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(testTopic).create(); - - @Cleanup final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic(testTopic) - .subscriptionName(sub) - .subscribe(); - - Transaction transaction = pulsarClient.newTransaction().withTransactionTimeout(5, TimeUnit.MINUTES) - .build().get(); - MessageIdImpl messageId = (MessageIdImpl) producer.newMessage().value("test message").send(); - - consumer.acknowledgeAsync(messageId, transaction).get(); - - return new Object[][]{ - // SCHEMA - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo(testTopic), - OperationAuthType.Lookup - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo( - testTopic, 0), - OperationAuthType.Lookup - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().getAllSchemas( - testTopic), - OperationAuthType.Lookup - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().createSchema(testTopic, - SchemaInfo.builder().type(SchemaType.STRING).build()), - OperationAuthType.Produce - }, - // TODO: improve the authorization check for testCompatibility and deleteSchema - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().testCompatibility( - testTopic, SchemaInfo.builder().type(SchemaType.STRING).build()), - OperationAuthType.AdminOrSuperUser - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().deleteSchema( - testTopic), - OperationAuthType.AdminOrSuperUser - }, - - // TRANSACTION - - // Modify transaction coordinator - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .abortTransaction(transaction.getTxnID()), - OperationAuthType.AdminOrSuperUser - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .scaleTransactionCoordinators(17), - OperationAuthType.AdminOrSuperUser - }, - // TODO: fix authorization check of check transaction coordinator stats. - // Check transaction coordinator stats - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getCoordinatorInternalStats(1, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getCoordinatorStats(), - OperationAuthType.AdminOrSuperUser - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getSlowTransactionsByCoordinatorId(1, 5, TimeUnit.SECONDS), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionMetadata(transaction.getTxnID()), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .listTransactionCoordinators(), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getSlowTransactions(5, TimeUnit.SECONDS), - OperationAuthType.AdminOrSuperUser - }, - - // TODO: Check the authorization of the topic when get stats of TB or TP - // Check stats related to transaction buffer and transaction pending ack - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getPendingAckInternalStats(testTopic, sub, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getPendingAckStats(testTopic, sub, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getPositionStatsInPendingAck(testTopic, sub, messageId.getLedgerId(), - messageId.getEntryId(), null), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionBufferInternalStats(testTopic, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionBufferStats(testTopic, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionBufferStats(testTopic, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionInBufferStats(transaction.getTxnID(), testTopic), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionInBufferStats(transaction.getTxnID(), testTopic), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionInPendingAckStats(transaction.getTxnID(), testTopic, sub), - OperationAuthType.NOAuth - }, - }; - } @Test @SneakyThrows @@ -1410,82 +1207,7 @@ public void testSchemaCompatibility() { deleteTopic(topic, false); } - @Test(dataProvider = "authFunction") - public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer adminConsumer, OperationAuthType topicOpType) - throws Exception { - final String subject = UUID.randomUUID().toString(); - final String token = Jwts.builder() - .claim("sub", subject).signWith(SECRET_KEY).compact(); - - - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() - .serviceHttpUrl(getPulsarService().getWebServiceAddress()) - .authentication(new AuthenticationToken(token)) - .build(); - // test tenant manager - if (topicOpType != OperationAuthType.AdminOrSuperUser) { - adminConsumer.accept(tenantManagerAdmin); - } - - if (topicOpType != OperationAuthType.NOAuth) { - Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> adminConsumer.accept(subAdmin)); - } - - AtomicBoolean execFlag = null; - if (topicOpType == OperationAuthType.Lookup) { - execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.LOOKUP); - } else if (topicOpType == OperationAuthType.Produce) { - execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PRODUCE); - } else if (topicOpType == OperationAuthType.Consume) { - execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); - } - - for (AuthAction action : AuthAction.values()) { - superUserAdmin.topics().grantPermission(testTopic, subject, Set.of(action)); - - if (authActionMatchOperation(topicOpType, action)) { - adminConsumer.accept(subAdmin); - } else { - Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> adminConsumer.accept(subAdmin)); - } - superUserAdmin.topics().revokePermissions(testTopic, subject); - } - - if (execFlag != null) { - Assert.assertTrue(execFlag.get()); - } - - } - private boolean authActionMatchOperation(OperationAuthType operationAuthType, AuthAction action) { - switch (operationAuthType) { - case Lookup -> { - if (AuthAction.consume == action || AuthAction.produce == action) { - return true; - } - } - case Consume -> { - if (AuthAction.consume == action) { - return true; - } - } - case Produce -> { - if (AuthAction.produce == action) { - return true; - } - } - case AdminOrSuperUser -> { - return false; - } - case NOAuth -> { - return true; - } - } - return false; - } @Test @SneakyThrows @@ -1507,8 +1229,10 @@ public void testGetEntryFilter() { // test tenant manager tenantManagerAdmin.topicPolicies().getEntryFiltersPerTopic(topic, true); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ENTRY_FILTERS, PolicyOperation.READ); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topicPolicies().getEntryFiltersPerTopic(topic, false)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -1553,8 +1277,10 @@ public void testSetEntryFilter() { // test tenant manager tenantManagerAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -1656,19 +1382,768 @@ public void testShadowTopic() { deleteTopic(topic, false); } - private void createTopic(String topic, boolean partitioned) throws Exception { - if (partitioned) { - superUserAdmin.topics().createPartitionedTopic(topic, 2); - } else { - superUserAdmin.topics().createNonPartitionedTopic(topic); - } + @Test + @SneakyThrows + public void testList() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, NamespaceOperation.GET_TOPICS); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getList("public/default")); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationTopicOperationChecker(subject, NamespaceOperation.GET_TOPICS); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPartitionedTopicList("public/default")); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); } - private void deleteTopic(String topic, boolean partitioned) throws Exception { - if (partitioned) { - superUserAdmin.topics().deletePartitionedTopic(topic, true); - } else { - superUserAdmin.topics().delete(topic, true); - } + @Test + @SneakyThrows + public void testPermissionsOnTopic() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + superUserAdmin.topics().getPermissions(topic); + superUserAdmin.topics().grantPermission(topic, subject, Sets.newHashSet(AuthAction.functions)); + superUserAdmin.topics().revokePermissions(topic, subject); + + // test tenant manager + tenantManagerAdmin.topics().getPermissions(topic); + tenantManagerAdmin.topics().grantPermission(topic, subject, Sets.newHashSet(AuthAction.functions)); + tenantManagerAdmin.topics().revokePermissions(topic, subject); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPermissions(topic)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().grantPermission(topic, subject, Sets.newHashSet(AuthAction.functions))); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().revokePermissions(topic, subject)); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testOffloadPolicies() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getOffloadPolicies(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setOffloadPolicies(topic, OffloadPolicies.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeOffloadPolicies(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxUnackedMessagesOnConsumer() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxUnackedMessagesOnConsumer(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxUnackedMessagesOnConsumer(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxUnackedMessagesOnConsumer(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testDeduplicationSnapshotInterval() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getDeduplicationSnapshotInterval(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setDeduplicationSnapshotInterval(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeDeduplicationSnapshotInterval(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testInactiveTopicPolicies() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.INACTIVE_TOPIC, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getInactiveTopicPolicies(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.INACTIVE_TOPIC, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setInactiveTopicPolicies(topic, new InactiveTopicPolicies())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.INACTIVE_TOPIC, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeInactiveTopicPolicies(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxUnackedMessagesOnSubscription() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxUnackedMessagesOnSubscription(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxUnackedMessagesOnSubscription(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxUnackedMessagesOnSubscription(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testDelayedDeliveryPolicies() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DELAYED_DELIVERY, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getDelayedDeliveryPolicy(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DELAYED_DELIVERY, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setDelayedDeliveryPolicy(topic, DelayedDeliveryPolicies.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DELAYED_DELIVERY, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeDelayedDeliveryPolicy(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testAutoSubscriptionCreation() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_SUBSCRIPTION_CREATION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getAutoSubscriptionCreation(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_SUBSCRIPTION_CREATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setAutoSubscriptionCreation(topic, AutoSubscriptionCreationOverride.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_SUBSCRIPTION_CREATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeAutoSubscriptionCreation(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testSubscribeRate() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getSubscribeRate(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setSubscribeRate(topic, new SubscribeRate())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeSubscribeRate(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testSubscriptionTypesEnabled() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getSubscriptionTypesEnabled(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setSubscriptionTypesEnabled(topic, new HashSet<>())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeSubscriptionTypesEnabled(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testPublishRate() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getPublishRate(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setPublishRate(topic, new PublishRate())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removePublishRate(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxConsumersPerSubscription() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxConsumersPerSubscription(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxConsumersPerSubscription(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxConsumersPerSubscription(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testCompactionThreshold() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.COMPACTION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getCompactionThreshold(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.COMPACTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setCompactionThreshold(topic, 20000)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.COMPACTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeCompactionThreshold(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testDispatchRate() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getDispatchRate(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setDispatchRate(topic, DispatchRate.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeDispatchRate(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxConsumers() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxConsumers(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxConsumers(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxConsumers(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxProducers() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_PRODUCERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxProducers(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_PRODUCERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxProducers(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_PRODUCERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxProducers(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testReplicatorDispatchRate() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION_RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getReplicatorDispatchRate(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION_RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setReplicatorDispatchRate(topic, DispatchRate.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION_RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeReplicatorDispatchRate(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testPersistence() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.PERSISTENCE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getPersistence(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.PERSISTENCE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setPersistence(topic, new PersistencePolicies())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.PERSISTENCE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removePersistence(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testRetention() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RETENTION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getRetention(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RETENTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setRetention(topic, new RetentionPolicies())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RETENTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeRetention(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testDeduplication() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getDeduplicationStatus(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setDeduplicationStatus(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeDeduplicationStatus(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMessageTTL() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.TTL, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMessageTTL(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.TTL, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMessageTTL(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.TTL, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMessageTTL(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testBacklogQuota() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.BACKLOG, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getBacklogQuotaMap(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.BACKLOG, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setBacklogQuota(topic, BacklogQuota.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.BACKLOG, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeBacklogQuota(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testReplicationClusters() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getReplicationClusters(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().setReplicationClusters(topic, new ArrayList<>())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().removeReplicationClusters(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java new file mode 100644 index 0000000000000..1bca6f6e30835 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.admin; + +import io.jsonwebtoken.Jwts; +import lombok.Cleanup; +import lombok.SneakyThrows; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TopicOperation; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +@Test(groups = "broker-admin") +public class TransactionAndSchemaAuthZTest extends AuthZTest { + + @SneakyThrows + @BeforeClass(alwaysRun = true) + public void setup() { + configureTokenAuthentication(); + configureDefaultAuthorization(); + enableTransaction(); + start(); + createTransactionCoordinatorAssign(16); + this.superUserAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + final TenantInfo tenantInfo = superUserAdmin.tenants().getTenantInfo("public"); + tenantInfo.getAdminRoles().add(TENANT_ADMIN_SUBJECT); + superUserAdmin.tenants().updateTenant("public", tenantInfo); + this.tenantManagerAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) + .build(); + + superUserAdmin.tenants().createTenant("pulsar", tenantInfo); + superUserAdmin.namespaces().createNamespace("pulsar/system"); + } + + @SneakyThrows + @AfterClass(alwaysRun = true) + public void cleanup() { + if (superUserAdmin != null) { + superUserAdmin.close(); + } + if (tenantManagerAdmin != null) { + tenantManagerAdmin.close(); + } + close(); + } + + @BeforeMethod + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + authorizationService = Mockito.spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + authorizationService, true); + } + + @AfterMethod + public void after() throws IllegalAccessException { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + orignalAuthorizationService, true); + } + + protected void createTransactionCoordinatorAssign(int numPartitionsOfTC) throws MetadataStoreException { + getPulsarService().getPulsarResources() + .getNamespaceResources() + .getPartitionedTopicResources() + .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, + new PartitionedTopicMetadata(numPartitionsOfTC)); + } + + public enum OperationAuthType { + Lookup, + Produce, + Consume, + AdminOrSuperUser, + NOAuth + } + + private final String testTopic = "persistent://public/default/" + UUID.randomUUID().toString(); + @FunctionalInterface + public interface ThrowingBiConsumer { + void accept(T t) throws PulsarAdminException; + } + + @DataProvider(name = "authFunction") + public Object[][] authFunction () throws Exception { + String sub = "my-sub"; + createTopic(testTopic, false); + @Cleanup final PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(getPulsarService().getBrokerServiceUrl()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .enableTransaction(true) + .build(); + @Cleanup final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(testTopic).create(); + + @Cleanup final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(testTopic) + .subscriptionName(sub) + .subscribe(); + + Transaction transaction = pulsarClient.newTransaction().withTransactionTimeout(5, TimeUnit.MINUTES) + .build().get(); + MessageIdImpl messageId = (MessageIdImpl) producer.newMessage().value("test message").send(); + + consumer.acknowledgeAsync(messageId, transaction).get(); + + return new Object[][]{ + // SCHEMA + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo(testTopic), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo( + testTopic, 0), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getAllSchemas( + testTopic), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().createSchema(testTopic, + SchemaInfo.builder().type(SchemaType.STRING).build()), + OperationAuthType.Produce + }, + // TODO: improve the authorization check for testCompatibility and deleteSchema + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().testCompatibility( + testTopic, SchemaInfo.builder().type(SchemaType.STRING).build()), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().deleteSchema( + testTopic), + OperationAuthType.AdminOrSuperUser + }, + + // TRANSACTION + + // Modify transaction coordinator + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .abortTransaction(transaction.getTxnID()), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .scaleTransactionCoordinators(17), + OperationAuthType.AdminOrSuperUser + }, + // TODO: fix authorization check of check transaction coordinator stats. + // Check transaction coordinator stats + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getCoordinatorInternalStats(1, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getCoordinatorStats(), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getSlowTransactionsByCoordinatorId(1, 5, TimeUnit.SECONDS), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionMetadata(transaction.getTxnID()), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .listTransactionCoordinators(), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getSlowTransactions(5, TimeUnit.SECONDS), + OperationAuthType.AdminOrSuperUser + }, + + // TODO: Check the authorization of the topic when get stats of TB or TP + // Check stats related to transaction buffer and transaction pending ack + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPendingAckInternalStats(testTopic, sub, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPendingAckStats(testTopic, sub, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPositionStatsInPendingAck(testTopic, sub, messageId.getLedgerId(), + messageId.getEntryId(), null), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferInternalStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInBufferStats(transaction.getTxnID(), testTopic), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInBufferStats(transaction.getTxnID(), testTopic), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInPendingAckStats(transaction.getTxnID(), testTopic, sub), + OperationAuthType.NOAuth + }, + }; + } + + @Test(dataProvider = "authFunction") + public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer adminConsumer, OperationAuthType topicOpType) + throws Exception { + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test tenant manager + if (topicOpType != OperationAuthType.AdminOrSuperUser) { + adminConsumer.accept(tenantManagerAdmin); + } + + if (topicOpType != OperationAuthType.NOAuth) { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> adminConsumer.accept(subAdmin)); + } + + AtomicBoolean execFlag = null; + if (topicOpType == OperationAuthType.Lookup) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.LOOKUP); + } else if (topicOpType == OperationAuthType.Produce) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PRODUCE); + } else if (topicOpType == OperationAuthType.Consume) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); + } + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(testTopic, subject, Set.of(action)); + + if (authActionMatchOperation(topicOpType, action)) { + adminConsumer.accept(subAdmin); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> adminConsumer.accept(subAdmin)); + } + superUserAdmin.topics().revokePermissions(testTopic, subject); + } + + if (execFlag != null) { + Assert.assertTrue(execFlag.get()); + } + + } + + private boolean authActionMatchOperation(OperationAuthType operationAuthType, AuthAction action) { + switch (operationAuthType) { + case Lookup -> { + if (AuthAction.consume == action || AuthAction.produce == action) { + return true; + } + } + case Consume -> { + if (AuthAction.consume == action) { + return true; + } + } + case Produce -> { + if (AuthAction.produce == action) { + return true; + } + } + case AdminOrSuperUser -> { + return false; + } + case NOAuth -> { + return true; + } + } + return false; + } + +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java index 01bfd03ceb81a..6c913d4290897 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java @@ -33,6 +33,7 @@ import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; +import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; @@ -56,12 +57,18 @@ public AuthorizationTest() { @Override public void setup() throws Exception { conf.setClusterName("c1"); + conf.setSystemTopicEnabled(false); + conf.setForceDeleteNamespaceAllowed(true); conf.setAuthenticationEnabled(true); + conf.setForceDeleteNamespaceAllowed(true); + conf.setForceDeleteTenantAllowed(true); conf.setAuthenticationProviders( Sets.newHashSet("org.apache.pulsar.broker.auth.MockAuthenticationProvider")); conf.setAuthorizationEnabled(true); conf.setAuthorizationAllowWildcardsMatching(true); conf.setSuperUserRoles(Sets.newHashSet("pulsar.super_user", "pass.pass")); + conf.setBrokerClientAuthenticationPlugin(MockAuthentication.class.getName()); + conf.setBrokerClientAuthenticationParameters("user:pass.pass"); internalSetup(); } @@ -70,6 +77,11 @@ protected void customizeNewPulsarAdminBuilder(PulsarAdminBuilder pulsarAdminBuil pulsarAdminBuilder.authentication(new MockAuthentication("pass.pass")); } + @Override + protected void customizeNewPulsarClientBuilder(ClientBuilder clientBuilder) { + clientBuilder.authentication(new MockAuthentication("pass.pass")); + } + @AfterClass(alwaysRun = true) @Override public void cleanup() throws Exception { @@ -96,8 +108,9 @@ public void simple() throws Exception { assertTrue(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null)); assertTrue(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null)); - admin.topics().grantPermission("persistent://p1/c1/ns1/ds2", "other-role", - EnumSet.of(AuthAction.consume)); + String topic = "persistent://p1/c1/ns1/ds2"; + admin.topics().createNonPartitionedTopic(topic); + admin.topics().grantPermission(topic, "other-role", EnumSet.of(AuthAction.consume)); waitForChange(); assertTrue(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null)); @@ -167,8 +180,9 @@ public void simple() throws Exception { assertFalse(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "my.role.1", null)); assertFalse(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "my.role.2", null)); - admin.topics().grantPermission("persistent://p1/c1/ns1/ds1", "my.*", - EnumSet.of(AuthAction.produce)); + String topic1 = "persistent://p1/c1/ns1/ds1"; + admin.topics().createNonPartitionedTopic(topic1); + admin.topics().grantPermission(topic1, "my.*", EnumSet.of(AuthAction.produce)); waitForChange(); assertTrue(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null)); @@ -231,8 +245,26 @@ public void simple() throws Exception { assertTrue(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "role2", null, "role2-sub2")); assertTrue(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "pulsar.super_user", null, "role3-sub1")); - admin.namespaces().deleteNamespace("p1/c1/ns1"); + admin.namespaces().deleteNamespace("p1/c1/ns1", true); admin.tenants().deleteTenant("p1"); + + admin.clusters().deleteCluster("c1"); + } + + @Test + public void testDeleteV1Tenant() throws Exception { + admin.clusters().createCluster("c1", ClusterData.builder().build()); + admin.tenants().createTenant("p1", new TenantInfoImpl(Sets.newHashSet("role1"), Sets.newHashSet("c1"))); + waitForChange(); + admin.namespaces().createNamespace("p1/c1/ns1"); + waitForChange(); + + + String topic = "persistent://p1/c1/ns1/ds2"; + admin.topics().createNonPartitionedTopic(topic); + + admin.namespaces().deleteNamespace("p1/c1/ns1", true); + admin.tenants().deleteTenant("p1", true); admin.clusters().deleteCluster("c1"); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java index ded4ee8e58d53..358410f1f28e3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java @@ -59,7 +59,10 @@ public void setup() throws Exception { @AfterMethod(alwaysRun = true) void shutdown() throws Exception { - bkEnsemble.stop(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } log.info("---- bk stopped ----"); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java index 7a2314b01a3d1..95aafd84ae406 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java @@ -147,11 +147,20 @@ void shutdown() throws Exception { log.info("--- Shutting down ---"); for (int i = 0; i < BROKER_COUNT; i++) { - pulsarAdmins[i].close(); - pulsarServices[i].close(); + if (pulsarAdmins[i] != null) { + pulsarAdmins[i].close(); + pulsarAdmins[i] = null; + } + if (pulsarServices[i] != null) { + pulsarServices[i].close(); + pulsarServices[i] = null; + } } - bkEnsemble.stop(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } private void loopUntilLeaderChangesForAllBroker(List activePulsars, LeaderBroker oldLeader) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java index f6154e3ec8e30..8f7aa17d0d7bf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java @@ -164,15 +164,33 @@ void setup() throws Exception { @AfterMethod(alwaysRun = true) void shutdown() throws Exception { log.info("--- Shutting down ---"); - executor.shutdownNow(); + if (executor != null) { + executor.shutdownNow(); + executor = null; + } - admin1.close(); - admin2.close(); + if (admin1 != null) { + admin1.close(); + admin1 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } - pulsar2.close(); - pulsar1.close(); + if (pulsar2 != null) { + pulsar2.close(); + pulsar2 = null; + } + if (pulsar1 != null) { + pulsar1.close(); + pulsar1 = null; + } - bkEnsemble.stop(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } private void createNamespacePolicies(PulsarService pulsar) throws Exception { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/AntiAffinityNamespaceGroupExtensionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/AntiAffinityNamespaceGroupExtensionTest.java index d77490e1b8210..cd653a964be36 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/AntiAffinityNamespaceGroupExtensionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/AntiAffinityNamespaceGroupExtensionTest.java @@ -38,6 +38,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; import org.apache.pulsar.broker.loadbalance.extensions.filter.AntiAffinityGroupPolicyFilter; import org.apache.pulsar.broker.loadbalance.extensions.policies.AntiAffinityGroupPolicyHelper; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; @@ -61,7 +62,8 @@ protected String getLoadManagerClassName() { protected String selectBroker(ServiceUnitId serviceUnit, Object loadManager) { try { - return ((ExtensibleLoadManagerImpl) loadManager).assign(Optional.empty(), serviceUnit).get() + return ((ExtensibleLoadManagerImpl) loadManager) + .assign(Optional.empty(), serviceUnit, LookupOptions.builder().build()).get() .get().getPulsarServiceUrl(); } catch (Throwable e) { throw new RuntimeException(e); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java index fdd1eb7272c30..42600a4203551 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java @@ -192,8 +192,14 @@ private BrokerRegistryImpl createBrokerRegistryImpl(PulsarService pulsar) { @AfterClass(alwaysRun = true) void shutdown() throws Exception { - executor.shutdownNow(); - bkEnsemble.stop(); + if (executor != null) { + executor.shutdownNow(); + executor = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } @AfterMethod(alwaysRun = true) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java new file mode 100644 index 0000000000000..41413f3e3a913 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.loadbalance.extensions; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +public class ExtensibleLoadManagerCloseTest { + + private static final String clusterName = "test"; + private final LocalBookkeeperEnsemble bk = new LocalBookkeeperEnsemble(1, 0, () -> 0); + private final List brokers = new ArrayList<>(); + private PulsarAdmin admin; + + @BeforeClass(alwaysRun = true) + public void setup() throws Exception { + bk.start(); + for (int i = 0; i < 3; i++) { + final var broker = new PulsarService(brokerConfig()); + broker.start(); + brokers.add(broker); + } + admin = brokers.get(0).getAdminClient(); + admin.clusters().createCluster(clusterName, ClusterData.builder().build()); + admin.tenants().createTenant("public", TenantInfo.builder() + .allowedClusters(Collections.singleton(clusterName)).build()); + admin.namespaces().createNamespace("public/default"); + } + + + @AfterClass(alwaysRun = true, timeOut = 30000) + public void cleanup() throws Exception { + bk.stop(); + } + + private ServiceConfiguration brokerConfig() { + final var config = new ServiceConfiguration(); + config.setClusterName(clusterName); + config.setAdvertisedAddress("localhost"); + config.setBrokerServicePort(Optional.of(0)); + config.setWebServicePort(Optional.of(0)); + config.setMetadataStoreUrl("zk:127.0.0.1:" + bk.getZookeeperPort()); + config.setManagedLedgerDefaultWriteQuorum(1); + config.setManagedLedgerDefaultAckQuorum(1); + config.setManagedLedgerDefaultEnsembleSize(1); + config.setDefaultNumberOfNamespaceBundles(16); + config.setLoadBalancerAutoBundleSplitEnabled(false); + config.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + config.setLoadBalancerDebugModeEnabled(true); + config.setBrokerShutdownTimeoutMs(100); + return config; + } + + + @Test + public void testCloseAfterLoadingBundles() throws Exception { + final var topic = "test"; + admin.topics().createPartitionedTopic(topic, 20); + admin.lookups().lookupPartitionedTopic(topic); + final var client = PulsarClient.builder().serviceUrl(brokers.get(0).getBrokerServiceUrl()).build(); + final var producer = client.newProducer().topic(topic).create(); + producer.close(); + client.close(); + + final var closeTimeMsList = new ArrayList(); + for (var broker : brokers) { + final var startTimeMs = System.currentTimeMillis(); + broker.close(); + closeTimeMsList.add(System.currentTimeMillis() - startTimeMs); + } + log.info("Brokers close time: {}", closeTimeMsList); + for (var closeTimeMs : closeTimeMsList) { + Assert.assertTrue(closeTimeMs < 5000L); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java index 651a544a04e82..32b7c5027281e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java @@ -21,6 +21,8 @@ import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import com.google.common.collect.Sets; + +import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.tuple.Pair; @@ -69,6 +71,8 @@ protected ServiceConfiguration initConfig(ServiceConfiguration conf) { conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); conf.setLoadBalancerSheddingEnabled(false); conf.setLoadBalancerDebugModeEnabled(true); + conf.setWebServicePortTls(Optional.of(0)); + conf.setBrokerServicePortTls(Optional.of(0)); return conf; } @@ -103,8 +107,15 @@ protected void setup() throws Exception { @Override @AfterClass(alwaysRun = true) protected void cleanup() throws Exception { - this.additionalPulsarTestContext.close(); + if (additionalPulsarTestContext != null) { + additionalPulsarTestContext.close(); + additionalPulsarTestContext = null; + } super.internalCleanup(); + pulsar1 = pulsar2 = null; + primaryLoadManager = secondaryLoadManager = null; + channel1 = channel2 = null; + lookupService = null; } @BeforeMethod(alwaysRun = true) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index b72ab77e81447..a385b0d3c5cca 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -156,10 +156,12 @@ public ExtensibleLoadManagerImplTest() { public void testAssignInternalTopic() throws Exception { Optional brokerLookupData1 = primaryLoadManager.assign( Optional.of(TopicName.get(ServiceUnitStateChannelImpl.TOPIC)), - getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get()).get(); + getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get(), + LookupOptions.builder().build()).get(); Optional brokerLookupData2 = secondaryLoadManager.assign( Optional.of(TopicName.get(ServiceUnitStateChannelImpl.TOPIC)), - getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get()).get(); + getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get(), + LookupOptions.builder().build()).get(); assertEquals(brokerLookupData1, brokerLookupData2); assertTrue(brokerLookupData1.isPresent()); @@ -167,7 +169,7 @@ public void testAssignInternalTopic() throws Exception { FieldUtils.readField(channel1, "leaderElectionService", true); Optional currentLeader = leaderElectionService.getCurrentLeader(); assertTrue(currentLeader.isPresent()); - assertEquals(brokerLookupData1.get().getWebServiceUrl(), currentLeader.get().getServiceUrl()); + assertEquals(brokerLookupData1.get().getWebServiceUrlTls(), currentLeader.get().getServiceUrl()); } @Test @@ -175,15 +177,17 @@ public void testAssign() throws Exception { Pair topicAndBundle = getBundleIsNotOwnByChangeEventTopic("test-assign"); TopicName topicName = topicAndBundle.getLeft(); NamespaceBundle bundle = topicAndBundle.getRight(); - Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle, + LookupOptions.builder().build()).get(); assertTrue(brokerLookupData.isPresent()); log.info("Assign the bundle {} to {}", bundle, brokerLookupData); // Should get owner info from channel. - Optional brokerLookupData1 = secondaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData1 = secondaryLoadManager.assign(Optional.empty(), bundle, + LookupOptions.builder().build()).get(); assertEquals(brokerLookupData, brokerLookupData1); Optional lookupResult = pulsar2.getNamespaceService() - .getBrokerServiceUrlAsync(topicName, null).get(); + .getBrokerServiceUrlAsync(topicName, LookupOptions.builder().build()).get(); assertTrue(lookupResult.isPresent()); assertEquals(lookupResult.get().getLookupData().getHttpUrl(), brokerLookupData.get().getWebServiceUrl()); @@ -193,6 +197,43 @@ public void testAssign() throws Exception { assertEquals(webServiceUrl.get().toString(), brokerLookupData.get().getWebServiceUrl()); } + @Test + public void testLookupOptions() throws Exception { + Pair topicAndBundle = + getBundleIsNotOwnByChangeEventTopic("test-lookup-options"); + TopicName topicName = topicAndBundle.getLeft(); + NamespaceBundle bundle = topicAndBundle.getRight(); + + admin.topics().createPartitionedTopic(topicName.toString(), 1); + + // Test LookupOptions.readOnly = true when the bundle is not owned by any broker. + Optional webServiceUrlReadOnlyTrue = pulsar1.getNamespaceService() + .getWebServiceUrl(bundle, LookupOptions.builder().readOnly(true).requestHttps(false).build()); + assertTrue(webServiceUrlReadOnlyTrue.isEmpty()); + + // Test LookupOptions.readOnly = false and the bundle assign to some broker. + Optional webServiceUrlReadOnlyFalse = pulsar1.getNamespaceService() + .getWebServiceUrl(bundle, LookupOptions.builder().readOnly(false).requestHttps(false).build()); + assertTrue(webServiceUrlReadOnlyFalse.isPresent()); + + // Test LookupOptions.requestHttps = true + Optional webServiceUrlHttps = pulsar2.getNamespaceService() + .getWebServiceUrl(bundle, LookupOptions.builder().requestHttps(true).build()); + assertTrue(webServiceUrlHttps.isPresent()); + assertTrue(webServiceUrlHttps.get().toString().startsWith("https")); + + // TODO: Support LookupOptions.loadTopicsInBundle = true + + // Test LookupOptions.advertisedListenerName = internal but the broker do not have internal listener. + try { + pulsar2.getNamespaceService() + .getWebServiceUrl(bundle, LookupOptions.builder().advertisedListenerName("internal").build()); + fail(); + } catch (Exception e) { + assertTrue(e.getMessage().contains("the broker do not have internal listener")); + } + } + @Test public void testCheckOwnershipAsync() throws Exception { Pair topicAndBundle = getBundleIsNotOwnByChangeEventTopic("test-check-ownership"); @@ -210,7 +251,7 @@ public void testCheckOwnershipAsync() throws Exception { assertFalse(secondaryLoadManager.checkOwnershipAsync(Optional.empty(), bundle).get()); // 2. Assign the bundle to a broker. - Optional lookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional lookupData = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); assertTrue(lookupData.isPresent()); if (lookupData.get().getPulsarServiceUrl().equals(pulsar1.getBrokerServiceUrl())) { assertTrue(primaryLoadManager.checkOwnershipAsync(Optional.empty(), bundle).get()); @@ -243,7 +284,7 @@ public CompletableFuture> filterAsync(Map brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); assertTrue(brokerLookupData.isPresent()); assertEquals(brokerLookupData.get().getWebServiceUrl(), pulsar2.getWebServiceAddress()); } @@ -263,7 +304,7 @@ public CompletableFuture> filterAsync(Map brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); assertTrue(brokerLookupData.isPresent()); } @@ -272,7 +313,7 @@ public void testUnloadUponTopicLookupFailure() throws Exception { TopicName topicName = TopicName.get("public/test/testUnloadUponTopicLookupFailure"); NamespaceBundle bundle = pulsar1.getNamespaceService().getBundle(topicName); - primaryLoadManager.assign(Optional.empty(), bundle).get(); + primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); CompletableFuture future1 = new CompletableFuture(); CompletableFuture future2 = new CompletableFuture(); @@ -869,7 +910,7 @@ public CompletableFuture> filterAsync(Map brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); Awaitility.waitAtMost(5, TimeUnit.SECONDS).untilAsserted(() -> { assertTrue(brokerLookupData.isPresent()); assertEquals(brokerLookupData.get().getWebServiceUrl(), pulsar2.getWebServiceAddress()); @@ -1564,7 +1605,7 @@ public void testGetOwnedServiceUnitsAndGetOwnedNamespaceStatus() throws Exceptio String topic = "persistent://" + defaultTestNamespace + "/test-get-owned-service-units"; admin.topics().createPartitionedTopic(topic, 1); NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).join(); - CompletableFuture> owner = primaryLoadManager.assign(Optional.empty(), bundle); + CompletableFuture> owner = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()); assertFalse(owner.join().isEmpty()); BrokerLookupData brokerLookupData = owner.join().get(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java index fe8387710eeae..1076f92037f10 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java @@ -502,7 +502,7 @@ public void transferTestWhenDestBrokerFails() // recovered, check the monitor update state : Assigned -> Owned doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); FieldUtils.writeDeclaredField(channel2, "producer", producer, true); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1 , true); @@ -724,7 +724,7 @@ public void handleBrokerDeletionEventTest() var owner1 = channel1.getOwnerAsync(bundle1); var owner2 = channel2.getOwnerAsync(bundle2); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); assertTrue(owner1.get().isEmpty()); assertTrue(owner2.get().isEmpty()); @@ -1126,7 +1126,7 @@ public void assignTestWhenDestBrokerProducerFails() FieldUtils.writeDeclaredField(channel2, "inFlightStateWaitingTimeInMillis", 3 * 1000, true); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); channel1.publishAssignEventAsync(bundle, brokerId2); // channel1 is broken. the assign won't be complete. waitUntilState(channel1, bundle); @@ -1525,7 +1525,7 @@ public void testOverrideInactiveBrokerStateData() // test stable metadata state doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); leaderChannel.handleMetadataSessionEvent(SessionReestablished); followerChannel.handleMetadataSessionEvent(SessionReestablished); FieldUtils.writeDeclaredField(leaderChannel, "lastMetadataSessionEventTimestamp", @@ -1590,7 +1590,7 @@ public void testOverrideOrphanStateData() // test stable metadata state doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); FieldUtils.writeDeclaredField(leaderChannel, "inFlightStateWaitingTimeInMillis", -1, true); FieldUtils.writeDeclaredField(followerChannel, "inFlightStateWaitingTimeInMillis", @@ -1645,7 +1645,7 @@ public void testActiveGetOwner() throws Exception { // simulate ownership cleanup(no selected owner) by the leader channel doReturn(CompletableFuture.completedFuture(Optional.empty())) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); var leaderChannel = channel1; String leader1 = channel1.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); String leader2 = channel2.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); @@ -1669,7 +1669,7 @@ public void testActiveGetOwner() throws Exception { overrideTableViews(bundle, new ServiceUnitStateData(Owned, broker, null, 1)); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); leaderChannel.handleMetadataSessionEvent(SessionReestablished); FieldUtils.writeDeclaredField(leaderChannel, "lastMetadataSessionEventTimestamp", System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java index 0d874e0f77117..66e8c917d1fc5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java @@ -18,13 +18,19 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.data; +import static org.testng.Assert.fail; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertTrue; + +import java.net.URI; +import java.net.URISyntaxException; import java.util.HashMap; import java.util.Map; import java.util.Optional; +import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.broker.lookup.LookupResult; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.policies.data.loadbalancer.AdvertisedListener; import org.testng.annotations.Test; @@ -32,12 +38,20 @@ public class BrokerLookupDataTest { @Test - public void testConstructors() { + public void testConstructors() throws PulsarServerException, URISyntaxException { String webServiceUrl = "http://localhost:8080"; String webServiceUrlTls = "https://localhoss:8081"; String pulsarServiceUrl = "pulsar://localhost:6650"; String pulsarServiceUrlTls = "pulsar+ssl://localhost:6651"; - Map advertisedListeners = new HashMap<>(); + final String listenerUrl = "pulsar://gateway:7000"; + final String listenerUrlTls = "pulsar://gateway:8000"; + final String listener = "internal"; + Map advertisedListeners = new HashMap<>(){{ + put(listener, AdvertisedListener.builder() + .brokerServiceUrl(new URI(listenerUrl)) + .brokerServiceUrlTls(new URI(listenerUrlTls)) + .build()); + }}; Map protocols = new HashMap<>(){{ put("kafka", "9092"); }}; @@ -56,10 +70,22 @@ public void testConstructors() { assertEquals("3.0", lookupData.brokerVersion()); - LookupResult lookupResult = lookupData.toLookupResult(); + LookupResult lookupResult = lookupData.toLookupResult(LookupOptions.builder().build()); assertEquals(webServiceUrl, lookupResult.getLookupData().getHttpUrl()); assertEquals(webServiceUrlTls, lookupResult.getLookupData().getHttpUrlTls()); assertEquals(pulsarServiceUrl, lookupResult.getLookupData().getBrokerUrl()); assertEquals(pulsarServiceUrlTls, lookupResult.getLookupData().getBrokerUrlTls()); + + try { + lookupData.toLookupResult(LookupOptions.builder().advertisedListenerName("others").build()); + fail(); + } catch (PulsarServerException ex) { + assertTrue(ex.getMessage().contains("the broker do not have others listener")); + } + lookupResult = lookupData.toLookupResult(LookupOptions.builder().advertisedListenerName(listener).build()); + assertEquals(listenerUrl, lookupResult.getLookupData().getBrokerUrl()); + assertEquals(listenerUrlTls, lookupResult.getLookupData().getBrokerUrlTls()); + assertEquals(webServiceUrl, lookupResult.getLookupData().getHttpUrl()); + assertEquals(webServiceUrlTls, lookupResult.getLookupData().getHttpUrlTls()); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/CustomBrokerSelectionStrategyTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/CustomBrokerSelectionStrategyTest.java new file mode 100644 index 0000000000000..3ac6df2595109 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/CustomBrokerSelectionStrategyTest.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.loadbalance.extensions.strategy; + +import java.util.Comparator; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import lombok.Cleanup; +import org.apache.pulsar.broker.MultiBrokerBaseTest; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder; +import org.apache.pulsar.client.impl.PartitionedProducerImpl; +import org.apache.pulsar.client.impl.ProducerImpl; +import org.testng.Assert; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class CustomBrokerSelectionStrategyTest extends MultiBrokerBaseTest { + + @Override + protected void startBroker() throws Exception { + addCustomConfigs(conf); + super.startBroker(); + } + + @Override + protected ServiceConfiguration createConfForAdditionalBroker(int additionalBrokerIndex) { + return addCustomConfigs(getDefaultConf()); + } + + private static ServiceConfiguration addCustomConfigs(ServiceConfiguration conf) { + conf.setLoadManagerClassName(CustomExtensibleLoadManager.class.getName()); + conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + conf.setLoadBalancerAutoBundleSplitEnabled(false); + conf.setDefaultNumberOfNamespaceBundles(8); + // Don't consider broker's load so the broker will be selected randomly with the default strategy + conf.setLoadBalancerAverageResourceUsageDifferenceThresholdPercentage(100); + return conf; + } + + @Test + public void testSingleBrokerSelected() throws Exception { + final var topic = "test-single-broker-selected"; + getAllAdmins().get(0).topics().createPartitionedTopic(topic, 16); + @Cleanup final var producer = (PartitionedProducerImpl) getAllClients().get(0).newProducer() + .topic(topic).create(); + Assert.assertNotNull(producer); + final var connections = producer.getProducers().stream().map(ProducerImpl::getClientCnx) + .collect(Collectors.toSet()); + Assert.assertEquals(connections.size(), 1); + final var port = Integer.parseInt(connections.stream().findFirst().orElseThrow().ctx().channel() + .remoteAddress().toString().replaceAll(".*:", "")); + final var expectedPort = Stream.concat(Stream.of(pulsar), additionalBrokers.stream()) + .min(Comparator.comparingInt(o -> o.getListenPortHTTP().orElseThrow())) + .map(PulsarService::getBrokerListenPort) + .orElseThrow().orElseThrow(); + Assert.assertEquals(port, expectedPort); + } + + public static class CustomExtensibleLoadManager extends ExtensibleLoadManagerImpl { + + @Override + public BrokerSelectionStrategy createBrokerSelectionStrategy() { + // The smallest HTTP port will always be selected because the host parts are all "localhost" + return (brokers, __, ___) -> brokers.stream().sorted().findFirst(); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTaskTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTaskTest.java index 3173987a3c8a8..bc49352f41d21 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTaskTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTaskTest.java @@ -150,8 +150,14 @@ public void testLoadBalancerNamespaceMaximumBundles() throws Exception { @AfterMethod(alwaysRun = true) void shutdown() throws Exception { log.info("--- Shutting down ---"); - pulsar.close(); - bkEnsemble.stop(); + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java index 824291c52da77..1f9cd806e19b5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java @@ -228,19 +228,36 @@ void setup() throws Exception { @AfterMethod(alwaysRun = true) void shutdown() throws Exception { log.info("--- Shutting down ---"); - executor.shutdownNow(); - - admin1.close(); - admin2.close(); + if (executor != null) { + executor.shutdownNow(); + executor = null; + } - pulsar2.close(); - pulsar1.close(); + if (admin1 != null) { + admin1.close(); + admin1 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } - if (pulsar3.isRunning()) { + if (pulsar2 != null) { + pulsar2.close(); + pulsar2 = null; + } + if (pulsar1 != null) { + pulsar1.close(); + pulsar1 = null; + } + if (pulsar3 != null && pulsar3.isRunning()) { pulsar3.close(); } - - bkEnsemble.stop(); + pulsar3 = null; + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } private NamespaceBundle makeBundle(final String property, final String cluster, final String namespace) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java index 8699c73246830..7aebf20896c2c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java @@ -43,6 +43,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; @@ -94,7 +95,7 @@ public void testRetryStartProducerStoppedByTopicRemove() throws Exception { final ReplicatorInTest replicator = new ReplicatorInTest(localCluster, localTopic, remoteCluster, topicName, replicatorPrefix, broker, remoteClient); replicator.startProducer(); - replicator.disconnect(); + replicator.terminate(); // Verify task will done. Awaitility.await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> { @@ -129,7 +130,7 @@ protected String getProducerName() { } @Override - protected void readEntries(Producer producer) { + protected void setProducerAndTriggerReadEntries(Producer producer) { } @@ -139,7 +140,22 @@ protected Position getReplicatorReadPosition() { } @Override - protected long getNumberOfEntriesInBacklog() { + public ReplicatorStatsImpl getStats() { + return null; + } + + @Override + public void updateRates() { + + } + + @Override + public boolean isConnected() { + return false; + } + + @Override + public long getNumberOfEntriesInBacklog() { return 0; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AdvertisedAddressTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AdvertisedAddressTest.java index 19e40ebf9960f..a60d6599e8f76 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AdvertisedAddressTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AdvertisedAddressTest.java @@ -66,8 +66,14 @@ public void setup() throws Exception { @AfterMethod(alwaysRun = true) public void shutdown() throws Exception { - pulsar.close(); - bkEnsemble.stop(); + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesTestBase.java index 3d9ba658f770e..71c5a995643c6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesTestBase.java @@ -119,9 +119,18 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { config = null; markCurrentSetupNumberCleaned(); - admin.close(); - pulsar.close(); - bkEnsemble.stop(); + if (admin != null) { + admin.close(); + admin = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java index 19aa3ae0bd1c9..d7272fcffa964 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java @@ -105,8 +105,12 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { if (pulsarService != null) { pulsarService.close(); + pulsarService = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; } - bkEnsemble.stop(); } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java index bc6df685ffcd7..a1cb4abc4c30b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java @@ -43,7 +43,6 @@ @Slf4j public abstract class CanReconnectZKClientPulsarServiceBaseTest extends TestRetrySupport { - protected final String defaultTenant = "public"; protected final String defaultNamespace = defaultTenant + "/default"; protected int numberOfBookies = 3; @@ -60,6 +59,7 @@ public abstract class CanReconnectZKClientPulsarServiceBaseTest extends TestRetr protected ZooKeeper localZkOfBroker; protected Object localMetaDataStoreClientCnx; protected final AtomicBoolean LocalMetadataStoreInReconnectFinishSignal = new AtomicBoolean(); + protected void startZKAndBK() throws Exception { // Start ZK. brokerConfigZk = new ZookeeperServerTest(0); @@ -198,18 +198,30 @@ protected void cleanup() throws Exception { stopLocalMetadataStoreAlwaysReconnect(); // Stop brokers. - client.close(); - admin.close(); + if (client != null) { + client.close(); + client = null; + } + if (admin != null) { + admin.close(); + admin = null; + } if (pulsar != null) { pulsar.close(); + pulsar = null; } // Stop ZK and BK. - bkEnsemble.stop(); - brokerConfigZk.stop(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } + if (brokerConfigZk != null) { + brokerConfigZk.stop(); + brokerConfigZk = null; + } // Reset configs. config = new ServiceConfiguration(); - setConfigDefaults(config, clusterName, bkEnsemble, brokerConfigZk); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MaxMessageSizeTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MaxMessageSizeTest.java index 780d33de521b3..84543a82d7725 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MaxMessageSizeTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MaxMessageSizeTest.java @@ -91,9 +91,18 @@ void setup() { @AfterMethod(alwaysRun = true) void shutdown() { try { - pulsar.close(); - bkEnsemble.stop(); - admin.close(); + if (admin != null) { + admin.close(); + admin = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } catch (Throwable t) { t.printStackTrace(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index 1accd04f4918c..9b8b567af081b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -18,28 +18,65 @@ */ package org.apache.pulsar.broker.service; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import com.google.common.collect.Sets; +import io.netty.util.concurrent.FastThreadLocalThread; import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.time.Duration; +import java.util.Arrays; import java.util.Optional; +import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.ProducerBuilderImpl; import org.apache.pulsar.client.impl.ProducerImpl; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.util.FutureUtil; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +@Slf4j @Test(groups = "broker") public class OneWayReplicatorTest extends OneWayReplicatorTestBase { @@ -64,6 +101,20 @@ private void waitReplicatorStarted(String topicName) { }); } + private void waitReplicatorStopped(String topicName) { + Awaitility.await().untilAsserted(() -> { + Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); + assertTrue(topicOptional2.isPresent()); + PersistentTopic persistentTopic2 = (PersistentTopic) topicOptional2.get(); + assertTrue(persistentTopic2.getProducers().isEmpty()); + Optional topicOptional1 = pulsar2.getBrokerService().getTopic(topicName, false).get(); + assertTrue(topicOptional1.isPresent()); + PersistentTopic persistentTopic1 = (PersistentTopic) topicOptional2.get(); + assertTrue(persistentTopic1.getReplicators().isEmpty() + || !persistentTopic1.getReplicators().get(cluster2).isConnected()); + }); + } + /** * Override "AbstractReplicator.producer" by {@param producer} and return the original value. */ @@ -78,9 +129,9 @@ private ProducerImpl overrideProducerForReplicator(AbstractReplicator replicator return originalValue; } - @Test + @Test(timeOut = 45 * 1000) public void testReplicatorProducerStatInTopic() throws Exception { - final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); final String subscribeName = "subscribe_1"; final byte[] msgValue = "test".getBytes(); @@ -104,9 +155,9 @@ public void testReplicatorProducerStatInTopic() throws Exception { }); } - @Test + @Test(timeOut = 45 * 1000) public void testCreateRemoteConsumerFirst() throws Exception { - final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); Producer producer1 = client1.newProducer(Schema.STRING).topic(topicName).create(); // The topic in cluster2 has a replicator created producer(schema Auto_Produce), but does not have any schema。 @@ -124,33 +175,481 @@ public void testCreateRemoteConsumerFirst() throws Exception { }); } - @Test + @Test(timeOut = 45 * 1000) public void testTopicCloseWhenInternalProducerCloseErrorOnce() throws Exception { - final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); admin1.topics().createNonPartitionedTopic(topicName); // Wait for replicator started. waitReplicatorStarted(topicName); - PersistentTopic persistentTopic = + PersistentTopic topic1 = (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); - PersistentReplicator replicator = - (PersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + PersistentReplicator replicator1 = + (PersistentReplicator) topic1.getReplicators().values().iterator().next(); // Mock an error when calling "replicator.disconnect()" - ProducerImpl mockProducer = Mockito.mock(ProducerImpl.class); - Mockito.when(mockProducer.closeAsync()).thenReturn(CompletableFuture.failedFuture(new Exception("mocked ex"))); - ProducerImpl originalProducer = overrideProducerForReplicator(replicator, mockProducer); + AtomicBoolean closeFailed = new AtomicBoolean(true); + final ProducerImpl mockProducer = Mockito.mock(ProducerImpl.class); + final AtomicReference originalProducer1 = new AtomicReference(); + doAnswer(invocation -> { + if (closeFailed.get()) { + return CompletableFuture.failedFuture(new Exception("mocked ex")); + } else { + return originalProducer1.get().closeAsync(); + } + }).when(mockProducer).closeAsync(); + originalProducer1.set(overrideProducerForReplicator(replicator1, mockProducer)); // Verify: since the "replicator.producer.closeAsync()" will retry after it failed, the topic unload should be // successful. admin1.topics().unload(topicName); // Verify: After "replicator.producer.closeAsync()" retry again, the "replicator.producer" will be closed // successful. - overrideProducerForReplicator(replicator, originalProducer); + closeFailed.set(false); + AtomicReference topic2 = new AtomicReference(); + AtomicReference replicator2 = new AtomicReference(); + Awaitility.await().untilAsserted(() -> { + topic2.set((PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get()); + replicator2.set((PersistentReplicator) topic2.get().getReplicators().values().iterator().next()); + // It is a new Topic after reloading. + assertNotEquals(topic2.get(), topic1); + assertNotEquals(replicator2.get(), replicator1); + }); Awaitility.await().untilAsserted(() -> { + // Old replicator should be closed. + Assert.assertFalse(replicator1.isConnected()); + Assert.assertFalse(originalProducer1.get().isConnected()); + // New replicator should be connected. + Assert.assertTrue(replicator2.get().isConnected()); + }); + // cleanup. + cleanupTopics(() -> { + admin1.topics().delete(topicName); + admin2.topics().delete(topicName); + }); + } + + private void injectMockReplicatorProducerBuilder( + BiFunction producerDecorator) + throws Exception { + String cluster2 = pulsar2.getConfig().getClusterName(); + BrokerService brokerService = pulsar1.getBrokerService(); + // Wait for the internal client created. + final String topicNameTriggerInternalClientCreate = + BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createNonPartitionedTopic(topicNameTriggerInternalClientCreate); + waitReplicatorStarted(topicNameTriggerInternalClientCreate); + cleanupTopics(() -> { + admin1.topics().delete(topicNameTriggerInternalClientCreate); + admin2.topics().delete(topicNameTriggerInternalClientCreate); + }); + + // Inject spy client. + ConcurrentOpenHashMap + replicationClients = WhiteboxImpl.getInternalState(brokerService, "replicationClients"); + PulsarClientImpl internalClient = (PulsarClientImpl) replicationClients.get(cluster2); + PulsarClient spyClient = spy(internalClient); + replicationClients.put(cluster2, spyClient); + + // Inject producer decorator. + doAnswer(invocation -> { + Schema schema = (Schema) invocation.getArguments()[0]; + ProducerBuilderImpl producerBuilder = (ProducerBuilderImpl) internalClient.newProducer(schema); + ProducerBuilder spyProducerBuilder = spy(producerBuilder); + doAnswer(ignore -> { + CompletableFuture producerFuture = new CompletableFuture<>(); + producerBuilder.createAsync().whenComplete((p, t) -> { + if (t != null) { + producerFuture.completeExceptionally(t); + return; + } + ProducerImpl pImpl = (ProducerImpl) p; + new FastThreadLocalThread(() -> { + try { + ProducerImpl newProducer = producerDecorator.apply(producerBuilder.getConf(), pImpl); + producerFuture.complete(newProducer); + } catch (Exception ex) { + producerFuture.completeExceptionally(ex); + } + }).start(); + }); + + return producerFuture; + }).when(spyProducerBuilder).createAsync(); + return spyProducerBuilder; + }).when(spyClient).newProducer(any(Schema.class)); + } + + private SpyCursor spyCursor(PersistentTopic persistentTopic, String cursorName) throws Exception { + ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().get(cursorName); + ManagedCursorImpl spyCursor = spy(cursor); + // remove cursor. + ml.getCursors().removeCursor(cursorName); + ml.deactivateCursor(cursor); + // Add the spy one. addCursor(ManagedCursorImpl cursor) + Method m = ManagedLedgerImpl.class.getDeclaredMethod("addCursor", new Class[]{ManagedCursorImpl.class}); + m.setAccessible(true); + m.invoke(ml, new Object[]{spyCursor}); + return new SpyCursor(cursor, spyCursor); + } + + @Data + @AllArgsConstructor + static class SpyCursor { + ManagedCursorImpl original; + ManagedCursorImpl spy; + } + + private CursorCloseSignal makeCursorClosingDelay(SpyCursor spyCursor) throws Exception { + CountDownLatch startCloseSignal = new CountDownLatch(1); + CountDownLatch startCallbackSignal = new CountDownLatch(1); + doAnswer(invocation -> { + AsyncCallbacks.CloseCallback originalCallback = (AsyncCallbacks.CloseCallback) invocation.getArguments()[0]; + Object ctx = invocation.getArguments()[1]; + AsyncCallbacks.CloseCallback newCallback = new AsyncCallbacks.CloseCallback() { + @Override + public void closeComplete(Object ctx) { + new FastThreadLocalThread(new Runnable() { + @Override + @SneakyThrows + public void run() { + startCallbackSignal.await(); + originalCallback.closeComplete(ctx); + } + }).start(); + } + + @Override + public void closeFailed(ManagedLedgerException exception, Object ctx) { + new FastThreadLocalThread(new Runnable() { + @Override + @SneakyThrows + public void run() { + startCallbackSignal.await(); + originalCallback.closeFailed(exception, ctx); + } + }).start(); + } + }; + startCloseSignal.await(); + spyCursor.original.asyncClose(newCallback, ctx); + return null; + }).when(spyCursor.spy).asyncClose(any(AsyncCallbacks.CloseCallback.class), any()); + return new CursorCloseSignal(startCloseSignal, startCallbackSignal); + } + + @AllArgsConstructor + static class CursorCloseSignal { + CountDownLatch startCloseSignal; + CountDownLatch startCallbackSignal; + + void startClose() { + startCloseSignal.countDown(); + } + + void startCallback() { + startCallbackSignal.countDown(); + } + } + + /** + * See the description and execution flow: https://github.com/apache/pulsar/pull/21946. + * Steps: + * - Create topic, but the internal producer of Replicator created failed. + * - Unload bundle, the Replicator will be closed, but the internal producer creation retry has not executed yet. + * - The internal producer creation retry execute successfully, the "repl.cursor" has not been closed yet. + * - The topic is wholly closed. + * - Verify: the delayed created internal producer will be closed. + */ + @Test(timeOut = 120 * 1000) + public void testConcurrencyOfUnloadBundleAndRecreateProducer() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + // Inject an error for "replicator.producer" creation. + // The delay time of next retry to create producer is below: + // 0.1s, 0.2, 0.4, 0.8, 1.6s, 3.2s, 6.4s... + // If the retry counter is larger than 6, the next creation will be slow enough to close Replicator. + final AtomicInteger createProducerCounter = new AtomicInteger(); + final int failTimes = 6; + injectMockReplicatorProducerBuilder((producerCnf, originalProducer) -> { + if (topicName.equals(producerCnf.getTopicName())) { + // There is a switch to determine create producer successfully or not. + if (createProducerCounter.incrementAndGet() > failTimes) { + return originalProducer; + } + log.info("Retry create replicator.producer count: {}", createProducerCounter); + // Release producer and fail callback. + originalProducer.closeAsync(); + throw new RuntimeException("mock error"); + } + return originalProducer; + }); + + // Create topic. + admin1.topics().createNonPartitionedTopic(topicName); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + PersistentReplicator replicator = + (PersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + // Since we inject a producer creation error, the replicator can not start successfully. + assertFalse(replicator.isConnected()); + + // Stuck the closing of the cursor("pulsar.repl"), until the internal producer of the replicator started. + SpyCursor spyCursor = + spyCursor(persistentTopic, "pulsar.repl." + pulsar2.getConfig().getClusterName()); + CursorCloseSignal cursorCloseSignal = makeCursorClosingDelay(spyCursor); + + // Unload bundle: call "topic.close(false)". + // Stuck start new producer, until the state of replicator change to Stopped. + // The next once of "createProducerSuccessAfterFailTimes" to create producer will be successfully. + Awaitility.await().pollInterval(Duration.ofMillis(100)).atMost(Duration.ofSeconds(60)).untilAsserted(() -> { + assertTrue(createProducerCounter.get() >= failTimes, + "count of retry to create producer is " + createProducerCounter.get()); + }); + CompletableFuture topicCloseFuture = persistentTopic.close(true); + Awaitility.await().atMost(Duration.ofSeconds(30)).untilAsserted(() -> { + String state = String.valueOf(replicator.getState()); + assertTrue(state.equals("Stopped") || state.equals("Terminated")); + }); + + // Delay close cursor, until "replicator.producer" create successfully. + // The next once retry time of create "replicator.producer" will be 3.2s. + Thread.sleep(4 * 1000); + log.info("Replicator.state: {}", replicator.getState()); + cursorCloseSignal.startClose(); + cursorCloseSignal.startCallback(); + + // Wait for topic close successfully. + // Verify there is no orphan producer on the remote cluster. + topicCloseFuture.join(); + Awaitility.await().pollInterval(Duration.ofSeconds(1)).untilAsserted(() -> { + PersistentTopic persistentTopic2 = + (PersistentTopic) pulsar2.getBrokerService().getTopic(topicName, false).join().get(); + assertEquals(persistentTopic2.getProducers().size(), 0); Assert.assertFalse(replicator.isConnected()); }); + // cleanup. cleanupTopics(() -> { admin1.topics().delete(topicName); admin2.topics().delete(topicName); }); } + + @Test + public void testPartitionedTopicLevelReplication() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); + final String partition0 = TopicName.get(topicName).getPartition(0).toString(); + final String partition1 = TopicName.get(topicName).getPartition(1).toString(); + admin1.topics().createPartitionedTopic(topicName, 2); + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); + // Check the partitioned topic has been created at the remote cluster. + PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); + assertEquals(topicMetadata2.partitions, 2); + // cleanup. + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); + waitReplicatorStopped(partition0); + waitReplicatorStopped(partition1); + admin1.topics().deletePartitionedTopic(topicName); + admin2.topics().deletePartitionedTopic(topicName); + } + + @Test + public void testPartitionedTopicLevelReplicationRemoteTopicExist() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); + final String partition0 = TopicName.get(topicName).getPartition(0).toString(); + final String partition1 = TopicName.get(topicName).getPartition(1).toString(); + admin1.topics().createPartitionedTopic(topicName, 2); + admin2.topics().createPartitionedTopic(topicName, 2); + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); + // Check the partitioned topic has been created at the remote cluster. + PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); + assertEquals(topicMetadata2.partitions, 2); + // cleanup. + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); + waitReplicatorStopped(partition0); + waitReplicatorStopped(partition1); + admin1.topics().deletePartitionedTopic(topicName); + admin2.topics().deletePartitionedTopic(topicName); + } + + @Test + public void testPartitionedTopicLevelReplicationRemoteConflictTopicExist() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); + admin2.topics().createPartitionedTopic(topicName, 3); + admin1.topics().createPartitionedTopic(topicName, 2); + try { + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); + fail("Expected error due to a conflict partitioned topic already exists."); + } catch (Exception ex) { + Throwable unWrapEx = FutureUtil.unwrapCompletionException(ex); + assertTrue(unWrapEx.getMessage().contains("with different partitions")); + } + // Check nothing changed. + PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); + assertEquals(topicMetadata2.partitions, 3); + assertEquals(admin1.topics().getReplicationClusters(topicName, true).size(), 1); + // cleanup. + admin1.topics().deletePartitionedTopic(topicName); + admin2.topics().deletePartitionedTopic(topicName); + } + + /** + * See the description and execution flow: https://github.com/apache/pulsar/pull/21948. + * Steps: + * 1.Create topic, does not enable replication now. + * - The topic will be loaded in the memory. + * 2.Enable namespace level replication. + * - Broker creates a replicator, and the internal producer of replicator is starting. + * - We inject an error to make the internal producer fail to connect,after few seconds, it will retry to start. + * 3.Unload bundle. + * - Starting to close the topic. + * - The replicator will be closed, but it will not close the internal producer, because the producer has not + * been created successfully. + * - We inject a sleeping into the progress of closing the "repl.cursor" to make it stuck. So the topic is still + * in the process of being closed now. + * 4.Internal producer retry to connect. + * - At the next retry, it connected successful. Since the state of "repl.cursor" is not "Closed", this producer + * will not be closed now. + * 5.Topic closed. + * - Cancel the stuck of closing the "repl.cursor". + * - The topic is wholly closed. + * 6.Verify: the delayed created internal producer will be closed. In other words, there is no producer is connected + * to the remote cluster. + */ + @Test + public void testConcurrencyOfUnloadBundleAndRecreateProducer2() throws Exception { + final String namespaceName = defaultTenant + "/" + UUID.randomUUID().toString().replaceAll("-", ""); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + namespaceName + "/tp_"); + // 1.Create topic, does not enable replication now. + admin1.namespaces().createNamespace(namespaceName); + admin2.namespaces().createNamespace(namespaceName); + admin1.topics().createNonPartitionedTopic(topicName); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + + // We inject an error to make the internal producer fail to connect. + // The delay time of next retry to create producer is below: + // 0.1s, 0.2, 0.4, 0.8, 1.6s, 3.2s, 6.4s... + // If the retry counter is larger than 6, the next creation will be slow enough to close Replicator. + final AtomicInteger createProducerCounter = new AtomicInteger(); + final int failTimes = 6; + injectMockReplicatorProducerBuilder((producerCnf, originalProducer) -> { + if (topicName.equals(producerCnf.getTopicName())) { + // There is a switch to determine create producer successfully or not. + if (createProducerCounter.incrementAndGet() > failTimes) { + return originalProducer; + } + log.info("Retry create replicator.producer count: {}", createProducerCounter); + // Release producer and fail callback. + originalProducer.closeAsync(); + throw new RuntimeException("mock error"); + } + return originalProducer; + }); + + // 2.Enable namespace level replication. + admin1.namespaces().setNamespaceReplicationClusters(namespaceName, Sets.newHashSet(cluster1, cluster2)); + AtomicReference replicator = new AtomicReference(); + Awaitility.await().untilAsserted(() -> { + assertFalse(persistentTopic.getReplicators().isEmpty()); + replicator.set( + (PersistentReplicator) persistentTopic.getReplicators().values().iterator().next()); + // Since we inject a producer creation error, the replicator can not start successfully. + assertFalse(replicator.get().isConnected()); + }); + + // We inject a sleeping into the progress of closing the "repl.cursor" to make it stuck, until the internal + // producer of the replicator started. + SpyCursor spyCursor = + spyCursor(persistentTopic, "pulsar.repl." + pulsar2.getConfig().getClusterName()); + CursorCloseSignal cursorCloseSignal = makeCursorClosingDelay(spyCursor); + + // 3.Unload bundle: call "topic.close(false)". + // Stuck start new producer, until the state of replicator change to Stopped. + // The next once of "createProducerSuccessAfterFailTimes" to create producer will be successfully. + Awaitility.await().pollInterval(Duration.ofMillis(100)).atMost(Duration.ofSeconds(60)).untilAsserted(() -> { + assertTrue(createProducerCounter.get() >= failTimes); + }); + CompletableFuture topicCloseFuture = persistentTopic.close(true); + Awaitility.await().atMost(Duration.ofSeconds(30)).untilAsserted(() -> { + String state = String.valueOf(replicator.get().getState()); + log.error("replicator state: {}", state); + assertTrue(state.equals("Disconnected") || state.equals("Terminated")); + }); + + // 5.Delay close cursor, until "replicator.producer" create successfully. + // The next once retry time of create "replicator.producer" will be 3.2s. + Thread.sleep(4 * 1000); + log.info("Replicator.state: {}", replicator.get().getState()); + cursorCloseSignal.startClose(); + cursorCloseSignal.startCallback(); + // Wait for topic close successfully. + topicCloseFuture.join(); + + // 6. Verify there is no orphan producer on the remote cluster. + Awaitility.await().pollInterval(Duration.ofSeconds(1)).untilAsserted(() -> { + PersistentTopic persistentTopic2 = + (PersistentTopic) pulsar2.getBrokerService().getTopic(topicName, false).join().get(); + assertEquals(persistentTopic2.getProducers().size(), 0); + Assert.assertFalse(replicator.get().isConnected()); + }); + + // cleanup. + cleanupTopics(namespaceName, () -> { + admin1.topics().delete(topicName); + admin2.topics().delete(topicName); + }); + admin1.namespaces().setNamespaceReplicationClusters(namespaceName, Sets.newHashSet(cluster1)); + admin1.namespaces().deleteNamespace(namespaceName); + admin2.namespaces().deleteNamespace(namespaceName); + } + + @Test + public void testUnFenceTopicToReuse() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp"); + // Wait for replicator started. + Producer producer1 = client1.newProducer(Schema.STRING).topic(topicName).create(); + waitReplicatorStarted(topicName); + + // Inject an error to make topic close fails. + final String mockProducerName = UUID.randomUUID().toString(); + final org.apache.pulsar.broker.service.Producer mockProducer = + mock(org.apache.pulsar.broker.service.Producer.class); + doAnswer(invocation -> CompletableFuture.failedFuture(new RuntimeException("mocked error"))) + .when(mockProducer).disconnect(any()); + doAnswer(invocation -> CompletableFuture.failedFuture(new RuntimeException("mocked error"))) + .when(mockProducer).disconnect(); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + persistentTopic.getProducers().put(mockProducerName, mockProducer); + + // Do close. + GeoPersistentReplicator replicator1 = + (GeoPersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + try { + persistentTopic.close(true, false).join(); + fail("Expected close fails due to a producer close fails"); + } catch (Exception ex) { + log.info("Expected error: {}", ex.getMessage()); + } + + // Broker will call `topic.unfenceTopicToResume` if close clients fails. + // Verify: the replicator will be re-created. + Awaitility.await().untilAsserted(() -> { + assertTrue(producer1.isConnected()); + GeoPersistentReplicator replicator2 = + (GeoPersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + assertNotEquals(replicator1, replicator2); + assertFalse(replicator1.isConnected()); + assertFalse(replicator1.producer != null && replicator1.producer.isConnected()); + assertTrue(replicator2.isConnected()); + assertTrue(replicator2.producer != null && replicator2.producer.isConnected()); + }); + + // cleanup. + persistentTopic.getProducers().remove(mockProducerName, mockProducer); + producer1.close(); + cleanupTopics(() -> { + admin1.topics().delete(topicName); + admin2.topics().delete(topicName); + }); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index 33620716288af..b4eed00c4470f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -18,27 +18,35 @@ */ package org.apache.pulsar.broker.service; +import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; import com.google.common.collect.Sets; import java.net.URL; +import java.time.Duration; import java.util.Collections; import java.util.Optional; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.tests.TestRetrySupport; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.apache.pulsar.zookeeper.ZookeeperServerTest; +import org.awaitility.Awaitility; +import org.testng.Assert; @Slf4j public abstract class OneWayReplicatorTestBase extends TestRetrySupport { protected final String defaultTenant = "public"; - protected final String defaultNamespace = defaultTenant + "/default"; + protected final String replicatedNamespace = defaultTenant + "/default"; + protected final String nonReplicatedNamespace = defaultTenant + "/ns1"; protected final String cluster1 = "r1"; protected URL url1; @@ -135,15 +143,43 @@ protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { admin2.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), Sets.newHashSet(cluster1, cluster2))); - admin1.namespaces().createNamespace(defaultNamespace, Sets.newHashSet(cluster1, cluster2)); - admin2.namespaces().createNamespace(defaultNamespace); + admin1.namespaces().createNamespace(replicatedNamespace, Sets.newHashSet(cluster1, cluster2)); + admin2.namespaces().createNamespace(replicatedNamespace); + admin1.namespaces().createNamespace(nonReplicatedNamespace); + admin2.namespaces().createNamespace(nonReplicatedNamespace); } protected void cleanupTopics(CleanupTopicAction cleanupTopicAction) throws Exception { - admin1.namespaces().setNamespaceReplicationClusters(defaultNamespace, Collections.singleton(cluster1)); - admin1.namespaces().unload(defaultNamespace); + cleanupTopics(replicatedNamespace, cleanupTopicAction); + } + + protected void cleanupTopics(String namespace, CleanupTopicAction cleanupTopicAction) throws Exception { + waitChangeEventsInit(namespace); + admin1.namespaces().setNamespaceReplicationClusters(namespace, Collections.singleton(cluster1)); + admin1.namespaces().unload(namespace); cleanupTopicAction.run(); - admin1.namespaces().setNamespaceReplicationClusters(defaultNamespace, Sets.newHashSet(cluster1, cluster2)); + admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet(cluster1, cluster2)); + waitChangeEventsInit(namespace); + } + + protected void waitChangeEventsInit(String namespace) { + PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService() + .getTopic(namespace + "/" + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME, false) + .join().get(); + Awaitility.await().atMost(Duration.ofSeconds(180)).untilAsserted(() -> { + TopicStatsImpl topicStats = topic.getStats(true, false, false); + topicStats.getSubscriptions().entrySet().forEach(entry -> { + // No wait for compaction. + if (COMPACTION_SUBSCRIPTION.equals(entry.getKey())) { + return; + } + // No wait for durable cursor. + if (entry.getValue().isDurable()) { + return; + } + Assert.assertTrue(entry.getValue().getMsgBacklog() == 0, entry.getKey()); + }); + }); } protected interface CleanupTopicAction { @@ -166,7 +202,7 @@ protected void setup() throws Exception { log.info("--- OneWayReplicatorTestBase::setup completed ---"); } - private void setConfigDefaults(ServiceConfiguration config, String clusterName, + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, LocalBookkeeperEnsemble bookkeeperEnsemble, ZookeeperServerTest brokerConfigZk) { config.setClusterName(clusterName); config.setAdvertisedAddress("localhost"); @@ -185,35 +221,70 @@ private void setConfigDefaults(ServiceConfiguration config, String clusterName, config.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); config.setEnableReplicatedSubscriptions(true); config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); + config.setLoadBalancerSheddingEnabled(false); } @Override protected void cleanup() throws Exception { + // delete namespaces. + waitChangeEventsInit(replicatedNamespace); + admin1.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster1)); + admin1.namespaces().deleteNamespace(replicatedNamespace); + admin2.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster2)); + admin2.namespaces().deleteNamespace(replicatedNamespace); + admin1.namespaces().deleteNamespace(nonReplicatedNamespace); + admin2.namespaces().deleteNamespace(nonReplicatedNamespace); + + // shutdown. markCurrentSetupNumberCleaned(); log.info("--- Shutting down ---"); // Stop brokers. - client1.close(); - client2.close(); - admin1.close(); - admin2.close(); + if (client1 != null) { + client1.close(); + client1 = null; + } + if (client2 != null) { + client2.close(); + client2 = null; + } + if (admin1 != null) { + admin1.close(); + admin1 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } if (pulsar2 != null) { pulsar2.close(); + pulsar2 = null; } if (pulsar1 != null) { pulsar1.close(); + pulsar1 = null; } // Stop ZK and BK. - bkEnsemble1.stop(); - bkEnsemble2.stop(); - brokerConfigZk1.stop(); - brokerConfigZk2.stop(); + if (bkEnsemble1 != null) { + bkEnsemble1.stop(); + bkEnsemble1 = null; + } + if (bkEnsemble2 != null) { + bkEnsemble2.stop(); + bkEnsemble2 = null; + } + if (brokerConfigZk1 != null) { + brokerConfigZk1.stop(); + brokerConfigZk1 = null; + } + if (brokerConfigZk2 != null) { + brokerConfigZk2.stop(); + brokerConfigZk2 = null; + } // Reset configs. config1 = new ServiceConfiguration(); - setConfigDefaults(config1, cluster1, bkEnsemble1, brokerConfigZk1); config2 = new ServiceConfiguration(); - setConfigDefaults(config2, cluster2, bkEnsemble2, brokerConfigZk2); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java index d5044276a5a63..de9d0272fc002 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java @@ -1799,12 +1799,12 @@ public void testClosingReplicationProducerTwice() throws Exception { any(), eq(null) ); - replicator.disconnect(false); - replicator.disconnect(false); + replicator.terminate(); + replicator.terminate(); replicator.startProducer(); - verify(clientImpl, Mockito.times(2)).createProducerAsync(any(), any(), any()); + verify(clientImpl, Mockito.times(1)).createProducerAsync(any(), any(), any()); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java index 4cc3a9ada7d04..8aeb902211db2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java @@ -757,7 +757,8 @@ private Object[][] isTopicPolicyEnabled() { * similar to step 1. *

*/ - @Test(dataProvider = "isTopicPolicyEnabled") + // TODO: this test causes OOME in the CI, need to investigate + @Test(dataProvider = "isTopicPolicyEnabled", enabled = false) public void testWriteMarkerTaskOfReplicateSubscriptions(boolean isTopicPolicyEnabled) throws Exception { // 1. Prepare resource and use proper configuration. String namespace = BrokerTestUtil.newUniqueName("pulsar/testReplicateSubBackLog"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index 88a668e8745d5..fa12eba1c6611 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -130,9 +130,11 @@ public class ReplicatorTest extends ReplicatorTestBase { @BeforeMethod(alwaysRun = true) public void beforeMethod(Method m) throws Exception { methodName = m.getName(); - admin1.namespaces().removeBacklogQuota("pulsar/ns"); - admin1.namespaces().removeBacklogQuota("pulsar/ns1"); - admin1.namespaces().removeBacklogQuota("pulsar/global/ns"); + if (admin1 != null) { + admin1.namespaces().removeBacklogQuota("pulsar/ns"); + admin1.namespaces().removeBacklogQuota("pulsar/ns1"); + admin1.namespaces().removeBacklogQuota("pulsar/global/ns"); + } } @Override @@ -152,7 +154,7 @@ public Object[][] partitionedTopicProvider() { return new Object[][] { { Boolean.TRUE }, { Boolean.FALSE } }; } - @Test + @Test(priority = Integer.MAX_VALUE) public void testConfigChange() throws Exception { log.info("--- Starting ReplicatorTest::testConfigChange ---"); // This test is to verify that the config change on global namespace is successfully applied in broker during @@ -895,7 +897,7 @@ public void testReplicatorProducerClosing() throws Exception { pulsar2 = null; pulsar3.close(); pulsar3 = null; - replicator.disconnect(false); + replicator.terminate(); Thread.sleep(100); Field field = AbstractReplicator.class.getDeclaredField("producer"); field.setAccessible(true); @@ -1834,7 +1836,7 @@ public void testReplicatorWithTTL() throws Exception { persistentTopic.getReplicators().forEach((cluster, replicator) -> { PersistentReplicator persistentReplicator = (PersistentReplicator) replicator; // Pause replicator - persistentReplicator.disconnect(); + pauseReplicator(persistentReplicator); }); persistentProducer1.send("V2".getBytes()); @@ -1874,4 +1876,11 @@ public void testReplicatorWithTTL() throws Exception { assertEquals(result, Lists.newArrayList("V1", "V2", "V3", "V4")); } + + private void pauseReplicator(PersistentReplicator replicator) { + Awaitility.await().untilAsserted(() -> { + assertTrue(replicator.isConnected()); + }); + replicator.closeProducerAsync(true); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java index ba9f850ff0cc1..d87f896e31a1c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java @@ -355,22 +355,18 @@ private void setConfigDefaults(ServiceConfiguration config, String clusterName, public void resetConfig1() { config1 = new ServiceConfiguration(); - setConfig1DefaultValue(); } public void resetConfig2() { config2 = new ServiceConfiguration(); - setConfig2DefaultValue(); } public void resetConfig3() { config3 = new ServiceConfiguration(); - setConfig3DefaultValue(); } public void resetConfig4() { config4 = new ServiceConfiguration(); - setConfig4DefaultValue(); } private int inSec(int time, TimeUnit unit) { @@ -386,29 +382,60 @@ protected void cleanup() throws Exception { executor = null; } - admin1.close(); - admin2.close(); - admin3.close(); - admin4.close(); + if (admin1 != null) { + admin1.close(); + admin1 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } + if (admin3 != null) { + admin3.close(); + admin3 = null; + } + if (admin4 != null) { + admin4.close(); + admin4 = null; + } if (pulsar4 != null) { pulsar4.close(); + pulsar4 = null; } if (pulsar3 != null) { pulsar3.close(); + pulsar3 = null; } if (pulsar2 != null) { pulsar2.close(); + pulsar2 = null; } if (pulsar1 != null) { pulsar1.close(); + pulsar1 = null; } - bkEnsemble1.stop(); - bkEnsemble2.stop(); - bkEnsemble3.stop(); - bkEnsemble4.stop(); - globalZkS.stop(); + if (bkEnsemble1 != null) { + bkEnsemble1.stop(); + bkEnsemble1 = null; + } + if (bkEnsemble2 != null) { + bkEnsemble2.stop(); + bkEnsemble2 = null; + } + if (bkEnsemble3 != null) { + bkEnsemble3.stop(); + bkEnsemble3 = null; + } + if (bkEnsemble4 != null) { + bkEnsemble4.stop(); + bkEnsemble4 = null; + } + if (globalZkS != null) { + globalZkS.stop(); + globalZkS = null; + } resetConfig1(); resetConfig2(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicOwnerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicOwnerTest.java index 5a8fd34c9cdba..521d68cebe599 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicOwnerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicOwnerTest.java @@ -117,10 +117,19 @@ void setup() throws Exception { @AfterMethod(alwaysRun = true) void tearDown() throws Exception { for (int i = 0; i < BROKER_COUNT; i++) { - pulsarServices[i].close(); - pulsarAdmins[i].close(); + if (pulsarAdmins[i] != null) { + pulsarAdmins[i].close(); + pulsarAdmins[i] = null; + } + if (pulsarServices[i] != null) { + pulsarServices[i].close(); + pulsarServices[i] = null; + } + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; } - bkEnsemble.stop(); } @SuppressWarnings("unchecked") diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumersTest.java index b2638d53ab1c3..6b0f48a57cfe3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -128,15 +128,15 @@ public void testSendMessage() throws BrokerServiceException { assertEquals(byteBuf.toString(UTF_8), "message" + index); }; return mockPromise; - }).when(consumerMock).sendMessages(any(List.class), any(EntryBatchSizes.class), any(), - anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); + }).when(consumerMock).sendMessages(any(List.class), any(List.class), any(EntryBatchSizes.class), any(), + anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class), anyLong()); try { nonpersistentDispatcher.sendMessages(entries); } catch (Exception e) { fail("Failed to sendMessages.", e); } - verify(consumerMock, times(1)).sendMessages(any(List.class), any(EntryBatchSizes.class), - eq(null), anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); + verify(consumerMock, times(1)).sendMessages(any(List.class), any(List.class), any(EntryBatchSizes.class), + eq(null), anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class), anyLong()); } @Test(timeOut = 10000) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorTest.java new file mode 100644 index 0000000000000..ed5c5a6335ceb --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorTest.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.stats.prometheus; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import com.google.common.util.concurrent.MoreExecutors; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.time.Clock; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.concurrent.ExecutionException; +import java.util.zip.GZIPInputStream; +import org.apache.commons.io.IOUtils; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.testng.annotations.Test; + +public class PrometheusMetricsGeneratorTest { + + // reproduce issue #22575 + @Test + public void testReproducingBufferOverflowExceptionAndEOFExceptionBugsInGzipCompression() + throws ExecutionException, InterruptedException, IOException { + PulsarService pulsar = mock(PulsarService.class); + ServiceConfiguration serviceConfiguration = new ServiceConfiguration(); + when(pulsar.getConfiguration()).thenReturn(serviceConfiguration); + + // generate a random byte buffer which is 8 bytes less than the minimum compress buffer size limit + // this will trigger the BufferOverflowException bug in writing the gzip trailer + // it will also trigger another bug in finishing the gzip compression stream when the compress buffer is full + // which results in EOFException + Random random = new Random(); + byte[] inputBytes = new byte[8192 - 8]; + random.nextBytes(inputBytes); + ByteBuf byteBuf = Unpooled.wrappedBuffer(inputBytes); + + PrometheusMetricsGenerator generator = + new PrometheusMetricsGenerator(pulsar, false, false, false, false, Clock.systemUTC()) { + // override the generateMetrics method to return the random byte buffer for gzip compression + // instead of the actual metrics + @Override + protected ByteBuf generateMetrics(List metricsProviders) { + return byteBuf; + } + }; + + PrometheusMetricsGenerator.MetricsBuffer metricsBuffer = + generator.renderToBuffer(MoreExecutors.directExecutor(), Collections.emptyList()); + try { + PrometheusMetricsGenerator.ResponseBuffer responseBuffer = metricsBuffer.getBufferFuture().get(); + + ByteBuf compressed = responseBuffer.getCompressedBuffer(MoreExecutors.directExecutor()).get(); + byte[] compressedBytes = new byte[compressed.readableBytes()]; + compressed.readBytes(compressedBytes); + try (GZIPInputStream gzipInputStream = new GZIPInputStream(new ByteArrayInputStream(compressedBytes))) { + byte[] uncompressedBytes = IOUtils.toByteArray(gzipInputStream); + assertEquals(uncompressedBytes, inputBytes); + } + } finally { + metricsBuffer.release(); + } + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/coordinator/TransactionMetaStoreTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/coordinator/TransactionMetaStoreTestBase.java index 7a0fb48f91150..5bf48932f3687 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/coordinator/TransactionMetaStoreTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/coordinator/TransactionMetaStoreTestBase.java @@ -122,22 +122,27 @@ public final void shutdownAll() throws Exception { protected void cleanup() throws Exception { if (transactionCoordinatorClient != null) { transactionCoordinatorClient.close(); + transactionCoordinatorClient = null; } - for (PulsarAdmin admin : pulsarAdmins) { - if (admin != null) { - admin.close(); + for (int i = 0; i < BROKER_COUNT; i++) { + if (pulsarAdmins[i] != null) { + pulsarAdmins[i].close(); + pulsarAdmins[i] = null; } } if (pulsarClient != null) { pulsarClient.close(); + pulsarClient = null; } - for (PulsarService service : pulsarServices) { - if (service != null) { - service.close(); + for (int i = 0; i < BROKER_COUNT; i++) { + if (pulsarServices[i] != null) { + pulsarServices[i].close(); + pulsarServices[i] = null; } } if (bkEnsemble != null) { bkEnsemble.stop(); + bkEnsemble = null; } Mockito.reset(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java new file mode 100644 index 0000000000000..7f7fa85bd3bb4 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.web; + +import static org.testng.Assert.assertTrue; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import nl.altindag.console.ConsoleCaptor; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.assertj.core.api.ThrowingConsumer; +import org.awaitility.Awaitility; +import org.eclipse.jetty.client.HttpClient; +import org.eclipse.jetty.client.ProxyProtocolClientConnectionFactory.V2; +import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.util.ssl.SslContextFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class WebServiceOriginalClientIPTest extends MockedPulsarServiceBaseTest { + HttpClient httpClient; + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + httpClient = new HttpClient(new SslContextFactory(true)); + httpClient.start(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + if (httpClient != null) { + httpClient.stop(); + } + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setWebServiceTrustXForwardedFor(true); + conf.setWebServiceHaProxyProtocolEnabled(true); + conf.setWebServicePortTls(Optional.of(0)); + conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); + conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + } + + @DataProvider(name = "tlsEnabled") + public Object[][] tlsEnabled() { + return new Object[][] { { true }, { false } }; + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromXForwardedForHeaderAndLogged(boolean tlsEnabled) throws Exception { + String metricsUrl = + (tlsEnabled ? pulsar.getWebServiceAddressTls() : pulsar.getWebServiceAddress()) + "/metrics/"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(metricsUrl) + .header("X-Forwarded-For", "11.22.33.44:12345") + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("process_cpu_seconds_total")); + + // Validate that the client IP passed in X-Forwarded-For is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("RequestLog") && line.contains("[R:11.22.33.44:12345 via "))); + }); + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromForwardedHeaderAndLogged(boolean tlsEnabled) throws Exception { + String metricsUrl = + (tlsEnabled ? pulsar.getWebServiceAddressTls() : pulsar.getWebServiceAddress()) + "/metrics/"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(metricsUrl) + .header("Forwarded", "for=11.22.33.44:12345") + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("process_cpu_seconds_total")); + + // Validate that the client IP passed in Forwarded is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("RequestLog") && line.contains("[R:11.22.33.44:12345 via "))); + }); + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromHAProxyProtocolAndLogged(boolean tlsEnabled) throws Exception { + String metricsUrl = (tlsEnabled ? pulsar.getWebServiceAddressTls() : pulsar.getWebServiceAddress()) + "/metrics/"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(metricsUrl) + // Jetty client will add HA Proxy protocol header with the given IP to the request + .tag(new V2.Tag(V2.Tag.Command.PROXY, null, V2.Tag.Protocol.STREAM, + // source IP and port + "99.22.33.44", 1234, + // destination IP and port + "5.4.3.1", 4321, + null)) + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("process_cpu_seconds_total")); + + // Validate that the client IP and destination IP passed in HA Proxy protocol is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("RequestLog") && line.contains("[R:99.22.33.44:1234 via ") + && line.contains(" dst 5.4.3.1:4321]"))); + }); + } + + void performLoggingTest(ThrowingConsumer testFunction) { + ConsoleCaptor consoleCaptor = new ConsoleCaptor(); + try { + Awaitility.await().atMost(Duration.of(2, ChronoUnit.SECONDS)).untilAsserted(() -> { + consoleCaptor.clearOutput(); + testFunction.accept(consoleCaptor); + }); + } finally { + consoleCaptor.close(); + System.out.println("--- Captured console output:"); + consoleCaptor.getStandardOutput().forEach(System.out::println); + consoleCaptor.getErrorOutput().forEach(System.err::println); + System.out.println("--- End of captured console output"); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java index f9aa17ea3c451..c46f4744cd5df 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java @@ -263,7 +263,9 @@ public void testAnonymousSyncProducerAndConsumer(int batchMessageDelayMs) throws closeAdmin(); admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrl.toString()).build()); admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); - admin.topics().grantPermission("persistent://my-property/my-ns/my-topic", "anonymousUser", + String topic = "persistent://my-property/my-ns/my-topic"; + admin.topics().createNonPartitionedTopic(topic); + admin.topics().grantPermission(topic, "anonymousUser", EnumSet.allOf(AuthAction.class)); // setup the client diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java index 769486054ab04..2638709abc5e2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java @@ -234,6 +234,7 @@ public void testSubscriberPermission() throws Exception { } // grant topic consume authorization to the subscriptionRole + tenantAdmin.topics().createNonPartitionedTopic(topicName); tenantAdmin.topics().grantPermission(topicName, subscriptionRole, Collections.singleton(AuthAction.consume)); @@ -349,7 +350,8 @@ public void testSubscriberPermission() throws Exception { } catch (Exception e) { // my-sub1 has no msg backlog, so expire message won't be issued on that subscription assertTrue(e.getMessage().startsWith("Expire message by timestamp not issued on topic")); - } sub1Admin.topics().peekMessages(topicName, subscriptionName, 1); + } + sub1Admin.topics().peekMessages(topicName, subscriptionName, 1); sub1Admin.topics().resetCursor(topicName, subscriptionName, 10); sub1Admin.topics().resetCursor(topicName, subscriptionName, MessageId.earliest); @@ -773,6 +775,7 @@ public void testPermissionForProducerCreateInitialSubscription() throws Exceptio admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); + admin.topics().createNonPartitionedTopic(topic); admin.topics().grantPermission(topic, invalidRole, Collections.singleton(AuthAction.produce)); admin.topics().grantPermission(topic, producerRole, Sets.newHashSet(AuthAction.produce, AuthAction.consume)); @@ -990,31 +993,6 @@ public CompletableFuture allowTopicOperationAsync( } } - /** - * This provider always fails authorization on consumer and passes on producer - * - */ - public static class TestAuthorizationProvider2 extends TestAuthorizationProvider { - - @Override - public CompletableFuture canProduceAsync(TopicName topicName, String role, - AuthenticationDataSource authenticationData) { - return CompletableFuture.completedFuture(true); - } - - @Override - public CompletableFuture canConsumeAsync(TopicName topicName, String role, - AuthenticationDataSource authenticationData, String subscription) { - return CompletableFuture.completedFuture(false); - } - - @Override - public CompletableFuture canLookupAsync(TopicName topicName, String role, - AuthenticationDataSource authenticationData) { - return CompletableFuture.completedFuture(true); - } - } - public static class TestAuthorizationProviderWithSubscriptionPrefix extends TestAuthorizationProvider { @Override public CompletableFuture allowTopicOperationAsync(TopicName topic, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationFailureTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationFailureTest.java index 6b3b05405baea..601a8d76aaacd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationFailureTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationFailureTest.java @@ -127,10 +127,22 @@ void setup(Method method) throws Exception { @AfterMethod(alwaysRun = true) void shutdown() throws Exception { log.info("--- Shutting down ---"); - pulsarClient.close(); - admin.close(); - pulsar.close(); - bkEnsemble.stop(); + if (pulsarClient != null) { + pulsarClient.close(); + pulsar = null; + } + if (admin != null) { + admin.close(); + admin = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } private static class ProducerThread implements Runnable { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java index cbf2f28b0b50b..e9b3531c7c2e2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java @@ -216,11 +216,26 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { void shutdown() throws Exception { try { log.info("--- Shutting down ---"); - pulsarClient.close(); - superUserAdmin.close(); - functionsWorkerService.stop(); - pulsar.close(); - bkEnsemble.stop(); + if (pulsarClient != null) { + pulsarClient.close(); + pulsarClient = null; + } + if (superUserAdmin != null) { + superUserAdmin.close(); + superUserAdmin = null; + } + if (functionsWorkerService != null) { + functionsWorkerService.stop(); + functionsWorkerService = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } finally { if (tempDirectory != null) { tempDirectory.delete(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java index 569c2d36ff3a7..50dc39a3a79d2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java @@ -219,11 +219,26 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { void shutdown() throws Exception { try { log.info("--- Shutting down ---"); - pulsarClient.close(); - admin.close(); - functionsWorkerService.stop(); - pulsar.close(); - bkEnsemble.stop(); + if (pulsarClient != null) { + pulsarClient.close(); + pulsarClient = null; + } + if (admin != null) { + admin.close(); + admin = null; + } + if (functionsWorkerService != null) { + functionsWorkerService.stop(); + functionsWorkerService = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } finally { if (tempDirectory != null) { tempDirectory.delete(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java index 3508cf0bfc7e6..3be16357d332b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java @@ -202,11 +202,13 @@ void tearDown() throws Exception { for (int i = 0; i < BROKER_COUNT; i++) { if (pulsarAdmins[i] != null) { pulsarAdmins[i].close(); + pulsarAdmins[i] = null; } } for (int i = 0; i < BROKER_COUNT; i++) { if (fnWorkerServices[i] != null) { fnWorkerServices[i].stop(); + fnWorkerServices[i] = null; } } for (int i = 0; i < BROKER_COUNT; i++) { @@ -221,9 +223,13 @@ void tearDown() throws Exception { getBrokerServicePort().ifPresent(PortManager::releaseLockedPort); pulsarServices[i].getConfiguration() .getWebServicePort().ifPresent(PortManager::releaseLockedPort); + pulsarServices[i] = null; } } - bkEnsemble.stop(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } finally { for (int i = 0; i < BROKER_COUNT; i++) { if (tempDirectories[i] != null) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java index 6226fa904885c..9c137e37095ed 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java @@ -131,11 +131,26 @@ void setup(Method method) throws Exception { void shutdown() { log.info("--- Shutting down ---"); try { - pulsarClient.close(); - admin.close(); - functionsWorkerService.stop(); - pulsar.close(); - bkEnsemble.stop(); + if (pulsarClient != null) { + pulsarClient.close(); + pulsarClient = null; + } + if (admin != null) { + admin.close(); + admin = null; + } + if (functionsWorkerService != null) { + functionsWorkerService.stop(); + functionsWorkerService = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } catch (Exception e) { log.warn("Encountered errors at shutting down PulsarWorkerAssignmentTest", e); } finally { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java index 3c0dd0822b7dc..d27e27639048e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java @@ -239,29 +239,35 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { void shutdown() throws Exception { log.info("--- Shutting down ---"); try { - if (fileServer != null) { - fileServer.stop(); - } + if (fileServer != null) { + fileServer.stop(); + fileServer = null; + } - if (pulsarClient != null) { - pulsarClient.close(); - } + if (pulsarClient != null) { + pulsarClient.close(); + pulsarClient = null; + } - if (admin != null) { - admin.close(); - } + if (admin != null) { + admin.close(); + admin = null; + } - if (functionsWorkerService != null) { - functionsWorkerService.stop(); - } + if (functionsWorkerService != null) { + functionsWorkerService.stop(); + functionsWorkerService = null; + } - if (pulsar != null) { - pulsar.close(); - } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } - if (bkEnsemble != null) { - bkEnsemble.stop(); - } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } finally { if (tempDirectory != null) { tempDirectory.delete(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java index 22b9ad0df3a69..aafd82d339a1d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java @@ -172,11 +172,26 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { @AfterMethod(alwaysRun = true) void shutdown() throws Exception { log.info("--- Shutting down ---"); - pulsarClient.close(); - admin.close(); - functionsWorkerService.stop(); - pulsar.close(); - bkEnsemble.stop(); + if (pulsarClient != null) { + pulsarClient.close(); + pulsarClient = null; + } + if (admin != null) { + admin.close(); + admin = null; + } + if (functionsWorkerService != null) { + functionsWorkerService.stop(); + functionsWorkerService = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration config) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java index 810ac69ac3eb3..da479321b8bc2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java @@ -180,10 +180,22 @@ void setup(Method method) throws Exception { void shutdown() throws Exception { log.info("--- Shutting down ---"); try { - functionAdmin.close(); - functionsWorkerService.stop(); - workerServer.stop(); - bkEnsemble.stop(); + if (functionAdmin != null) { + functionAdmin.close(); + functionAdmin = null; + } + if (functionsWorkerService != null) { + functionsWorkerService.stop(); + functionsWorkerService = null; + } + if (workerServer != null) { + workerServer.stop(); + workerServer = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } finally { if (tempDirectory != null) { tempDirectory.delete(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java index d4f7c72bed016..2d00e15a13f19 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java @@ -55,6 +55,7 @@ public ProxyAuthorizationTest() { @Override protected void setup() throws Exception { conf.setClusterName(configClusterName); + conf.setForceDeleteNamespaceAllowed(true); internalSetup(); WebSocketProxyConfiguration config = new WebSocketProxyConfiguration(); @@ -99,8 +100,9 @@ public void test() throws Exception { assertTrue(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null)); assertTrue(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null)); - admin.topics().grantPermission("persistent://p1/c1/ns1/ds2", "other-role", - EnumSet.of(AuthAction.consume)); + String topic = "persistent://p1/c1/ns1/ds2"; + admin.topics().createNonPartitionedTopic(topic); + admin.topics().grantPermission(topic, "other-role", EnumSet.of(AuthAction.consume)); waitForChange(); assertTrue(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null)); @@ -117,7 +119,7 @@ public void test() throws Exception { assertTrue(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null)); assertTrue(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null, null)); - admin.namespaces().deleteNamespace("p1/c1/ns1"); + admin.namespaces().deleteNamespace("p1/c1/ns1", true); admin.tenants().deleteTenant("p1"); admin.clusters().deleteCluster("c1"); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/ZookeeperServerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/ZookeeperServerTest.java index eec61b9144e47..355d2a0b1dbe1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/ZookeeperServerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/ZookeeperServerTest.java @@ -68,15 +68,20 @@ public void start() throws IOException { } public void stop() throws IOException { - zks.shutdown(); - serverFactory.shutdown(); + if (zks != null) { + zks.shutdown(); + zks = null; + } + if (serverFactory != null) { + serverFactory.shutdown(); + serverFactory = null; + } log.info("Stopped ZK server at {}", hostPort); } @Override public void close() throws IOException { - zks.shutdown(); - serverFactory.shutdown(); + stop(); zkTmpDir.delete(); } diff --git a/pulsar-broker/src/test/resources/log4j2.xml b/pulsar-broker/src/test/resources/log4j2.xml index 38a57df80d57b..09a89702ee2ac 100644 --- a/pulsar-broker/src/test/resources/log4j2.xml +++ b/pulsar-broker/src/test/resources/log4j2.xml @@ -23,7 +23,8 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://logging.apache.org/log4j/2.0/config https://logging.apache.org/log4j/2.0/log4j-core.xsd"> - + + diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 781c03721fb69..eca03ee1502b0 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -66,7 +66,7 @@
org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl test diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index 3917e2996e180..f79af79d57452 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -86,7 +86,6 @@ io.swagger swagger-annotations - provided diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java index 51e181811c228..6c40aa3f2edd0 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java @@ -79,8 +79,9 @@ public class OffloadPoliciesImpl implements Serializable, OffloadPolicies { } } - public static final int DEFAULT_MAX_BLOCK_SIZE_IN_BYTES = 64 * 1024 * 1024; // 64MB - public static final int DEFAULT_READ_BUFFER_SIZE_IN_BYTES = 1024 * 1024; // 1MB + public static final int DEFAULT_MAX_BLOCK_SIZE_IN_BYTES = 64 * 1024 * 1024; // 64MiB + public static final int DEFAULT_GCS_MAX_BLOCK_SIZE_IN_BYTES = 128 * 1024 * 1024; // 128MiB + public static final int DEFAULT_READ_BUFFER_SIZE_IN_BYTES = 1024 * 1024; // 1MiB public static final int DEFAULT_OFFLOAD_MAX_THREADS = 2; public static final int DEFAULT_OFFLOAD_MAX_PREFETCH_ROUNDS = 1; public static final String DEFAULT_OFFLOADER_DIRECTORY = "./offloaders"; @@ -163,7 +164,7 @@ public class OffloadPoliciesImpl implements Serializable, OffloadPolicies { private String gcsManagedLedgerOffloadBucket = null; @Configuration @JsonProperty(access = JsonProperty.Access.READ_WRITE) - private Integer gcsManagedLedgerOffloadMaxBlockSizeInBytes = DEFAULT_MAX_BLOCK_SIZE_IN_BYTES; + private Integer gcsManagedLedgerOffloadMaxBlockSizeInBytes = DEFAULT_GCS_MAX_BLOCK_SIZE_IN_BYTES; @Configuration @JsonProperty(access = JsonProperty.Access.READ_WRITE) private Integer gcsManagedLedgerOffloadReadBufferSizeInBytes = DEFAULT_READ_BUFFER_SIZE_IN_BYTES; diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index b8d197c0683d3..84d8a032d3bde 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -36,7 +36,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl org.apache.logging.log4j diff --git a/pulsar-functions/localrun/pom.xml b/pulsar-functions/localrun/pom.xml index 5118a30b92f94..001780bdb5178 100644 --- a/pulsar-functions/localrun/pom.xml +++ b/pulsar-functions/localrun/pom.xml @@ -43,7 +43,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl org.apache.logging.log4j diff --git a/pulsar-functions/runtime-all/pom.xml b/pulsar-functions/runtime-all/pom.xml index 5e300c78aac46..d367038ff4945 100644 --- a/pulsar-functions/runtime-all/pom.xml +++ b/pulsar-functions/runtime-all/pom.xml @@ -105,7 +105,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java index ec0e620d0ae8b..036311ea13230 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java @@ -163,6 +163,22 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { + "(0 to disable limiting)") private int maxHttpServerConnections = 2048; + @FieldContext(category = CATEGORY_WORKER, + doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(category = CATEGORY_WORKER, doc = + "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(category = CATEGORY_WORKER, doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext( category = CATEGORY_WORKER, required = false, diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java index 2b3ea30121015..583d8ce558b08 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java @@ -35,10 +35,17 @@ import org.apache.pulsar.functions.worker.rest.api.v2.WorkerApiV2Resource; import org.apache.pulsar.functions.worker.rest.api.v2.WorkerStatsApiV2Resource; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.HttpConfiguration; +import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerCollection; @@ -88,10 +95,21 @@ private void init() { server.addBean(new ConnectionLimit(workerConfig.getMaxHttpServerConnections(), server)); } + HttpConfiguration httpConfig = new HttpConfiguration(); + if (workerConfig.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); + List connectors = new ArrayList<>(); if (this.workerConfig.getWorkerPort() != null) { log.info("Configuring http server on port={}", this.workerConfig.getWorkerPort()); - httpConnector = new ServerConnector(server); + List connectionFactories = new ArrayList<>(); + if (workerConfig.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + httpConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpConnector.setPort(this.workerConfig.getWorkerPort()); connectors.add(httpConnector); } @@ -109,7 +127,10 @@ private void init() { workerConfig.isAuthenticateMetricsEndpoint(), filterInitializer)); RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = workerConfig.getWebServiceLogDetailedAddresses() != null + ? workerConfig.getWebServiceLogDetailedAddresses() : + (workerConfig.isWebServiceHaProxyProtocolEnabled() || workerConfig.isWebServiceTrustXForwardedFor()); + requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); @@ -161,7 +182,18 @@ private void init() { workerConfig.getTlsCertRefreshCheckDurationSec() ); } - httpsConnector = new ServerConnector(server, sslCtxFactory); + List connectionFactories = new ArrayList<>(); + if (workerConfig.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + httpsConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpsConnector.setPort(this.workerConfig.getWorkerPortTls()); connectors.add(httpsConnector); } catch (Exception e) { diff --git a/pulsar-io/alluxio/pom.xml b/pulsar-io/alluxio/pom.xml index 53f2ed095c683..86d76ec9578ee 100644 --- a/pulsar-io/alluxio/pom.xml +++ b/pulsar-io/alluxio/pom.xml @@ -57,6 +57,12 @@ org.alluxio alluxio-core-client-fs ${alluxio.version} + + + log4j-slf4j-impl + org.apache.logging.log4j + + @@ -69,6 +75,10 @@ org.glassfish javax.el + + log4j-slf4j-impl + org.apache.logging.log4j + diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticSearchJavaRestClient.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticSearchJavaRestClient.java index 4749ea2e2d383..afda5ba0e7449 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticSearchJavaRestClient.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticSearchJavaRestClient.java @@ -84,6 +84,7 @@ public ElasticSearchJavaRestClient(ElasticSearchConfig elasticSearchConfig, .setConnectionRequestTimeout(config.getConnectionRequestTimeoutInMs()) .setConnectTimeout(config.getConnectTimeoutInMs()) .setSocketTimeout(config.getSocketTimeoutInMs())) + .setCompressionEnabled(config.isCompressionEnabled()) .setHttpClientConfigCallback(this.configCallback) .setFailureListener(new org.elasticsearch.client.RestClient.FailureListener() { public void onFailure(Node node) { diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java index 7b7041967026e..bb92047f17a31 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java @@ -112,6 +112,7 @@ public OpenSearchHighLevelRestClient(ElasticSearchConfig elasticSearchConfig, .setConnectionRequestTimeout(config.getConnectionRequestTimeoutInMs()) .setConnectTimeout(config.getConnectTimeoutInMs()) .setSocketTimeout(config.getSocketTimeoutInMs())) + .setCompressionEnabled(config.isCompressionEnabled()) .setHttpClientConfigCallback(this.configCallback) .setFailureListener(new org.opensearch.client.RestClient.FailureListener() { @Override diff --git a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java index c1e0eafe03a55..468d78d989cf1 100644 --- a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java +++ b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java @@ -30,8 +30,10 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; +import co.elastic.clients.transport.rest_client.RestClientTransport; import eu.rekawek.toxiproxy.model.ToxicDirection; import java.io.IOException; +import java.lang.reflect.Field; import java.util.Map; import java.util.Optional; import java.util.UUID; @@ -46,6 +48,8 @@ import org.apache.pulsar.io.elasticsearch.client.opensearch.OpenSearchHighLevelRestClient; import org.apache.pulsar.io.elasticsearch.testcontainers.ElasticToxiproxiContainer; import org.awaitility.Awaitility; +import org.opensearch.client.RestClient; +import org.opensearch.client.RestHighLevelClient; import org.testcontainers.containers.Network; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testng.annotations.AfterClass; @@ -110,11 +114,41 @@ public void fail() { public void testClientInstance() throws Exception { try (ElasticSearchClient client = new ElasticSearchClient(new ElasticSearchConfig() .setElasticSearchUrl("http://" + container.getHttpHostAddress()) + .setCompressionEnabled(true) .setIndexName(INDEX), mock(SinkContext.class));) { if (elasticImageName.equals(OPENSEARCH) || elasticImageName.equals(ELASTICSEARCH_7)) { assertTrue(client.getRestClient() instanceof OpenSearchHighLevelRestClient); + OpenSearchHighLevelRestClient osRestHighLevelClient = (OpenSearchHighLevelRestClient) client.getRestClient(); + RestHighLevelClient restHighLevelClient = osRestHighLevelClient.getClient(); + assertNotNull(restHighLevelClient); + + Field field = RestHighLevelClient.class.getDeclaredField("client"); + field.setAccessible(true); + RestClient restClient = (RestClient) field.get(restHighLevelClient); + assertNotNull(restClient); + + Field compressionEnabledFiled = RestClient.class.getDeclaredField("compressionEnabled"); + compressionEnabledFiled.setAccessible(true); + boolean compressionEnabled = (boolean) compressionEnabledFiled.get(restClient); + assertTrue(compressionEnabled); } else { assertTrue(client.getRestClient() instanceof ElasticSearchJavaRestClient); + ElasticSearchJavaRestClient javaRestClient = (ElasticSearchJavaRestClient) client.getRestClient(); + + Field field = ElasticSearchJavaRestClient.class.getDeclaredField("transport"); + field.setAccessible(true); + RestClientTransport transport = (RestClientTransport) field.get(javaRestClient); + assertNotNull(transport); + + Field restClientFiled = RestClientTransport.class.getDeclaredField("restClient"); + restClientFiled.setAccessible(true); + org.elasticsearch.client.RestClient restClient = (org.elasticsearch.client.RestClient) restClientFiled.get(transport); + assertNotNull(restClient); + + Field compressionEnabledFiled = org.elasticsearch.client.RestClient.class.getDeclaredField("compressionEnabled"); + compressionEnabledFiled.setAccessible(true); + boolean compressionEnabled = (boolean) compressionEnabledFiled.get(restClient); + assertTrue(compressionEnabled); } } } diff --git a/pulsar-io/hdfs2/pom.xml b/pulsar-io/hdfs2/pom.xml index ef198579544a4..38f1460a33c57 100644 --- a/pulsar-io/hdfs2/pom.xml +++ b/pulsar-io/hdfs2/pom.xml @@ -61,7 +61,7 @@ org.slf4j - slf4j-log4j12 + * org.apache.avro diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index dcb0226a1f8d0..1a7a975098bec 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -65,7 +65,7 @@ org.slf4j - slf4j-log4j12 + * org.apache.avro diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index 9074faed3b7cf..f8a5c0601d146 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -49,7 +49,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl @@ -83,7 +83,7 @@ org.apache.qpid qpid-broker - 6.1.6 + 9.2.0 test diff --git a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java index 507313c86fd7f..83331bf7de810 100644 --- a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java +++ b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java @@ -18,28 +18,29 @@ */ package org.apache.pulsar.io.rabbitmq; -import org.apache.qpid.server.Broker; -import org.apache.qpid.server.BrokerOptions; - import java.io.File; import java.io.FileOutputStream; import java.nio.file.Files; import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; +import org.apache.qpid.server.SystemLauncher; +import org.apache.qpid.server.model.SystemConfig; public class RabbitMQBrokerManager { - private final Broker broker = new Broker(); + private final SystemLauncher systemLauncher = new SystemLauncher(); public void startBroker(String port) throws Exception { - BrokerOptions brokerOptions = getBrokerOptions(port); - broker.startup(brokerOptions); + Map brokerOptions = getBrokerOptions(port); + systemLauncher.startup(brokerOptions); } public void stopBroker() { - broker.shutdown(); + systemLauncher.shutdown(); } - BrokerOptions getBrokerOptions(String port) throws Exception { + Map getBrokerOptions(String port) throws Exception { Path tmpFolder = Files.createTempDirectory("qpidWork"); Path homeFolder = Files.createTempDirectory("qpidHome"); File etc = new File(homeFolder.toFile(), "etc"); @@ -48,15 +49,17 @@ BrokerOptions getBrokerOptions(String port) throws Exception { fos.write("guest:guest\n".getBytes()); fos.close(); - BrokerOptions brokerOptions = new BrokerOptions(); - - brokerOptions.setConfigProperty("qpid.work_dir", tmpFolder.toAbsolutePath().toString()); - brokerOptions.setConfigProperty("qpid.amqp_port", port); - brokerOptions.setConfigProperty("qpid.home_dir", homeFolder.toAbsolutePath().toString()); + Map config = new HashMap<>(); + config.put("qpid.work_dir", tmpFolder.toAbsolutePath().toString()); + config.put("qpid.amqp_port", port); + config.put("qpid.home_dir", homeFolder.toAbsolutePath().toString()); String configPath = getFile("qpid.json").getAbsolutePath(); - brokerOptions.setInitialConfigurationLocation(configPath); - return brokerOptions; + Map context = new HashMap<>(); + context.put(SystemConfig.INITIAL_CONFIGURATION_LOCATION, configPath); + context.put(SystemConfig.TYPE, "Memory"); + context.put(SystemConfig.CONTEXT, config); + return context; } private File getFile(String name) { diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index 5be2639c718fb..2b7893fc945a1 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -70,6 +70,10 @@ jose4j org.bitbucket.b_c + + log4j-slf4j-impl + org.apache.logging.log4j + test diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index 8600d0ea1919b..163a3058dc4bc 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -65,7 +65,6 @@ io.streamnative.oxia oxia-client - shaded diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index 079ae3e2ae5c3..2e88cb3332467 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -192,7 +192,20 @@ protected void batchOperation(List ops) { Code code = Code.get(rc); if (code == Code.CONNECTIONLOSS) { // There is the chance that we caused a connection reset by sending or requesting a batch - // that passed the max ZK limit. Retry with the individual operations + // that passed the max ZK limit. + + // Build the log warning message + // summarize the operations by type + String countsByType = ops.stream().collect( + Collectors.groupingBy(MetadataOp::getType, Collectors.summingInt(op -> 1))) + .entrySet().stream().map(e -> e.getValue() + " " + e.getKey().name() + " entries") + .collect(Collectors.joining(", ")); + Long totalSize = ops.stream().collect(Collectors.summingLong(MetadataOp::size)); + log.warn("Connection loss while executing batch operation of {} " + + "of total data size of {}. " + + "Retrying individual operations one-by-one.", countsByType, totalSize); + + // Retry with the individual operations executor.schedule(() -> { ops.forEach(o -> batchOperation(Collections.singletonList(o))); }, 100, TimeUnit.MILLISECONDS); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/TestZKServer.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/TestZKServer.java index 33034ddb3fe0f..0d01d9c56abc8 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/TestZKServer.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/TestZKServer.java @@ -131,6 +131,7 @@ public void checkContainers() throws Exception { public void stop() throws Exception { if (zooKeeperServerEmbedded != null) { zooKeeperServerEmbedded.close(); + zooKeeperServerEmbedded = null; } log.info("Stopped test ZK server"); } diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 64ca301facf4d..a30e23b8d4781 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -209,6 +209,12 @@ ${wiremock.version} test + + io.github.hakky54 + consolecaptor + ${consolecaptor.version} + test + diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java index 39c8fb5e086fd..d65408748f432 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java @@ -268,6 +268,22 @@ public class ProxyConfiguration implements PulsarConfiguration { doc = "Enable or disable the proxy protocol.") private boolean haProxyProtocolEnabled; + @FieldContext(category = CATEGORY_SERVER, + doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext(category = CATEGORY_SERVER, doc = "Enables zero-copy transport of data across network interfaces using the spice. " + "Zero copy mode cannot be used when TLS is enabled or when proxyLogLevel is > 0.") diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index 50a8e3ab7d753..10121e7f5d61d 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -35,6 +35,7 @@ import java.util.Collections; import java.util.Date; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import lombok.Getter; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.core.util.datetime.FixedDateFormat; @@ -109,8 +110,15 @@ public class ProxyServiceStarter { private WebServer server; private WebSocketService webSocketService; private static boolean metricsInitialized; + private boolean embeddedMode; public ProxyServiceStarter(String[] args) throws Exception { + this(args, null, false); + } + + public ProxyServiceStarter(String[] args, Consumer proxyConfigurationCustomizer, + boolean embeddedMode) throws Exception { + this.embeddedMode = embeddedMode; try { DateFormat dateFormat = new SimpleDateFormat( FixedDateFormat.FixedFormat.ISO8601_OFFSET_DATE_TIME_HHMM.getPattern()); @@ -132,15 +140,26 @@ public ProxyServiceStarter(String[] args) throws Exception { CmdGenerateDocs cmd = new CmdGenerateDocs("pulsar"); cmd.addCommand("proxy", commander); cmd.run(null); - System.exit(0); + if (embeddedMode) { + return; + } else { + System.exit(0); + } } } catch (Exception e) { commander.getErr().println(e); - System.exit(1); + if (embeddedMode) { + return; + } else { + System.exit(1); + } } // load config file config = PulsarConfigurationLoader.create(configFile, ProxyConfiguration.class); + if (proxyConfigurationCustomizer != null) { + proxyConfigurationCustomizer.accept(config); + } if (!isBlank(zookeeperServers)) { // Use zookeeperServers from command line @@ -230,7 +249,9 @@ public void start() throws Exception { // create a web-service server = new WebServer(config, authenticationService); - Runtime.getRuntime().addShutdownHook(new Thread(this::close)); + if (!embeddedMode) { + Runtime.getRuntime().addShutdownHook(new Thread(this::close)); + } proxyService.start(); @@ -293,7 +314,9 @@ public void close() { } catch (Exception e) { log.warn("server couldn't stop gracefully {}", e.getMessage(), e); } finally { - LogManager.shutdown(); + if (!embeddedMode) { + LogManager.shutdown(); + } } } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java index b95bbcab08b11..478b911eb23cf 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java @@ -37,13 +37,18 @@ import org.apache.pulsar.broker.web.RateLimitingFilter; import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; import org.eclipse.jetty.server.Connector; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerCollection; @@ -93,12 +98,21 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication List connectors = new ArrayList<>(); HttpConfiguration httpConfig = new HttpConfiguration(); + if (config.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } httpConfig.setOutputBufferSize(config.getHttpOutputBufferSize()); httpConfig.setRequestHeaderSize(config.getHttpMaxRequestHeaderSize()); + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); if (config.getWebServicePort().isPresent()) { this.externalServicePort = config.getWebServicePort().get(); - connector = new ServerConnector(server, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + connector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connector.setHost(config.getBindAddress()); connector.setPort(externalServicePort); connectors.add(connector); @@ -133,7 +147,18 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication config.getWebServiceTlsProtocols(), config.getTlsCertRefreshCheckDurationSec()); } - connectorTls = new ServerConnector(server, sslCtxFactory, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + connectorTls = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connectorTls.setPort(config.getWebServicePortTls().get()); connectorTls.setHost(config.getBindAddress()); connectors.add(connectorTls); @@ -281,7 +306,10 @@ public int getExternalServicePort() { public void start() throws Exception { RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = config.getWebServiceLogDetailedAddresses() != null + ? config.getWebServiceLogDetailedAddresses() : + (config.isWebServiceHaProxyProtocolEnabled() || config.isWebServiceTrustXForwardedFor()); + requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java new file mode 100644 index 0000000000000..b267439d47113 --- /dev/null +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.proxy.server; + +import static org.testng.Assert.assertTrue; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import nl.altindag.console.ConsoleCaptor; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.assertj.core.api.ThrowingConsumer; +import org.awaitility.Awaitility; +import org.eclipse.jetty.client.HttpClient; +import org.eclipse.jetty.client.ProxyProtocolClientConnectionFactory.V2; +import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.util.ssl.SslContextFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class ProxyOriginalClientIPTest extends MockedPulsarServiceBaseTest { + static final String[] ARGS = new String[]{"-c", "./src/test/resources/proxy.conf"}; + HttpClient httpClient; + ProxyServiceStarter serviceStarter; + String webServiceUrl; + String webServiceUrlTls; + + @Override + @BeforeClass + protected void setup() throws Exception { + internalSetup(); + serviceStarter = new ProxyServiceStarter(ARGS, proxyConfig -> { + proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); + proxyConfig.setBrokerWebServiceURL(pulsar.getWebServiceAddress()); + proxyConfig.setWebServicePort(Optional.of(0)); + proxyConfig.setWebServicePortTls(Optional.of(0)); + proxyConfig.setTlsEnabledWithBroker(false); + proxyConfig.setTlsCertificateFilePath(PROXY_CERT_FILE_PATH); + proxyConfig.setTlsKeyFilePath(PROXY_KEY_FILE_PATH); + proxyConfig.setServicePort(Optional.of(0)); + proxyConfig.setWebSocketServiceEnabled(true); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setClusterName(configClusterName); + proxyConfig.setWebServiceTrustXForwardedFor(true); + proxyConfig.setWebServiceHaProxyProtocolEnabled(true); + }, true); + serviceStarter.start(); + webServiceUrl = "http://localhost:" + serviceStarter.getServer().getListenPortHTTP().get(); + webServiceUrlTls = "https://localhost:" + serviceStarter.getServer().getListenPortHTTPS().get(); + httpClient = new HttpClient(new SslContextFactory(true)); + httpClient.start(); + } + + @Override + @AfterClass(alwaysRun = true) + protected void cleanup() throws Exception { + internalCleanup(); + if (serviceStarter != null) { + serviceStarter.close(); + } + if (httpClient != null) { + httpClient.stop(); + } + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setWebServiceTrustXForwardedFor(true); + } + + @DataProvider(name = "tlsEnabled") + public Object[][] tlsEnabled() { + return new Object[][] { { true }, { false } }; + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromXForwardedForHeaderAndLogged(boolean tlsEnabled) throws Exception { + String url = (tlsEnabled ? webServiceUrlTls : webServiceUrl) + "/admin/v2/brokers/leaderBroker"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(url) + .header("X-Forwarded-For", "11.22.33.44") + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("\"brokerId\":\"" + pulsar.getBrokerId() + "\"")); + + // Validate that the client IP passed in X-Forwarded-For is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-external-web-") && line.contains("RequestLog") + && line.contains("R:11.22.33.44")), "Expected to find client IP in proxy logs"); + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-web-") && line.contains("RequestLog") + && line.contains("R:11.22.33.44")), "Expected to find client IP in broker logs"); + }); + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromHAProxyProtocolAndLogged(boolean tlsEnabled) throws Exception { + String url = (tlsEnabled ? webServiceUrlTls : webServiceUrl) + "/admin/v2/brokers/leaderBroker"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(url) + // Jetty client will add HA Proxy protocol header with the given IP to the request + .tag(new V2.Tag("99.22.33.44", 1234)) + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("\"brokerId\":\"" + pulsar.getBrokerId() + "\"")); + + // Validate that the client IP passed in HA proxy protocol is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-external-web-") && line.contains("RequestLog") + && line.contains("R:99.22.33.44")), "Expected to find client IP in proxy logs"); + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-web-") && line.contains("RequestLog") + && line.contains("R:99.22.33.44")), "Expected to find client IP in broker logs"); + }); + } + + void performLoggingTest(ThrowingConsumer testFunction) { + ConsoleCaptor consoleCaptor = new ConsoleCaptor(); + try { + Awaitility.await().atMost(Duration.of(2, ChronoUnit.SECONDS)).untilAsserted(() -> { + consoleCaptor.clearOutput(); + testFunction.accept(consoleCaptor); + }); + } finally { + consoleCaptor.close(); + System.out.println("--- Captured console output:"); + consoleCaptor.getStandardOutput().forEach(System.out::println); + consoleCaptor.getErrorOutput().forEach(System.err::println); + System.out.println("--- End of captured console output"); + } + } +} diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java index 3e598a57277a2..937526629acf0 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java @@ -27,7 +27,7 @@ public class ProxyServiceStarterDisableZeroCopyTest extends ProxyServiceStarterT @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS); + serviceStarter = new ProxyServiceStarter(ARGS, null, true); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); serviceStarter.getConfig().setWebServicePort(Optional.of(0)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java index f263286125353..0b9b6f17d1254 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java @@ -54,7 +54,7 @@ public class ProxyServiceStarterTest extends MockedPulsarServiceBaseTest { @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS); + serviceStarter = new ProxyServiceStarter(ARGS, null, true); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); serviceStarter.getConfig().setWebServicePort(Optional.of(0)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java index 61718bbac3ab0..770424d93747c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java @@ -55,7 +55,7 @@ public class ProxyServiceTlsStarterTest extends MockedPulsarServiceBaseTest { @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS); + serviceStarter = new ProxyServiceStarter(ARGS, null, true); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerServiceURLTLS(pulsar.getBrokerServiceUrlTls()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); diff --git a/pulsar-proxy/src/test/resources/log4j2.xml b/pulsar-proxy/src/test/resources/log4j2.xml new file mode 100644 index 0000000000000..261bd2edf6980 --- /dev/null +++ b/pulsar-proxy/src/test/resources/log4j2.xml @@ -0,0 +1,36 @@ + + + + + + + + + + + + + + + diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java index 7aed43d056c67..bbb34a3e3f73d 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java @@ -35,10 +35,17 @@ import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.HttpConfiguration; +import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerCollection; @@ -73,10 +80,22 @@ public ProxyServer(WebSocketProxyConfiguration config) if (config.getMaxHttpServerConnections() > 0) { server.addBean(new ConnectionLimit(config.getMaxHttpServerConnections(), server)); } + + HttpConfiguration httpConfig = new HttpConfiguration(); + if (config.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); + List connectors = new ArrayList<>(); if (config.getWebServicePort().isPresent()) { - connector = new ServerConnector(server); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + connector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connector.setPort(config.getWebServicePort().get()); connectors.add(connector); } @@ -111,7 +130,18 @@ public ProxyServer(WebSocketProxyConfiguration config) config.getWebServiceTlsProtocols(), config.getTlsCertRefreshCheckDurationSec()); } - connectorTls = new ServerConnector(server, sslCtxFactory); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + connectorTls = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connectorTls.setPort(config.getWebServicePortTls().get()); connectors.add(connectorTls); } catch (Exception e) { @@ -169,7 +199,10 @@ public void start() throws PulsarServerException { .map(ServerConnector.class::cast).map(ServerConnector::getPort).map(Object::toString) .collect(Collectors.joining(","))); RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = conf.getWebServiceLogDetailedAddresses() != null + ? conf.getWebServiceLogDetailedAddresses() : + (conf.isWebServiceHaProxyProtocolEnabled() || conf.isWebServiceTrustXForwardedFor()); + requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java index 7acfd4a64ad35..3fcbcf4b21567 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java @@ -96,6 +96,20 @@ public class WebSocketProxyConfiguration implements PulsarConfiguration { @FieldContext(doc = "Hostname or IP address the service binds on, default is 0.0.0.0.") private String bindAddress = "0.0.0.0"; + @FieldContext(doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(doc = "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext(doc = "Maximum size of a text message during parsing in WebSocket proxy") private int webSocketMaxTextFrameSize = 1024 * 1024; // --- Authentication --- diff --git a/structured-event-log/pom.xml b/structured-event-log/pom.xml index d09637fca76ae..cc34f921dc2bb 100644 --- a/structured-event-log/pom.xml +++ b/structured-event-log/pom.xml @@ -51,7 +51,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl test diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java index 1f29e19f01873..ee7497010adfc 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java @@ -403,9 +403,10 @@ public void testIsolationPolicy() throws Exception { () -> { try { admin.lookups().lookupTopicAsync(topic).get(5, TimeUnit.SECONDS); + fail(); } catch (Exception ex) { log.error("Failed to lookup topic: ", ex); - assertThat(ex.getMessage()).contains("Failed to select the new owner broker for bundle"); + assertThat(ex.getMessage()).contains("Service Unavailable"); } } ); diff --git a/tiered-storage/file-system/pom.xml b/tiered-storage/file-system/pom.xml index 040d17a8b40da..8df8aa21c42f6 100644 --- a/tiered-storage/file-system/pom.xml +++ b/tiered-storage/file-system/pom.xml @@ -48,7 +48,7 @@ org.slf4j - slf4j-log4j12 + * @@ -118,6 +118,10 @@ org.bouncycastle * + + org.slf4j + * +