diff --git a/conf/bookkeeper.conf b/conf/bookkeeper.conf index 8503694cb7c21..7d1c67eb88249 100644 --- a/conf/bookkeeper.conf +++ b/conf/bookkeeper.conf @@ -592,7 +592,7 @@ compactionRateByBytes=1000000 # enableStatistics=true # Stats Provider Class (if statistics are enabled) -statsProviderClass=org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider +statsProviderClass=org.apache.pulsar.metrics.prometheus.bookkeeper.PrometheusMetricsProvider # Default port for Prometheus metrics exporter prometheusStatsHttpPort=8000 diff --git a/conf/zookeeper.conf b/conf/zookeeper.conf index db85e688e5c60..c497ff77008a4 100644 --- a/conf/zookeeper.conf +++ b/conf/zookeeper.conf @@ -76,6 +76,6 @@ portUnification=false ## Metrics Providers # # https://prometheus.io Metrics Exporter -metricsProvider.className=org.apache.zookeeper.metrics.prometheus.PrometheusMetricsProvider +metricsProvider.className=org.apache.pulsar.metrics.prometheus.zookeeper.PrometheusMetricsProvider metricsProvider.httpPort=8000 -metricsProvider.exportJvmInfo=true +metricsProvider.exportJvmInfo=true \ No newline at end of file diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 86698baa11c66..339b402c9808e 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -82,29 +82,21 @@ - jline - jline - ${jline.version} + ${project.groupId} + pulsar-bookkeeper-prometheus-metrics-provider + ${project.version} - org.apache.zookeeper - zookeeper-prometheus-metrics - ${zookeeper.version} - - - ch.qos.logback - logback-core - - - ch.qos.logback - logback-classic - - - io.netty - netty-tcnative - - + ${project.groupId} + pulsar-zookeeper-prometheus-metrics + ${project.version} + + + + jline + jline + ${jline.version} @@ -141,6 +133,12 @@ ${project.groupId} pulsar-testclient ${project.version} + + + org.apache.zookeeper + zookeeper + + @@ -199,11 +197,6 @@ log4j-slf4j2-impl - - org.apache.bookkeeper.stats - prometheus-metrics-provider - - io.prometheus simpleclient_log4j2 diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index a2c0cc8fff773..0aaf02e3f4668 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -330,7 +330,6 @@ The Apache Software License, Version 2.0 - io.prometheus-simpleclient_common-0.16.0.jar - io.prometheus-simpleclient_hotspot-0.16.0.jar - io.prometheus-simpleclient_httpserver-0.16.0.jar - - io.prometheus-simpleclient_jetty-0.16.0.jar - io.prometheus-simpleclient_log4j2-0.16.0.jar - io.prometheus-simpleclient_servlet-0.16.0.jar - io.prometheus-simpleclient_servlet_common-0.16.0.jar @@ -354,8 +353,8 @@ The Apache Software License, Version 2.0 - org.apache.logging.log4j-log4j-web-2.25.3.jar - org.apache.logging.log4j-log4j-layout-template-json-2.25.3.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 + - net.java.dev.jna-jna-jpms-5.18.1.jar + - net.java.dev.jna-jna-platform-jpms-5.18.1.jar * BookKeeper - org.apache.bookkeeper-bookkeeper-common-4.17.2.jar - org.apache.bookkeeper-bookkeeper-common-allocator-4.17.2.jar @@ -376,12 +375,10 @@ The Apache Software License, Version 2.0 - org.apache.bookkeeper.http-http-server-4.17.2.jar - org.apache.bookkeeper.http-vertx-http-server-4.17.2.jar - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.17.2.jar - - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.17.2.jar - org.apache.distributedlog-distributedlog-common-4.17.2.jar - org.apache.distributedlog-distributedlog-core-4.17.2-tests.jar - org.apache.distributedlog-distributedlog-core-4.17.2.jar - org.apache.distributedlog-distributedlog-protocol-4.17.2.jar - - org.apache.bookkeeper.stats-codahale-metrics-provider-4.17.2.jar - org.apache.bookkeeper-bookkeeper-slogger-api-4.17.2.jar - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.17.2.jar - org.apache.bookkeeper-native-io-4.17.2.jar @@ -394,25 +391,43 @@ The Apache Software License, Version 2.0 - org.asynchttpclient-async-http-client-2.12.4.jar - org.asynchttpclient-async-http-client-netty-utils-2.12.4.jar * Jetty - - org.eclipse.jetty-jetty-client-9.4.58.v20250814.jar - - org.eclipse.jetty-jetty-continuation-9.4.58.v20250814.jar - - org.eclipse.jetty-jetty-http-9.4.58.v20250814.jar - - org.eclipse.jetty-jetty-io-9.4.58.v20250814.jar - - org.eclipse.jetty-jetty-proxy-9.4.58.v20250814.jar - - org.eclipse.jetty-jetty-security-9.4.58.v20250814.jar - - org.eclipse.jetty-jetty-server-9.4.58.v20250814.jar - - org.eclipse.jetty-jetty-servlet-9.4.58.v20250814.jar - - org.eclipse.jetty-jetty-servlets-9.4.58.v20250814.jar - - org.eclipse.jetty-jetty-util-9.4.58.v20250814.jar - - org.eclipse.jetty-jetty-util-ajax-9.4.58.v20250814.jar - - org.eclipse.jetty.websocket-javax-websocket-client-impl-9.4.58.v20250814.jar - - org.eclipse.jetty.websocket-websocket-api-9.4.58.v20250814.jar - - org.eclipse.jetty.websocket-websocket-client-9.4.58.v20250814.jar - - org.eclipse.jetty.websocket-websocket-common-9.4.58.v20250814.jar - - org.eclipse.jetty.websocket-websocket-server-9.4.58.v20250814.jar - - org.eclipse.jetty.websocket-websocket-servlet-9.4.58.v20250814.jar - - org.eclipse.jetty-jetty-alpn-conscrypt-server-9.4.58.v20250814.jar - - org.eclipse.jetty-jetty-alpn-server-9.4.58.v20250814.jar + - org.eclipse.jetty-jetty-alpn-client-12.1.5.jar + - org.eclipse.jetty-jetty-alpn-conscrypt-server-12.1.5.jar + - org.eclipse.jetty-jetty-alpn-server-12.1.5.jar + - org.eclipse.jetty-jetty-annotations-12.1.5.jar + - org.eclipse.jetty-jetty-client-12.1.5.jar + - org.eclipse.jetty-jetty-http-12.1.5.jar + - org.eclipse.jetty-jetty-io-12.1.5.jar + - org.eclipse.jetty-jetty-jndi-12.1.5.jar + - org.eclipse.jetty-jetty-plus-12.1.5.jar + - org.eclipse.jetty-jetty-security-12.1.5.jar + - org.eclipse.jetty-jetty-server-12.1.5.jar + - org.eclipse.jetty-jetty-session-12.1.5.jar + - org.eclipse.jetty-jetty-util-12.1.5.jar + - org.eclipse.jetty-jetty-xml-12.1.5.jar + - org.eclipse.jetty.compression-jetty-compression-common-12.1.5.jar + - org.eclipse.jetty.compression-jetty-compression-gzip-12.1.5.jar + - org.eclipse.jetty.compression-jetty-compression-server-12.1.5.jar + - org.eclipse.jetty.ee-jetty-ee-webapp-12.1.5.jar + - org.eclipse.jetty.ee8-jetty-ee8-annotations-12.1.5.jar + - org.eclipse.jetty.ee8-jetty-ee8-nested-12.1.5.jar + - org.eclipse.jetty.ee8-jetty-ee8-plus-12.1.5.jar + - org.eclipse.jetty.ee8-jetty-ee8-proxy-12.1.5.jar + - org.eclipse.jetty.ee8-jetty-ee8-security-12.1.5.jar + - org.eclipse.jetty.ee8-jetty-ee8-servlet-12.1.5.jar + - org.eclipse.jetty.ee8-jetty-ee8-servlets-12.1.5.jar + - org.eclipse.jetty.ee8-jetty-ee8-webapp-12.1.5.jar + - org.eclipse.jetty.ee8.websocket-jetty-ee8-websocket-jetty-api-12.1.5.jar + - org.eclipse.jetty.ee8.websocket-jetty-ee8-websocket-jetty-common-12.1.5.jar + - org.eclipse.jetty.ee8.websocket-jetty-ee8-websocket-jetty-server-12.1.5.jar + - org.eclipse.jetty.ee8.websocket-jetty-ee8-websocket-servlet-12.1.5.jar + - org.eclipse.jetty.toolchain-jetty-servlet-api-4.0.6.jar + - org.eclipse.jetty.websocket-jetty-websocket-core-client-12.1.5.jar + - org.eclipse.jetty.websocket-jetty-websocket-core-common-12.1.5.jar + - org.eclipse.jetty.websocket-jetty-websocket-core-server-12.1.5.jar + - org.eclipse.jetty.websocket-jetty-websocket-jetty-api-12.1.5.jar + - org.eclipse.jetty.websocket-jetty-websocket-jetty-client-12.1.5.jar + - org.eclipse.jetty.websocket-jetty-websocket-jetty-common-12.1.5.jar * SnakeYaml -- org.yaml-snakeyaml-2.0.jar * RocksDB - org.rocksdb-rocksdbjni-7.9.2.jar * Google Error Prone Annotations - com.google.errorprone-error_prone_annotations-2.45.0.jar @@ -479,7 +494,6 @@ The Apache Software License, Version 2.0 - io.dropwizard.metrics-metrics-core-4.1.12.1.jar - io.dropwizard.metrics-metrics-graphite-4.1.12.1.jar - io.dropwizard.metrics-metrics-jvm-4.1.12.1.jar - - io.dropwizard.metrics-metrics-jmx-4.1.12.1.jar * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia @@ -500,9 +514,7 @@ The Apache Software License, Version 2.0 - io.vertx-vertx-web-4.5.22.jar - io.vertx-vertx-web-common-4.5.22.jar * Apache ZooKeeper - - org.apache.zookeeper-zookeeper-3.9.4.jar - org.apache.zookeeper-zookeeper-jute-3.9.4.jar - - org.apache.zookeeper-zookeeper-prometheus-metrics-3.9.4.jar * Snappy Java - org.xerial.snappy-snappy-java-1.1.10.5.jar * Google HTTP Client @@ -555,6 +567,10 @@ 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 + * OW2 ASM + - org.ow2.asm-asm-9.9.jar -- ../licenses/LICENSE-ASM.txt + - org.ow2.asm-asm-commons-9.9.jar -- ../licenses/LICENSE-ASM.txt + - org.ow2.asm-asm-tree-9.9.jar -- ../licenses/LICENSE-ASM.txt BSD 2-Clause License * HdrHistogram -- org.hdrhistogram-HdrHistogram-2.1.9.jar -- ../licenses/LICENSE-HdrHistogram.txt @@ -580,7 +596,6 @@ CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API - com.sun.activation-jakarta.activation-1.2.2.jar * Java Servlet API -- javax.servlet-javax.servlet-api-3.1.0.jar - * WebSocket Server API -- javax.websocket-javax.websocket-client-api-1.0.jar * HK2 - Dependency Injection Kernel - org.glassfish.hk2-hk2-api-2.6.1.jar - org.glassfish.hk2-hk2-locator-2.6.1.jar @@ -608,6 +623,7 @@ Eclipse Public License - v2.0 -- ../licenses/LICENSE-EPL-2.0.txt * Jakarta Annotations API -- jakarta.annotation-jakarta.annotation-api-1.3.5.jar * Jakarta RESTful Web Services -- jakarta.ws.rs-jakarta.ws.rs-api-2.1.6.jar * Jakarta Injection -- org.glassfish.hk2.external-jakarta.inject-2.6.1.jar + * Jakarta Transactions API -- jakarta.transaction-jakarta.transaction-api-1.3.3.jar Public Domain (CC0) -- ../licenses/LICENSE-CC0.txt * Reactive Streams -- org.reactivestreams-reactive-streams-1.0.3.jar diff --git a/distribution/server/src/assemble/bin.xml b/distribution/server/src/assemble/bin.xml index 4dfec015c0e6f..ea359c973638b 100644 --- a/distribution/server/src/assemble/bin.xml +++ b/distribution/server/src/assemble/bin.xml @@ -128,6 +128,8 @@ com.google.android:annotations net.java.dev.jna:jna + + org.apache.zookeeper:zookeeper diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 32f2078f9d0a0..9a0a74764c305 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -403,14 +403,18 @@ The Apache Software License, Version 2.0 - async-http-client-2.12.4.jar - async-http-client-netty-utils-2.12.4.jar * Jetty - - jetty-client-9.4.58.v20250814.jar - - jetty-http-9.4.58.v20250814.jar - - jetty-io-9.4.58.v20250814.jar - - jetty-util-9.4.58.v20250814.jar - - javax-websocket-client-impl-9.4.58.v20250814.jar - - websocket-api-9.4.58.v20250814.jar - - websocket-client-9.4.58.v20250814.jar - - websocket-common-9.4.58.v20250814.jar + - jetty-alpn-client-12.1.5.jar + - jetty-client-12.1.5.jar + - jetty-compression-common-12.1.5.jar + - jetty-compression-gzip-12.1.5.jar + - jetty-http-12.1.5.jar + - jetty-io-12.1.5.jar + - jetty-util-12.1.5.jar + - jetty-websocket-core-client-12.1.5.jar + - jetty-websocket-core-common-12.1.5.jar + - jetty-websocket-jetty-api-12.1.5.jar + - jetty-websocket-jetty-client-12.1.5.jar + - jetty-websocket-jetty-common-12.1.5.jar * SnakeYaml -- snakeyaml-2.0.jar * Google Error Prone Annotations - error_prone_annotations-2.45.0.jar * Javassist -- javassist-3.25.0-GA.jar @@ -433,7 +437,6 @@ MIT License CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API - jakarta.activation-1.2.2.jar - * WebSocket Server API -- javax.websocket-client-api-1.0.jar * HK2 - Dependency Injection Kernel - hk2-api-2.6.1.jar - hk2-locator-2.6.1.jar diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/pom.xml b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/pom.xml new file mode 100644 index 0000000000000..b181cc70c3ee9 --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/pom.xml @@ -0,0 +1,85 @@ + + + 4.0.0 + + org.apache.pulsar + jetty-upgrade + 4.2.0-SNAPSHOT + + pulsar-bookkeeper-prometheus-metrics-provider + Apache Pulsar :: BookKeeper Stats Providers :: Prometheus + + + org.apache.bookkeeper.stats + bookkeeper-stats-api + ${bookkeeper.version} + + + + io.prometheus + simpleclient + + + + io.prometheus + simpleclient_hotspot + + + + io.prometheus + simpleclient_servlet + + + + io.netty + netty-common + + + io.netty + netty-buffer + test + + + org.eclipse.jetty.ee8 + jetty-ee8-servlet + + + + + com.google.guava + guava + + + + com.yahoo.datasketches + sketches-core + + + + org.apache.bookkeeper + testtools + ${bookkeeper.version} + test + + + diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/DataSketchesOpStatsLogger.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/DataSketchesOpStatsLogger.java new file mode 100644 index 0000000000000..5c5bf560799a6 --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/DataSketchesOpStatsLogger.java @@ -0,0 +1,222 @@ +/* + * 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.metrics.prometheus.bookkeeper; + +import com.yahoo.sketches.quantiles.DoublesSketch; +import com.yahoo.sketches.quantiles.DoublesSketchBuilder; +import com.yahoo.sketches.quantiles.DoublesUnion; +import com.yahoo.sketches.quantiles.DoublesUnionBuilder; +import io.netty.util.concurrent.FastThreadLocal; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.LongAdder; +import java.util.concurrent.locks.StampedLock; +import org.apache.bookkeeper.stats.OpStatsData; +import org.apache.bookkeeper.stats.OpStatsLogger; + +/** + * OpStatsLogger implementation that uses DataSketches library to calculate the approximated latency quantiles. + */ +public class DataSketchesOpStatsLogger implements OpStatsLogger { + + /* + * Use 2 rotating thread local accessor so that we can safely swap them. + */ + private volatile ThreadLocalAccessor current; + private volatile ThreadLocalAccessor replacement; + + /* + * These are the sketches where all the aggregated results are published. + */ + private volatile DoublesSketch successResult; + private volatile DoublesSketch failResult; + + private final LongAdder successCountAdder = new LongAdder(); + private final LongAdder failCountAdder = new LongAdder(); + + private final LongAdder successSumAdder = new LongAdder(); + private final LongAdder failSumAdder = new LongAdder(); + + private Map labels; + + // used for lazy registration for thread scoped metrics + private boolean threadInitialized; + + public DataSketchesOpStatsLogger(Map labels) { + this.current = new ThreadLocalAccessor(); + this.replacement = new ThreadLocalAccessor(); + this.labels = labels; + } + + @Override + public void registerFailedEvent(long eventLatency, TimeUnit unit) { + double valueMillis = unit.toMicros(eventLatency) / 1000.0; + + failCountAdder.increment(); + failSumAdder.add((long) valueMillis); + + LocalData localData = current.localData.get(); + + long stamp = localData.lock.readLock(); + try { + localData.failSketch.update(valueMillis); + } finally { + localData.lock.unlockRead(stamp); + } + } + + @Override + public void registerSuccessfulEvent(long eventLatency, TimeUnit unit) { + double valueMillis = unit.toMicros(eventLatency) / 1000.0; + + successCountAdder.increment(); + successSumAdder.add((long) valueMillis); + + LocalData localData = current.localData.get(); + + long stamp = localData.lock.readLock(); + try { + localData.successSketch.update(valueMillis); + } finally { + localData.lock.unlockRead(stamp); + } + } + + @Override + public void registerSuccessfulValue(long value) { + successCountAdder.increment(); + successSumAdder.add(value); + + LocalData localData = current.localData.get(); + + long stamp = localData.lock.readLock(); + try { + localData.successSketch.update(value); + } finally { + localData.lock.unlockRead(stamp); + } + } + + @Override + public void registerFailedValue(long value) { + failCountAdder.increment(); + failSumAdder.add(value); + + LocalData localData = current.localData.get(); + + long stamp = localData.lock.readLock(); + try { + localData.failSketch.update(value); + } finally { + localData.lock.unlockRead(stamp); + } + } + + @Override + public OpStatsData toOpStatsData() { + // Not relevant as we don't use JMX here + throw new UnsupportedOperationException(); + } + + @Override + public void clear() { + // Not relevant as we don't use JMX here + throw new UnsupportedOperationException(); + } + + public void rotateLatencyCollection() { + // Swap current with replacement + ThreadLocalAccessor local = current; + current = replacement; + replacement = local; + + final DoublesUnion aggregateSuccesss = new DoublesUnionBuilder().build(); + final DoublesUnion aggregateFail = new DoublesUnionBuilder().build(); + local.map.forEach((localData, b) -> { + long stamp = localData.lock.writeLock(); + try { + aggregateSuccesss.update(localData.successSketch); + localData.successSketch.reset(); + aggregateFail.update(localData.failSketch); + localData.failSketch.reset(); + } finally { + localData.lock.unlockWrite(stamp); + } + }); + + successResult = aggregateSuccesss.getResultAndReset(); + failResult = aggregateFail.getResultAndReset(); + } + + public long getCount(boolean success) { + return success ? successCountAdder.sum() : failCountAdder.sum(); + } + + public long getSum(boolean success) { + return success ? successSumAdder.sum() : failSumAdder.sum(); + } + + public double getQuantileValue(boolean success, double quantile) { + DoublesSketch s = success ? successResult : failResult; + return s != null ? s.getQuantile(quantile) : Double.NaN; + } + + public Map getLabels() { + return labels; + } + + public boolean isThreadInitialized() { + return threadInitialized; + } + + public void initializeThread(Map labels) { + this.labels = labels; + this.threadInitialized = true; + } + + private static class LocalData { + private final DoublesSketch successSketch = new DoublesSketchBuilder().build(); + private final DoublesSketch failSketch = new DoublesSketchBuilder().build(); + private final StampedLock lock = new StampedLock(); + } + + private static class ThreadLocalAccessor { + private final Map map = new ConcurrentHashMap<>(); + private final FastThreadLocal localData = new FastThreadLocal() { + + @Override + protected LocalData initialValue() throws Exception { + LocalData localData = new LocalData(); + map.put(localData, Boolean.TRUE); + return localData; + } + + @Override + protected void onRemoval(LocalData value) throws Exception { + map.remove(value); + } + }; + } + + @Override + public String toString() { + return "DataSketchesOpStatsLogger{labels=" + labels + ", id=" + System.identityHashCode(this) + "}"; + } +} diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/LongAdderCounter.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/LongAdderCounter.java new file mode 100644 index 0000000000000..a5a8e4d2f029f --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/LongAdderCounter.java @@ -0,0 +1,91 @@ +/* + * 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.metrics.prometheus.bookkeeper; + +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.LongAdder; +import org.apache.bookkeeper.stats.Counter; + +/** + * {@link Counter} implementation based on {@link LongAdder}. + * + *

LongAdder keeps a counter per-thread and then aggregates to get the result, in order to avoid contention between + * multiple threads. + */ +public class LongAdderCounter implements Counter { + private final LongAdder counter = new LongAdder(); + + private Map labels; + + // used for lazy registration for thread scoped metric + private boolean threadInitialized; + + public LongAdderCounter(Map labels) { + this.labels = labels; + } + + @Override + public void clear() { + counter.reset(); + } + + @Override + public void inc() { + counter.increment(); + } + + @Override + public void dec() { + counter.decrement(); + } + + @Override + public void addCount(long delta) { + counter.add(delta); + } + + /** + * When counter is used to count time. + * consistent with the {@link DataSketchesOpStatsLogger#registerSuccessfulEvent(long, TimeUnit)} 's logic + * */ + @Override + public void addLatency(long eventLatency, TimeUnit unit) { + long valueMillis = unit.toMillis(eventLatency); + counter.add(valueMillis); + } + + @Override + public Long get() { + return counter.sum(); + } + + public Map getLabels() { + return labels; + } + + public boolean isThreadInitialized() { + return threadInitialized; + } + + public void initializeThread(Map labels) { + this.labels = labels; + this.threadInitialized = true; + } +} diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusMetricsProvider.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusMetricsProvider.java new file mode 100644 index 0000000000000..e5568625328f4 --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusMetricsProvider.java @@ -0,0 +1,253 @@ +/* + * 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.metrics.prometheus.bookkeeper; + +// CHECKSTYLE.OFF: IllegalImport + +import com.google.common.annotations.VisibleForTesting; +import io.netty.util.concurrent.DefaultThreadFactory; +import io.netty.util.internal.PlatformDependent; +import io.prometheus.client.Collector; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.Gauge; +import io.prometheus.client.Gauge.Child; +import io.prometheus.client.hotspot.GarbageCollectorExports; +import io.prometheus.client.hotspot.MemoryPoolsExports; +import io.prometheus.client.hotspot.StandardExports; +import io.prometheus.client.hotspot.ThreadExports; +import java.io.IOException; +import java.io.Writer; +import java.lang.management.BufferPoolMXBean; +import java.lang.management.ManagementFactory; +import java.lang.reflect.Field; +import java.net.InetSocketAddress; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Supplier; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.StatsProvider; +import org.apache.bookkeeper.stats.ThreadRegistry; +import org.apache.commons.configuration2.Configuration; +import org.apache.commons.lang3.StringUtils; +import org.eclipse.jetty.ee8.servlet.ServletContextHandler; +import org.eclipse.jetty.ee8.servlet.ServletHolder; +import org.eclipse.jetty.server.Server; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +// CHECKSTYLE.ON: IllegalImport + +/** + * A Prometheus based {@link StatsProvider} implementation. + */ +public class PrometheusMetricsProvider implements StatsProvider { + + private ScheduledExecutorService executor; + + public static final String PROMETHEUS_STATS_HTTP_ENABLE = "prometheusStatsHttpEnable"; + public static final boolean DEFAULT_PROMETHEUS_STATS_HTTP_ENABLE = true; + + public static final String PROMETHEUS_STATS_HTTP_ADDRESS = "prometheusStatsHttpAddress"; + public static final String DEFAULT_PROMETHEUS_STATS_HTTP_ADDR = "0.0.0.0"; + + public static final String PROMETHEUS_STATS_HTTP_PORT = "prometheusStatsHttpPort"; + public static final int DEFAULT_PROMETHEUS_STATS_HTTP_PORT = 8000; + + public static final String PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS = "prometheusStatsLatencyRolloverSeconds"; + public static final int DEFAULT_PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS = 60; + + final CollectorRegistry registry; + + Server server; + + /* + * These acts a registry of the metrics defined in this provider + */ + final ConcurrentMap counters = new ConcurrentHashMap<>(); + final ConcurrentMap> gauges = new ConcurrentHashMap<>(); + final ConcurrentMap opStats = new ConcurrentHashMap<>(); + final ConcurrentMap threadScopedOpStats = + new ConcurrentHashMap<>(); + final ConcurrentMap threadScopedCounters = + new ConcurrentHashMap<>(); + + public PrometheusMetricsProvider() { + this(CollectorRegistry.defaultRegistry); + } + + public PrometheusMetricsProvider(CollectorRegistry registry) { + this.registry = registry; + } + + @Override + public void start(Configuration conf) { + boolean httpEnabled = conf.getBoolean(PROMETHEUS_STATS_HTTP_ENABLE, DEFAULT_PROMETHEUS_STATS_HTTP_ENABLE); + boolean bkHttpServerEnabled = conf.getBoolean("httpServerEnabled", false); + boolean exposeDefaultJVMMetrics = conf.getBoolean("exposeDefaultJVMMetrics", true); + // only start its own http server when prometheus http is enabled and bk http server is not enabled. + if (httpEnabled && !bkHttpServerEnabled) { + String httpAddr = conf.getString(PROMETHEUS_STATS_HTTP_ADDRESS, DEFAULT_PROMETHEUS_STATS_HTTP_ADDR); + int httpPort = conf.getInt(PROMETHEUS_STATS_HTTP_PORT, DEFAULT_PROMETHEUS_STATS_HTTP_PORT); + InetSocketAddress httpEndpoint = InetSocketAddress.createUnresolved(httpAddr, httpPort); + this.server = new Server(httpEndpoint); + ServletContextHandler context = new ServletContextHandler(); + context.setContextPath("/"); + server.setHandler(context); + + context.addServlet(new ServletHolder(new PrometheusServlet(this)), "/metrics"); + + try { + server.start(); + log.info("Started Prometheus stats endpoint at {}", httpEndpoint); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + if (exposeDefaultJVMMetrics) { + // Include standard JVM stats + registerMetrics(new StandardExports()); + registerMetrics(new MemoryPoolsExports()); + registerMetrics(new GarbageCollectorExports()); + registerMetrics(new ThreadExports()); + + // Add direct memory allocated through unsafe + registerMetrics(Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { + @Override + public double get() { + return getDirectMemoryUsage.get(); + } + })); + + registerMetrics(Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { + @Override + public double get() { + return PlatformDependent.estimateMaxDirectMemory(); + } + })); + } + + executor = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("metrics")); + + int latencyRolloverSeconds = conf.getInt(PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS, + DEFAULT_PROMETHEUS_STATS_LATENCY_ROLLOVER_SECONDS); + + executor.scheduleAtFixedRate(() -> { + rotateLatencyCollection(); + }, 1, latencyRolloverSeconds, TimeUnit.SECONDS); + + } + + @Override + public void stop() { + if (server != null) { + try { + server.stop(); + } catch (Exception e) { + log.warn("Failed to shutdown Jetty server", e); + } finally { + ThreadRegistry.clear(); + } + } + } + + @Override + public StatsLogger getStatsLogger(String scope) { + return new PrometheusStatsLogger(PrometheusMetricsProvider.this, scope, Collections.emptyMap()); + } + + @Override + public void writeAllMetrics(Writer writer) throws IOException { + PrometheusTextFormat prometheusTextFormat = new PrometheusTextFormat(); + PrometheusTextFormat.writeMetricsCollectedByPrometheusClient(writer, registry); + + gauges.forEach((sc, gauge) -> prometheusTextFormat.writeGauge(writer, sc.getScope(), gauge)); + counters.forEach((sc, counter) -> prometheusTextFormat.writeCounter(writer, sc.getScope(), counter)); + opStats.forEach((sc, opStatLogger) -> + prometheusTextFormat.writeOpStat(writer, sc.getScope(), opStatLogger)); + } + + @Override + public String getStatsName(String... statsComponents) { + String completeName; + if (statsComponents.length == 0) { + return ""; + } else if (statsComponents[0].isEmpty()) { + completeName = StringUtils.join(statsComponents, '_', 1, statsComponents.length); + } else { + completeName = StringUtils.join(statsComponents, '_'); + } + return Collector.sanitizeMetricName(completeName); + } + + @VisibleForTesting + void rotateLatencyCollection() { + opStats.forEach((name, metric) -> { + metric.rotateLatencyCollection(); + }); + } + + private void registerMetrics(Collector collector) { + try { + collector.register(registry); + } catch (Exception e) { + // Ignore if these were already registered + if (log.isDebugEnabled()) { + log.debug("Failed to register Prometheus collector exports", e); + } + } + } + + private static final Logger log = LoggerFactory.getLogger(PrometheusMetricsProvider.class); + + /* + * Try to get Netty counter of used direct memory. This will be correct, unlike the JVM values. + */ + private static final AtomicLong directMemoryUsage; + private static final Optional poolMxBeanOp; + private static final Supplier getDirectMemoryUsage; + + static { + if (PlatformDependent.useDirectBufferNoCleaner()) { + poolMxBeanOp = Optional.empty(); + AtomicLong tmpDirectMemoryUsage = null; + try { + Field field = PlatformDependent.class.getDeclaredField("DIRECT_MEMORY_COUNTER"); + field.setAccessible(true); + tmpDirectMemoryUsage = (AtomicLong) field.get(null); + } catch (Throwable t) { + log.warn("Failed to access netty DIRECT_MEMORY_COUNTER field {}", t.getMessage()); + } + directMemoryUsage = tmpDirectMemoryUsage; + getDirectMemoryUsage = () -> directMemoryUsage != null ? directMemoryUsage.get() : Double.NaN; + } else { + directMemoryUsage = null; + List platformMXBeans = ManagementFactory.getPlatformMXBeans(BufferPoolMXBean.class); + poolMxBeanOp = platformMXBeans.stream() + .filter(bufferPoolMXBean -> bufferPoolMXBean.getName().equals("direct")).findAny(); + getDirectMemoryUsage = () -> poolMxBeanOp.isPresent() ? poolMxBeanOp.get().getMemoryUsed() : Double.NaN; + } + } +} diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusServlet.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusServlet.java new file mode 100644 index 0000000000000..88c572d8c4db3 --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusServlet.java @@ -0,0 +1,60 @@ +/* + * 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.metrics.prometheus.bookkeeper; + +import io.prometheus.client.exporter.common.TextFormat; +import java.io.IOException; +import java.io.Writer; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +/** + * Servlet used to export metrics in prometheus text format. + */ +public class PrometheusServlet extends HttpServlet { + private static final long serialVersionUID = 1L; + + private final transient PrometheusMetricsProvider provider; + + public PrometheusServlet(PrometheusMetricsProvider provider) { + this.provider = provider; + } + + @Override + protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + resp.setStatus(HttpServletResponse.SC_OK); + resp.setContentType(TextFormat.CONTENT_TYPE_004); + + Writer writer = resp.getWriter(); + try { + provider.writeAllMetrics(writer); + writer.flush(); + } finally { + writer.close(); + } + } + + @Override + protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { + doGet(req, resp); + } + +} diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusStatsLogger.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusStatsLogger.java new file mode 100644 index 0000000000000..b3f8098590fe6 --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusStatsLogger.java @@ -0,0 +1,100 @@ +/* + * 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.metrics.prometheus.bookkeeper; + +import com.google.common.base.Joiner; +import io.prometheus.client.Collector; +import java.util.Map; +import java.util.TreeMap; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; + +/** + * A {@code Prometheus} based {@link StatsLogger} implementation. + */ +public class PrometheusStatsLogger implements StatsLogger { + + private final PrometheusMetricsProvider provider; + private final String scope; + private final Map labels; + + PrometheusStatsLogger(PrometheusMetricsProvider provider, String scope, Map labels) { + this.provider = provider; + this.scope = scope; + this.labels = labels; + } + + @Override + public OpStatsLogger getOpStatsLogger(String name) { + return provider.opStats.computeIfAbsent(scopeContext(name), x -> new DataSketchesOpStatsLogger(labels)); + } + + @Override + public OpStatsLogger getThreadScopedOpStatsLogger(String name) { + return provider.threadScopedOpStats.computeIfAbsent(scopeContext(name), + x -> new ThreadScopedDataSketchesStatsLogger(provider, x, labels)); + } + + @Override + public Counter getCounter(String name) { + return provider.counters.computeIfAbsent(scopeContext(name), x -> new LongAdderCounter(labels)); + } + + public Counter getThreadScopedCounter(String name) { + return provider.threadScopedCounters.computeIfAbsent(scopeContext(name), + x -> new ThreadScopedLongAdderCounter(provider, x, labels)); + } + + @Override + public void registerGauge(String name, Gauge gauge) { + provider.gauges.computeIfAbsent(scopeContext(name), x -> new SimpleGauge(gauge, labels)); + } + + @Override + public void unregisterGauge(String name, Gauge gauge) { + // no-op + } + + @Override + public void removeScope(String name, StatsLogger statsLogger) { + // no-op + } + + @Override + public StatsLogger scope(String name) { + return new PrometheusStatsLogger(provider, completeName(name), labels); + } + + @Override + public StatsLogger scopeLabel(String labelName, String labelValue) { + Map newLabels = new TreeMap<>(labels); + newLabels.put(labelName, labelValue); + return new PrometheusStatsLogger(provider, scope, newLabels); + } + + private ScopeContext scopeContext(String name) { + return new ScopeContext(completeName(name), labels); + } + + private String completeName(String name) { + return Collector.sanitizeMetricName(scope.isEmpty() ? name : Joiner.on('_').join(scope, name)); + } +} diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusTextFormat.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusTextFormat.java new file mode 100644 index 0000000000000..97ad0ea8fbef2 --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/PrometheusTextFormat.java @@ -0,0 +1,240 @@ +/* + * 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.metrics.prometheus.bookkeeper; + +import io.prometheus.client.Collector; +import io.prometheus.client.Collector.MetricFamilySamples; +import io.prometheus.client.Collector.MetricFamilySamples.Sample; +import io.prometheus.client.CollectorRegistry; +import java.io.IOException; +import java.io.Writer; +import java.util.Enumeration; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * Logic to write metrics in Prometheus text format. + */ +public class PrometheusTextFormat { + + Set metricNameSet = new HashSet<>(); + + void writeGauge(Writer w, String name, SimpleGauge gauge) { + // Example: + // # TYPE bookie_storage_entries_count gauge + // bookie_storage_entries_count 519 + try { + writeType(w, name, "gauge"); + w.append(name); + writeLabels(w, gauge.getLabels()); + w.append(' ').append(gauge.getSample().toString()).append('\n'); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + void writeCounter(Writer w, String name, LongAdderCounter counter) { + // Example: + // # TYPE jvm_threads_started_total counter + // jvm_threads_started_total 59 + try { + writeType(w, name, "counter"); + w.append(name); + writeLabels(w, counter.getLabels()); + w.append(' ').append(counter.get().toString()).append('\n'); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + void writeOpStat(Writer w, String name, DataSketchesOpStatsLogger opStat) { + // Example: + // # TYPE bookie_journal_JOURNAL_ADD_ENTRY summary + // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.5",} NaN + // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.75",} NaN + // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.95",} NaN + // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.99",} NaN + // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.999",} NaN + // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="0.9999",} NaN + // bookie_journal_JOURNAL_ADD_ENTRY{success="false",quantile="1.0",} NaN + // bookie_journal_JOURNAL_ADD_ENTRY_count{success="false",} 0.0 + // bookie_journal_JOURNAL_ADD_ENTRY_sum{success="false",} 0.0 + // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.5",} 1.706 + // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.75",} 1.89 + // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.95",} 2.121 + // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.99",} 10.708 + // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.999",} 10.902 + // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="0.9999",} 10.902 + // bookie_journal_JOURNAL_ADD_ENTRY{success="true",quantile="1.0",} 10.902 + // bookie_journal_JOURNAL_ADD_ENTRY_count{success="true",} 658.0 + // bookie_journal_JOURNAL_ADD_ENTRY_sum{success="true",} 1265.0800000000002 + try { + writeType(w, name, "summary"); + writeQuantile(w, opStat, name, false, 0.5); + writeQuantile(w, opStat, name, false, 0.75); + writeQuantile(w, opStat, name, false, 0.95); + writeQuantile(w, opStat, name, false, 0.99); + writeQuantile(w, opStat, name, false, 0.999); + writeQuantile(w, opStat, name, false, 0.9999); + writeQuantile(w, opStat, name, false, 1.0); + writeCount(w, opStat, name, false); + writeSum(w, opStat, name, false); + + writeQuantile(w, opStat, name, true, 0.5); + writeQuantile(w, opStat, name, true, 0.75); + writeQuantile(w, opStat, name, true, 0.95); + writeQuantile(w, opStat, name, true, 0.99); + writeQuantile(w, opStat, name, true, 0.999); + writeQuantile(w, opStat, name, true, 0.9999); + writeQuantile(w, opStat, name, true, 1.0); + writeCount(w, opStat, name, true); + writeSum(w, opStat, name, true); + + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private void writeLabels(Writer w, Map labels) throws IOException { + if (labels.isEmpty()) { + return; + } + + w.append('{'); + writeLabelsNoBraces(w, labels); + w.append('}'); + } + + private void writeLabelsNoBraces(Writer w, Map labels) throws IOException { + if (labels.isEmpty()) { + return; + } + + boolean isFirst = true; + for (Map.Entry e : labels.entrySet()) { + if (!isFirst) { + w.append(','); + } + isFirst = false; + w.append(e.getKey()) + .append("=\"") + .append(e.getValue()) + .append('"'); + } + } + + private void writeQuantile(Writer w, DataSketchesOpStatsLogger opStat, String name, Boolean success, + double quantile) throws IOException { + w.append(name) + .append("{success=\"").append(success.toString()) + .append("\",quantile=\"").append(Double.toString(quantile)) + .append("\""); + if (!opStat.getLabels().isEmpty()) { + w.append(", "); + writeLabelsNoBraces(w, opStat.getLabels()); + } + w.append("} ") + .append(Double.toString(opStat.getQuantileValue(success, quantile))).append('\n'); + } + + private void writeCount(Writer w, DataSketchesOpStatsLogger opStat, String name, Boolean success) + throws IOException { + w.append(name).append("_count{success=\"").append(success.toString()).append("\""); + if (!opStat.getLabels().isEmpty()) { + w.append(", "); + writeLabelsNoBraces(w, opStat.getLabels()); + } + w.append("} ") + .append(Long.toString(opStat.getCount(success))).append('\n'); + } + + private void writeSum(Writer w, DataSketchesOpStatsLogger opStat, String name, Boolean success) + throws IOException { + w.append(name).append("_sum{success=\"").append(success.toString()).append("\""); + if (!opStat.getLabels().isEmpty()) { + w.append(", "); + writeLabelsNoBraces(w, opStat.getLabels()); + } + w.append("} ") + .append(Double.toString(opStat.getSum(success))).append('\n'); + } + + static void writeMetricsCollectedByPrometheusClient(Writer w, CollectorRegistry registry) throws IOException { + Enumeration metricFamilySamples = registry.metricFamilySamples(); + while (metricFamilySamples.hasMoreElements()) { + MetricFamilySamples metricFamily = metricFamilySamples.nextElement(); + // Write type of metric + w.append("# TYPE ").append(metricFamily.name).append(getTypeNameSuffix(metricFamily.type)).append(' ') + .append(getTypeStr(metricFamily.type)).write('\n'); + + for (int i = 0; i < metricFamily.samples.size(); i++) { + Sample sample = metricFamily.samples.get(i); + w.write(sample.name); + w.write('{'); + for (int j = 0; j < sample.labelNames.size(); j++) { + if (j != 0) { + w.write(", "); + } + w.write(sample.labelNames.get(j)); + w.write("=\""); + w.write(sample.labelValues.get(j)); + w.write('"'); + } + + w.write("} "); + w.write(Collector.doubleToGoString(sample.value)); + w.write('\n'); + } + } + } + + static String getTypeNameSuffix(Collector.Type type) { + if (type.equals(Collector.Type.INFO)) { + return "_info"; + } + return ""; + } + + static String getTypeStr(Collector.Type type) { + switch (type) { + case COUNTER: + return "counter"; + case GAUGE: + case INFO: + return "gauge"; + case SUMMARY: + return "summary"; + case HISTOGRAM: + return "histogram"; + case UNKNOWN: + default: + return "unknown"; + } + } + + void writeType(Writer w, String name, String type) throws IOException { + if (metricNameSet.contains(name)) { + return; + } + metricNameSet.add(name); + w.append("# TYPE ").append(name).append(" ").append(type).append("\n"); + } + +} diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ScopeContext.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ScopeContext.java new file mode 100644 index 0000000000000..4f15c006569ef --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ScopeContext.java @@ -0,0 +1,56 @@ +/* + * 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.metrics.prometheus.bookkeeper; + +import java.util.Map; +import java.util.Objects; + +/** + * Holder for a scope and a set of associated labels. + */ +public class ScopeContext { + private final String scope; + private final Map labels; + + public ScopeContext(String scope, Map labels) { + this.scope = scope; + this.labels = labels; + } + + public String getScope() { + return scope; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ScopeContext that = (ScopeContext) o; + return Objects.equals(scope, that.scope) && Objects.equals(labels, that.labels); + } + + @Override + public int hashCode() { + return Objects.hash(scope, labels); + } +} diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/SimpleGauge.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/SimpleGauge.java new file mode 100644 index 0000000000000..384cc682c2ff3 --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/SimpleGauge.java @@ -0,0 +1,44 @@ +/* + * 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.metrics.prometheus.bookkeeper; + +import java.util.Map; +import org.apache.bookkeeper.stats.Gauge; + +/** + * A {@link Gauge} implementation that forwards on the value supplier. + */ +public class SimpleGauge { + + private final Map labels; + private final Gauge gauge; + + public SimpleGauge(final Gauge gauge, Map labels) { + this.gauge = gauge; + this.labels = labels; + } + + Number getSample() { + return gauge.getSample(); + } + + public Map getLabels() { + return labels; + } +} diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ThreadScopedDataSketchesStatsLogger.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ThreadScopedDataSketchesStatsLogger.java new file mode 100644 index 0000000000000..4d3d5662d9c9b --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ThreadScopedDataSketchesStatsLogger.java @@ -0,0 +1,137 @@ +/* + * 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.metrics.prometheus.bookkeeper; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.stats.OpStatsData; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.ThreadRegistry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * OpStatsLogger implementation that lazily registers OpStatsLoggers per thread + * with added labels for the threadpool/thresd name and thread no. + */ +public class ThreadScopedDataSketchesStatsLogger implements OpStatsLogger { + + private static Logger logger = LoggerFactory.getLogger(ThreadScopedDataSketchesStatsLogger.class); + + private ThreadLocal statsLoggers; + private DataSketchesOpStatsLogger defaultStatsLogger; + private Map originalLabels; + private ScopeContext scopeContext; + private PrometheusMetricsProvider provider; + + public ThreadScopedDataSketchesStatsLogger(PrometheusMetricsProvider provider, + ScopeContext scopeContext, + Map labels) { + this.provider = provider; + this.scopeContext = scopeContext; + this.originalLabels = labels; + this.defaultStatsLogger = new DataSketchesOpStatsLogger(labels); + + Map defaultLabels = new HashMap<>(labels); + defaultLabels.put("threadPool", "?"); + defaultLabels.put("thread", "?"); + this.defaultStatsLogger.initializeThread(defaultLabels); + + this.statsLoggers = ThreadLocal.withInitial(() -> { + return new DataSketchesOpStatsLogger(labels); + }); + } + + @Override + public void registerFailedEvent(long eventLatency, TimeUnit unit) { + getStatsLogger().registerFailedEvent(eventLatency, unit); + } + + @Override + public void registerSuccessfulEvent(long eventLatency, TimeUnit unit) { + getStatsLogger().registerSuccessfulEvent(eventLatency, unit); + } + + @Override + public void registerSuccessfulValue(long value) { + getStatsLogger().registerSuccessfulValue(value); + } + + @Override + public void registerFailedValue(long value) { + getStatsLogger().registerFailedValue(value); + } + + @Override + public OpStatsData toOpStatsData() { + // Not relevant as we don't use JMX here + throw new UnsupportedOperationException(); + } + + @Override + public void clear() { + // Not relevant as we don't use JMX here + throw new UnsupportedOperationException(); + } + + private DataSketchesOpStatsLogger getStatsLogger() { + DataSketchesOpStatsLogger statsLogger = statsLoggers.get(); + + // Lazy registration + // Update the stats logger with the thread labels then add to the provider + // If for some reason this thread did not get registered, + // then we fallback to a standard OpsStatsLogger (defaultStatsLogger) + if (!statsLogger.isThreadInitialized()) { + ThreadRegistry.ThreadPoolThread tpt = ThreadRegistry.get(); + if (tpt == null) { + logger.warn("Thread {} was not registered in the thread registry. Using default stats logger {}.", + Thread.currentThread(), defaultStatsLogger); + statsLoggers.set(defaultStatsLogger); + DataSketchesOpStatsLogger previous = provider.opStats + .put(new ScopeContext(scopeContext.getScope(), originalLabels), defaultStatsLogger); + // If we overwrite a logger, metrics will not be collected correctly + if (previous != null && previous != defaultStatsLogger) { + logger.error("Invalid state for thead " + Thread.currentThread() + ". Overwrote a stats logger." + + "New is {}, previous was {}", + defaultStatsLogger, previous); + throw new IllegalStateException("Invalid state. Overwrote a stats logger."); + } + return defaultStatsLogger; + } else { + Map threadScopedlabels = new HashMap<>(originalLabels); + threadScopedlabels.put("threadPool", tpt.getThreadPool()); + threadScopedlabels.put("thread", String.valueOf(tpt.getOrdinal())); + + statsLogger.initializeThread(threadScopedlabels); + DataSketchesOpStatsLogger previous = provider.opStats + .put(new ScopeContext(scopeContext.getScope(), threadScopedlabels), statsLogger); + // If we overwrite a logger, metrics will not be collected correctly + if (previous != null && previous != statsLogger) { + logger.error("Invalid state for thead " + Thread.currentThread() + ". Overwrote a stats logger." + + "New is {}, previous was {}", + defaultStatsLogger, previous); + throw new IllegalStateException("Invalid state. Overwrote a stats logger."); + } + } + } + + return statsLogger; + } +} \ No newline at end of file diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ThreadScopedLongAdderCounter.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ThreadScopedLongAdderCounter.java new file mode 100644 index 0000000000000..1c2287f5b9c85 --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/ThreadScopedLongAdderCounter.java @@ -0,0 +1,111 @@ +/* + * 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.metrics.prometheus.bookkeeper; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.ThreadRegistry; + +/** + * {@link Counter} implementation that lazily registers LongAdderCounters per thread + * * with added labels for the threadpool/thread name and thread no. + */ +public class ThreadScopedLongAdderCounter implements Counter { + private ThreadLocal counters; + private LongAdderCounter defaultCounter; + private Map originalLabels; + private ScopeContext scopeContext; + private PrometheusMetricsProvider provider; + + public ThreadScopedLongAdderCounter(PrometheusMetricsProvider provider, + ScopeContext scopeContext, + Map labels) { + this.provider = provider; + this.scopeContext = scopeContext; + this.originalLabels = new HashMap<>(labels); + this.defaultCounter = new LongAdderCounter(labels); + Map defaultLabels = new HashMap<>(labels); + defaultLabels.put("threadPool", "?"); + defaultLabels.put("thread", "?"); + this.defaultCounter.initializeThread(defaultLabels); + + this.counters = ThreadLocal.withInitial(() -> { + return new LongAdderCounter(labels); + }); + } + + @Override + public void clear() { + getCounter().clear(); + } + + @Override + public void inc() { + getCounter().inc(); + } + + @Override + public void dec() { + getCounter().dec(); + } + + @Override + public void addCount(long delta) { + getCounter().addCount(delta); + } + + @Override + public void addLatency(long eventLatency, TimeUnit unit) { + getCounter().addLatency(eventLatency, unit); + } + + @Override + public Long get() { + return getCounter().get(); + } + + private LongAdderCounter getCounter() { + LongAdderCounter counter = counters.get(); + + // Lazy registration + // Update the counter with the thread labels then add to the provider + // If for some reason this thread did not get registered, + // then we fallback to a standard counter (defaultCounter) + if (!counter.isThreadInitialized()) { + ThreadRegistry.ThreadPoolThread tpt = ThreadRegistry.get(); + + if (tpt == null) { + counters.set(defaultCounter); + provider.counters.put(new ScopeContext(scopeContext.getScope(), originalLabels), defaultCounter); + return defaultCounter; + } else { + Map threadScopedlabels = new HashMap<>(originalLabels); + threadScopedlabels.put("threadPool", tpt.getThreadPool()); + threadScopedlabels.put("thread", String.valueOf(tpt.getOrdinal())); + + counter.initializeThread(threadScopedlabels); + provider.counters.put(new ScopeContext(scopeContext.getScope(), threadScopedlabels), counter); + } + } + + return counter; + } +} diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/package-info.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/package-info.java new file mode 100644 index 0000000000000..fd5cf0e154a06 --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/main/java/org/apache/pulsar/metrics/prometheus/bookkeeper/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +/** + * A stats provider implementation for BookKeeper that uses Jetty 12. + */ +package org.apache.pulsar.metrics.prometheus.bookkeeper; diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/test/java/org/apache/pulsar/metrics/prometheus/bookkeeper/TestPrometheusFormatter.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/test/java/org/apache/pulsar/metrics/prometheus/bookkeeper/TestPrometheusFormatter.java new file mode 100644 index 0000000000000..fceee727270a6 --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/test/java/org/apache/pulsar/metrics/prometheus/bookkeeper/TestPrometheusFormatter.java @@ -0,0 +1,296 @@ +/* + * 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.metrics.prometheus.bookkeeper; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertTrue; +import static org.testng.AssertJUnit.fail; +import com.google.common.base.MoreObjects; +import com.google.common.base.Splitter; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.Gauge; +import io.prometheus.client.hotspot.GarbageCollectorExports; +import io.prometheus.client.hotspot.MemoryPoolsExports; +import io.prometheus.client.hotspot.StandardExports; +import io.prometheus.client.hotspot.ThreadExports; +import java.io.IOException; +import java.io.StringWriter; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.testng.annotations.Test; + +/** + * Test for {@link PrometheusMetricsProvider}. + */ +public class TestPrometheusFormatter { + + @Test(timeOut = 30000) + public void testStatsOutput() throws Exception { + PrometheusMetricsProvider provider = new PrometheusMetricsProvider(); + StatsLogger statsLogger = provider.getStatsLogger("test"); + Counter counter = statsLogger.getCounter("my_counter"); + + counter.inc(); + counter.inc(); + + OpStatsLogger opStats = statsLogger.getOpStatsLogger("op"); + opStats.registerSuccessfulEvent(10, TimeUnit.MILLISECONDS); + opStats.registerSuccessfulEvent(5, TimeUnit.MILLISECONDS); + + OpStatsLogger opStats1 = statsLogger.scopeLabel("test_label", "test_value") + .getOpStatsLogger("op_label"); + opStats1.registerSuccessfulEvent(10, TimeUnit.MILLISECONDS); + opStats1.registerSuccessfulEvent(5, TimeUnit.MILLISECONDS); + opStats1.registerFailedEvent(1, TimeUnit.MILLISECONDS); + + provider.rotateLatencyCollection(); + + StringWriter writer = new StringWriter(); + provider.writeAllMetrics(writer); + + writer.write("jvm_memory_direct_bytes_max{} 4.77626368E8\n"); + writer.write("jvm_memory_pool_bytes_used{pool=\"Code Cache\"} 3347712.0\n"); + writer.write("jvm_memory_pool_bytes_used{pool=\"CodeHeap 'non-nmethods'\"} 1207168.0\n"); + System.out.println(writer); + Multimap metrics = parseMetrics(writer.toString()); + System.out.println(metrics); + + List cm = (List) metrics.get("test_my_counter"); + assertEquals(1, cm.size()); + assertEquals(0, cm.get(0).tags.size()); + assertEquals(2.0, cm.get(0).value, 0.0); + + // test_op_sum + cm = (List) metrics.get("test_op_sum"); + assertEquals(2, cm.size()); + Metric m = cm.get(0); + assertEquals(1, cm.get(0).tags.size()); + assertEquals(0.0, m.value, 0.0); + assertEquals(1, m.tags.size()); + assertEquals("false", m.tags.get("success")); + + m = cm.get(1); + assertEquals(1, cm.get(0).tags.size()); + assertEquals(15.0, m.value, 0.0); + assertEquals(1, m.tags.size()); + assertEquals("true", m.tags.get("success")); + + // test_op_count + cm = (List) metrics.get("test_op_count"); + assertEquals(2, cm.size()); + m = cm.get(0); + assertEquals(1, cm.get(0).tags.size()); + assertEquals(0.0, m.value, 0.0); + assertEquals(1, m.tags.size()); + assertEquals("false", m.tags.get("success")); + + m = cm.get(1); + assertEquals(1, cm.get(0).tags.size()); + assertEquals(2.0, m.value, 0.0); + assertEquals(1, m.tags.size()); + assertEquals("true", m.tags.get("success")); + + // Latency + cm = (List) metrics.get("test_op"); + assertEquals(14, cm.size()); + + boolean found = false; + for (Metric mt : cm) { + if ("true".equals(mt.tags.get("success")) && "1.0".equals(mt.tags.get("quantile"))) { + assertEquals(10.0, mt.value, 0.0); + found = true; + } + } + + assertTrue(found); + + // test_op_label_sum + cm = (List) metrics.get("test_op_label_sum"); + assertEquals(2, cm.size()); + m = cm.get(0); + assertEquals(2, m.tags.size()); + assertEquals(1.0, m.value, 0.0); + assertEquals("false", m.tags.get("success")); + assertEquals("test_value", m.tags.get("test_label")); + + m = cm.get(1); + assertEquals(15.0, m.value, 0.0); + assertEquals(2, m.tags.size()); + assertEquals("true", m.tags.get("success")); + assertEquals("test_value", m.tags.get("test_label")); + + // test_op_label_count + cm = (List) metrics.get("test_op_label_count"); + assertEquals(2, cm.size()); + m = cm.get(0); + assertEquals(1, m.value, 0.0); + assertEquals(2, m.tags.size()); + assertEquals("false", m.tags.get("success")); + assertEquals("test_value", m.tags.get("test_label")); + + m = cm.get(1); + assertEquals(2.0, m.value, 0.0); + assertEquals(2, m.tags.size()); + assertEquals("true", m.tags.get("success")); + assertEquals("test_value", m.tags.get("test_label")); + + // Latency + cm = (List) metrics.get("test_op_label"); + assertEquals(14, cm.size()); + + found = false; + for (Metric mt : cm) { + if ("true".equals(mt.tags.get("success")) + && "test_value".equals(mt.tags.get("test_label")) + && "1.0".equals(mt.tags.get("quantile"))) { + assertEquals(10.0, mt.value, 0.0); + found = true; + } + } + + assertTrue(found); + } + + @Test + public void testWriteMetricsCollectedByPrometheusClient() { + CollectorRegistry registry = CollectorRegistry.defaultRegistry; + registry.register(new StandardExports()); + registry.register(new MemoryPoolsExports()); + registry.register(new GarbageCollectorExports()); + registry.register(new ThreadExports()); + registry.register(Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Gauge.Child() { + @Override + public double get() { + return 1.0; + } + })); + registry.register(Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Gauge.Child() { + @Override + public double get() { + return 100.0; + } + })); + PrometheusMetricsProvider provider = new PrometheusMetricsProvider(registry); + StringWriter writer = new StringWriter(); + try { + provider.rotateLatencyCollection(); + provider.writeAllMetrics(writer); + String output = writer.toString(); + parseMetrics(output); + assertTrue(output.contains("# TYPE jvm_memory_direct_bytes_max gauge")); + assertTrue(output.contains("# TYPE jvm_memory_direct_bytes_used gauge")); + assertTrue(output.contains("# TYPE jvm_gc_collection_seconds summary")); + assertTrue(output.contains("# TYPE jvm_memory_pool_bytes_committed gauge")); + assertTrue(output.contains("# TYPE process_cpu_seconds counter")); + } catch (Exception e) { + fail(); + } + + } + + @Test + public void testPrometheusTypeDuplicate() throws IOException { + PrometheusTextFormat prometheusTextFormat = new PrometheusTextFormat(); + StringWriter writer = new StringWriter(); + prometheusTextFormat.writeType(writer, "counter", "gauge"); + prometheusTextFormat.writeType(writer, "counter", "gauge"); + String string = writer.toString(); + assertEquals("# TYPE counter gauge\n", string); + } + + + /** + * Hacky parsing of Prometheus text format. Sould be good enough for unit tests + */ + private static Multimap parseMetrics(String metrics) { + Multimap parsed = ArrayListMultimap.create(); + + // Example of lines are + // jvm_threads_current{cluster="standalone",} 203.0 + // or + // pulsar_subscriptions_count{cluster="standalone", namespace="sample/standalone/ns1", + // topic="persistent://sample/standalone/ns1/test-2"} 0.0 1517945780897 + Pattern pattern = Pattern.compile("^(\\w+)(\\{([^\\}]*)\\})?\\s(-?[\\d\\w\\.]+)(\\s(\\d+))?$"); + Pattern formatPattern = + Pattern.compile("^(\\w+)(\\{((\\w+=[-\\s\\\'\\\"\\.\\w]+(,\\s?\\w+=[\\\"\\.\\w]+)*))?\\})?" + + "\\s(-?[\\d\\w\\.]+)(\\s(\\d+))?$"); + Pattern tagsPattern = Pattern.compile("(\\w+)=\"([^\"]+)\"(,\\s?)?"); + + Splitter.on("\n").split(metrics).forEach(line -> { + if (line.isEmpty() || line.startsWith("#")) { + return; + } + + System.err.println("LINE: '" + line + "'"); + Matcher matcher = pattern.matcher(line); + Matcher formatMatcher = formatPattern.matcher(line); + System.err.println("Matches: " + matcher.matches()); + System.err.println(matcher); + assertTrue(matcher.matches()); + assertTrue("failed to validate line: " + line, formatMatcher.matches()); + + assertEquals(6, matcher.groupCount()); + System.err.println("groups: " + matcher.groupCount()); + for (int i = 0; i < matcher.groupCount(); i++) { + System.err.println(" GROUP " + i + " -- " + matcher.group(i)); + } + + checkArgument(matcher.matches()); + checkArgument(formatMatcher.matches()); + String name = matcher.group(1); + + Metric m = new Metric(); + m.value = Double.parseDouble(matcher.group(4)); + + String tags = matcher.group(3); + if (tags != null) { + Matcher tagsMatcher = tagsPattern.matcher(tags); + while (tagsMatcher.find()) { + String tag = tagsMatcher.group(1); + String value = tagsMatcher.group(2); + m.tags.put(tag, value); + } + } + + parsed.put(name, m); + }); + + return parsed; + } + + static class Metric { + Map tags = new TreeMap<>(); + double value; + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("tags", tags).add("value", value).toString(); + } + } +} diff --git a/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/test/java/org/apache/pulsar/metrics/prometheus/bookkeeper/TestPrometheusMetricsProvider.java b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/test/java/org/apache/pulsar/metrics/prometheus/bookkeeper/TestPrometheusMetricsProvider.java new file mode 100644 index 0000000000000..a92616062c3c7 --- /dev/null +++ b/jetty-upgrade/bookkeeper-prometheus-metrics-provider/src/test/java/org/apache/pulsar/metrics/prometheus/bookkeeper/TestPrometheusMetricsProvider.java @@ -0,0 +1,166 @@ +/* + * 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.metrics.prometheus.bookkeeper; + +import static org.testng.Assert.assertNotEquals; +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertNotNull; +import static org.testng.AssertJUnit.assertNull; +import static org.testng.AssertJUnit.assertSame; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import java.io.StringWriter; +import java.util.Collections; +import java.util.HashMap; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.commons.configuration2.PropertiesConfiguration; +import org.testng.annotations.Test; + +/** + * Unit test of {@link PrometheusMetricsProvider}. + */ +public class TestPrometheusMetricsProvider { + + @Test + public void testStartNoHttp() { + PropertiesConfiguration config = new PropertiesConfiguration(); + config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, false); + PrometheusMetricsProvider provider = new PrometheusMetricsProvider(); + try { + provider.start(config); + assertNull(provider.server); + } finally { + provider.stop(); + } + } + + @Test + public void testStartNoHttpWhenBkHttpEnabled() { + PropertiesConfiguration config = new PropertiesConfiguration(); + config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, true); + config.setProperty("httpServerEnabled", true); + @Cleanup("stop") PrometheusMetricsProvider provider = new PrometheusMetricsProvider(); + provider.start(config); + assertNull(provider.server); + } + + @Test + public void testStartWithHttp() { + PropertiesConfiguration config = new PropertiesConfiguration(); + config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, true); + config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_PORT, 0); // ephemeral + PrometheusMetricsProvider provider = new PrometheusMetricsProvider(); + try { + provider.start(config); + assertNotNull(provider.server); + } finally { + provider.stop(); + } + } + + @Test + public void testStartWithHttpSpecifyAddr() { + PropertiesConfiguration config = new PropertiesConfiguration(); + config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, true); + config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_PORT, 0); // ephemeral + config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ADDRESS, "127.0.0.1"); + PrometheusMetricsProvider provider = new PrometheusMetricsProvider(); + try { + provider.start(config); + assertNotNull(provider.server); + } finally { + provider.stop(); + } + } + + @Test + public void testCounter() { + LongAdderCounter counter = new LongAdderCounter(Collections.emptyMap()); + long value = counter.get(); + assertEquals(0L, value); + counter.inc(); + assertEquals(1L, counter.get().longValue()); + counter.dec(); + assertEquals(0L, counter.get().longValue()); + counter.addCount(3); + assertEquals(3L, counter.get().longValue()); + } + + @Test + public void testCounter2() { + LongAdderCounter counter = new LongAdderCounter(Collections.emptyMap()); + long value = counter.get(); + assertEquals(0L, value); + counter.addLatency(3 * 1000 * 1000L, TimeUnit.NANOSECONDS); + assertEquals(3L, counter.get().longValue()); + } + + @Test + public void testTwoCounters() throws Exception { + PrometheusMetricsProvider provider = new PrometheusMetricsProvider(); + StatsLogger statsLogger = provider.getStatsLogger("test"); + + Counter counter1 = statsLogger.getCounter("counter"); + Counter counter2 = statsLogger.getCounter("counter"); + assertEquals(counter1, counter2); + assertSame(counter1, counter2); + + assertEquals(1, provider.counters.size()); + } + + @Test + public void testJvmDirectMemoryMetrics() throws Exception { + PropertiesConfiguration config = new PropertiesConfiguration(); + config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ENABLE, true); + config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_PORT, 0); + config.setProperty(PrometheusMetricsProvider.PROMETHEUS_STATS_HTTP_ADDRESS, "127.0.0.1"); + ByteBuf byteBuf = ByteBufAllocator.DEFAULT.directBuffer(25); + PrometheusMetricsProvider provider = new PrometheusMetricsProvider(); + try { + provider.start(config); + assertNotNull(provider.server); + StringWriter writer = new StringWriter(); + provider.writeAllMetrics(writer); + String s = writer.toString(); + String[] split = s.split(System.lineSeparator()); + HashMap map = new HashMap<>(); + for (String str : split) { + String[] aux = str.split(" "); + map.put(aux[0], aux[1]); + } + String directBytesMax = map.get("jvm_memory_direct_bytes_max{}"); + assertNotNull(directBytesMax); + assertNotEquals("Nan", directBytesMax); + assertNotEquals("-1", directBytesMax); + String directBytesUsed = map.get("jvm_memory_direct_bytes_used{}"); + assertNotNull(directBytesUsed); + assertNotEquals("Nan", directBytesUsed); + // this condition is flaky + //assertTrue(Double.parseDouble(directBytesUsed) > 25); + // ensure byteBuffer doesn't gc + byteBuf.release(); + } finally { + provider.stop(); + } + } + +} diff --git a/jetty-upgrade/pom.xml b/jetty-upgrade/pom.xml new file mode 100644 index 0000000000000..6c35a3b41e22c --- /dev/null +++ b/jetty-upgrade/pom.xml @@ -0,0 +1,39 @@ + + + 4.0.0 + pom + + org.apache.pulsar + pulsar + 4.2.0-SNAPSHOT + + + jetty-upgrade + Pulsar Jetty Upgrade :: Parent + + + bookkeeper-prometheus-metrics-provider + zookeeper-prometheus-metrics + zookeeper-with-patched-admin + + diff --git a/jetty-upgrade/zookeeper-prometheus-metrics/pom.xml b/jetty-upgrade/zookeeper-prometheus-metrics/pom.xml new file mode 100755 index 0000000000000..6adc8f628be27 --- /dev/null +++ b/jetty-upgrade/zookeeper-prometheus-metrics/pom.xml @@ -0,0 +1,94 @@ + + + 4.0.0 + + org.apache.pulsar + jetty-upgrade + 4.2.0-SNAPSHOT + + + pulsar-zookeeper-prometheus-metrics + jar + Apache Pulsar :: Apache ZooKeeper Prometheus Metrics Provider + ZooKeeper Prometheus.io Metrics Provider implementation for Jetty 12 + + + + org.apache.zookeeper + zookeeper + ${zookeeper.version} + provided + + + io.prometheus + simpleclient + + + io.prometheus + simpleclient_hotspot + + + io.prometheus + simpleclient_servlet + + + org.eclipse.jetty + jetty-server + provided + + + org.eclipse.jetty.ee8 + jetty-ee8-servlet + provided + + + org.mockito + mockito-core + test + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + package + + copy-dependencies + + + ${project.build.directory}/lib + false + true + false + + + + + + + diff --git a/jetty-upgrade/zookeeper-prometheus-metrics/src/main/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProvider.java b/jetty-upgrade/zookeeper-prometheus-metrics/src/main/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProvider.java new file mode 100644 index 0000000000000..7b1999004caf1 --- /dev/null +++ b/jetty-upgrade/zookeeper-prometheus-metrics/src/main/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProvider.java @@ -0,0 +1,632 @@ +/* + * 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.metrics.prometheus.zookeeper; + +import io.prometheus.client.Collector; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.exporter.MetricsServlet; +import io.prometheus.client.hotspot.DefaultExports; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Enumeration; +import java.util.Objects; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import org.apache.zookeeper.metrics.Counter; +import org.apache.zookeeper.metrics.CounterSet; +import org.apache.zookeeper.metrics.Gauge; +import org.apache.zookeeper.metrics.GaugeSet; +import org.apache.zookeeper.metrics.MetricsContext; +import org.apache.zookeeper.metrics.MetricsProvider; +import org.apache.zookeeper.metrics.MetricsProviderLifeCycleException; +import org.apache.zookeeper.metrics.Summary; +import org.apache.zookeeper.metrics.SummarySet; +import org.apache.zookeeper.server.RateLogger; +import org.eclipse.jetty.ee8.nested.ServletConstraint; +import org.eclipse.jetty.ee8.security.ConstraintMapping; +import org.eclipse.jetty.ee8.security.ConstraintSecurityHandler; +import org.eclipse.jetty.ee8.servlet.ServletContextHandler; +import org.eclipse.jetty.ee8.servlet.ServletHolder; +import org.eclipse.jetty.server.Server; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A Metrics Provider implementation based on https://prometheus.io. + * + * @since 3.6.0 + */ +public class PrometheusMetricsProvider implements MetricsProvider { + + private static final Logger LOG = LoggerFactory.getLogger(PrometheusMetricsProvider.class); + private static final String LABEL = "key"; + private static final String[] LABELS = {LABEL}; + + /** + * Number of worker threads for reporting Prometheus summary metrics. + * Default value is 1. + * If the number is less than 1, the main thread will be used. + */ + static final String NUM_WORKER_THREADS = "numWorkerThreads"; + + /** + * The max queue size for Prometheus summary metrics reporting task. + * Default value is 1000000. + */ + static final String MAX_QUEUE_SIZE = "maxQueueSize"; + + /** + * The timeout in ms for Prometheus worker threads shutdown. + * Default value is 1000ms. + */ + static final String WORKER_SHUTDOWN_TIMEOUT_MS = "workerShutdownTimeoutMs"; + + /** + * We are using the 'defaultRegistry'. + *

+ * When you are running ZooKeeper (server or client) together with other + * libraries every metrics will be expected as a single view. + *

+ */ + private final CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry; + private final RateLogger rateLogger = new RateLogger(LOG, 60 * 1000); + private String host = "0.0.0.0"; + private int port = 7000; + private boolean exportJvmInfo = true; + private Server server; + private final MetricsServletImpl servlet = new MetricsServletImpl(); + private final Context rootContext = new Context(); + private int numWorkerThreads = 1; + private int maxQueueSize = 1000000; + private long workerShutdownTimeoutMs = 1000; + private Optional executorOptional = Optional.empty(); + + @Override + public void configure(Properties configuration) throws MetricsProviderLifeCycleException { + LOG.info("Initializing metrics, configuration: {}", configuration); + this.host = configuration.getProperty("httpHost", "0.0.0.0"); + this.port = Integer.parseInt(configuration.getProperty("httpPort", "7000")); + this.exportJvmInfo = Boolean.parseBoolean(configuration.getProperty("exportJvmInfo", "true")); + this.numWorkerThreads = Integer.parseInt( + configuration.getProperty(NUM_WORKER_THREADS, "1")); + this.maxQueueSize = Integer.parseInt( + configuration.getProperty(MAX_QUEUE_SIZE, "1000000")); + this.workerShutdownTimeoutMs = Long.parseLong( + configuration.getProperty(WORKER_SHUTDOWN_TIMEOUT_MS, "1000")); + } + + @Override + public void start() throws MetricsProviderLifeCycleException { + this.executorOptional = createExecutor(); + try { + LOG.info("Starting /metrics HTTP endpoint at host: {}, port: {}, exportJvmInfo: {}", + host, port, exportJvmInfo); + if (exportJvmInfo) { + DefaultExports.initialize(); + } + server = new Server(new InetSocketAddress(host, port)); + ServletContextHandler context = new ServletContextHandler(); + context.setContextPath("/"); + constrainTraceMethod(context); + server.setHandler(context); + context.addServlet(new ServletHolder(servlet), "/metrics"); + server.start(); + } catch (Exception err) { + LOG.error("Cannot start /metrics server", err); + if (server != null) { + try { + server.stop(); + } catch (Exception suppressed) { + err.addSuppressed(suppressed); + } finally { + server = null; + } + } + throw new MetricsProviderLifeCycleException(err); + } + } + + // for tests + MetricsServletImpl getServlet() { + return servlet; + } + + @Override + public MetricsContext getRootContext() { + return rootContext; + } + + @Override + public void stop() { + shutdownExecutor(); + if (server != null) { + try { + server.stop(); + } catch (Exception err) { + LOG.error("Cannot safely stop Jetty server", err); + } finally { + server = null; + } + } + } + + /** + * Dump all values to the 4lw interface and to the Admin server. + *

+ * This method is not expected to be used to serve metrics to Prometheus. We + * are using the MetricsServlet provided by Prometheus for that, leaving the + * real representation to the Prometheus Java client. + *

+ * + * @param sink the receiver of data (4lw interface, Admin server or tests) + */ + @Override + public void dump(BiConsumer sink) { + sampleGauges(); + Enumeration samplesFamilies = collectorRegistry.metricFamilySamples(); + while (samplesFamilies.hasMoreElements()) { + Collector.MetricFamilySamples samples = samplesFamilies.nextElement(); + samples.samples.forEach(sample -> { + String key = buildKeyForDump(sample); + sink.accept(key, sample.value); + }); + } + } + + private static String buildKeyForDump(Collector.MetricFamilySamples.Sample sample) { + StringBuilder keyBuilder = new StringBuilder(); + keyBuilder.append(sample.name); + if (sample.labelNames.size() > 0) { + keyBuilder.append('{'); + for (int i = 0; i < sample.labelNames.size(); ++i) { + if (i > 0) { + keyBuilder.append(','); + } + keyBuilder.append(sample.labelNames.get(i)); + keyBuilder.append("=\""); + keyBuilder.append(sample.labelValues.get(i)); + keyBuilder.append('"'); + } + keyBuilder.append('}'); + } + return keyBuilder.toString(); + } + + /** + * Update Gauges. In ZooKeeper Metrics API Gauges are callbacks served by + * internal components and the value is not held by Prometheus structures. + */ + private void sampleGauges() { + rootContext.gauges.values() + .forEach(PrometheusGaugeWrapper::sample); + + rootContext.gaugeSets.values() + .forEach(PrometheusLabelledGaugeWrapper::sample); + } + + @Override + public void resetAllValues() { + // not supported on Prometheus + } + + /** + * Add constraint to a given context to disallow TRACE method. + * @param ctxHandler the context to modify + */ + private void constrainTraceMethod(ServletContextHandler ctxHandler) { + ServletConstraint c = new ServletConstraint(); + c.setAuthenticate(true); + + ConstraintMapping cmt = new ConstraintMapping(); + cmt.setConstraint(c); + cmt.setMethod("TRACE"); + cmt.setPathSpec("/*"); + + ConstraintSecurityHandler securityHandler = new ConstraintSecurityHandler(); + securityHandler.setConstraintMappings(new ConstraintMapping[] {cmt}); + + ctxHandler.setSecurityHandler(securityHandler); + } + + private class Context implements MetricsContext { + + private final ConcurrentMap gauges = new ConcurrentHashMap<>(); + private final ConcurrentMap gaugeSets = new ConcurrentHashMap<>(); + private final ConcurrentMap counters = new ConcurrentHashMap<>(); + private final ConcurrentMap counterSets = new ConcurrentHashMap<>(); + private final ConcurrentMap basicSummaries = new ConcurrentHashMap<>(); + private final ConcurrentMap summaries = new ConcurrentHashMap<>(); + private final ConcurrentMap basicSummarySets = new ConcurrentHashMap<>(); + private final ConcurrentMap summarySets = new ConcurrentHashMap<>(); + + @Override + public MetricsContext getContext(String name) { + // no hierarchy yet + return this; + } + + @Override + public Counter getCounter(String name) { + return counters.computeIfAbsent(name, PrometheusCounter::new); + } + + @Override + public CounterSet getCounterSet(final String name) { + Objects.requireNonNull(name, "Cannot register a CounterSet with null name"); + return counterSets.computeIfAbsent(name, PrometheusLabelledCounter::new); + } + + /** + * Gauges may go up and down, in ZooKeeper they are a way to export + * internal values with a callback. + * + * @param name the name of the gauge + * @param gauge the callback + */ + @Override + public void registerGauge(String name, Gauge gauge) { + Objects.requireNonNull(name); + gauges.compute(name, (id, prev) -> + new PrometheusGaugeWrapper(id, gauge, prev != null ? prev.inner : null)); + } + + @Override + public void unregisterGauge(String name) { + PrometheusGaugeWrapper existing = gauges.remove(name); + if (existing != null) { + existing.unregister(); + } + } + + @Override + public void registerGaugeSet(final String name, final GaugeSet gaugeSet) { + Objects.requireNonNull(name, "Cannot register a GaugeSet with null name"); + Objects.requireNonNull(gaugeSet, "Cannot register a null GaugeSet for " + name); + + gaugeSets.compute(name, (id, prev) -> + new PrometheusLabelledGaugeWrapper(name, gaugeSet, prev != null ? prev.inner : null)); + } + + @Override + public void unregisterGaugeSet(final String name) { + Objects.requireNonNull(name, "Cannot unregister GaugeSet with null name"); + + final PrometheusLabelledGaugeWrapper existing = gaugeSets.remove(name); + if (existing != null) { + existing.unregister(); + } + } + + @Override + public Summary getSummary(String name, DetailLevel detailLevel) { + if (detailLevel == DetailLevel.BASIC) { + return basicSummaries.computeIfAbsent(name, (n) -> { + if (summaries.containsKey(n)) { + throw new IllegalArgumentException("Already registered a non basic summary as " + n); + } + return new PrometheusSummary(name, detailLevel); + }); + } else { + return summaries.computeIfAbsent(name, (n) -> { + if (basicSummaries.containsKey(n)) { + throw new IllegalArgumentException("Already registered a basic summary as " + n); + } + return new PrometheusSummary(name, detailLevel); + }); + } + } + + @Override + public SummarySet getSummarySet(String name, DetailLevel detailLevel) { + if (detailLevel == DetailLevel.BASIC) { + return basicSummarySets.computeIfAbsent(name, (n) -> { + if (summarySets.containsKey(n)) { + throw new IllegalArgumentException("Already registered a non basic summary set as " + n); + } + return new PrometheusLabelledSummary(name, detailLevel); + }); + } else { + return summarySets.computeIfAbsent(name, (n) -> { + if (basicSummarySets.containsKey(n)) { + throw new IllegalArgumentException("Already registered a basic summary set as " + n); + } + return new PrometheusLabelledSummary(name, detailLevel); + }); + } + } + + } + + private class PrometheusCounter implements Counter { + + private final io.prometheus.client.Counter inner; + private final String name; + + public PrometheusCounter(String name) { + this.name = name; + this.inner = io.prometheus.client.Counter + .build(name, name) + .register(collectorRegistry); + } + + @Override + public void add(long delta) { + try { + inner.inc(delta); + } catch (IllegalArgumentException err) { + LOG.error("invalid delta {} for metric {}", delta, name, err); + } + } + + @Override + public long get() { + // this method is used only for tests + // Prometheus returns a "double" + // it is safe to fine to a long + // we are never setting non-integer values + return (long) inner.get(); + } + + } + + private class PrometheusLabelledCounter implements CounterSet { + private final String name; + private final io.prometheus.client.Counter inner; + + public PrometheusLabelledCounter(final String name) { + this.name = name; + this.inner = io.prometheus.client.Counter + .build(name, name) + .labelNames(LABELS) + .register(collectorRegistry); + } + + @Override + public void add(final String key, final long delta) { + try { + inner.labels(key).inc(delta); + } catch (final IllegalArgumentException e) { + LOG.error("invalid delta {} for metric {} with key {}", delta, name, key, e); + } + } + } + + private class PrometheusGaugeWrapper { + + private final io.prometheus.client.Gauge inner; + private final Gauge gauge; + private final String name; + + public PrometheusGaugeWrapper(String name, Gauge gauge, io.prometheus.client.Gauge prev) { + this.name = name; + this.gauge = gauge; + this.inner = prev != null ? prev + : io.prometheus.client.Gauge + .build(name, name) + .register(collectorRegistry); + } + + /** + * Call the callack and update Prometheus Gauge. This method is called + * when the server is polling for a value. + */ + private void sample() { + Number value = gauge.get(); + this.inner.set(value != null ? value.doubleValue() : 0); + } + + private void unregister() { + collectorRegistry.unregister(inner); + } + } + + /** + * Prometheus implementation of GaugeSet interface. It wraps the GaugeSet object and + * uses the callback API to update the Prometheus Gauge. + */ + private class PrometheusLabelledGaugeWrapper { + private final GaugeSet gaugeSet; + private final io.prometheus.client.Gauge inner; + + private PrometheusLabelledGaugeWrapper(final String name, + final GaugeSet gaugeSet, + final io.prometheus.client.Gauge prev) { + this.gaugeSet = gaugeSet; + this.inner = prev != null ? prev : + io.prometheus.client.Gauge + .build(name, name) + .labelNames(LABELS) + .register(collectorRegistry); + } + + /** + * Call the callback provided by the GaugeSet and update Prometheus Gauge. + * This method is called when the server is polling for a value. + */ + private void sample() { + gaugeSet.values().forEach((key, value) -> + this.inner.labels(key).set(value != null ? value.doubleValue() : 0)); + } + + private void unregister() { + collectorRegistry.unregister(inner); + } + } + + private class PrometheusSummary implements Summary { + + private final io.prometheus.client.Summary inner; + private final String name; + + public PrometheusSummary(String name, MetricsContext.DetailLevel level) { + this.name = name; + if (level == MetricsContext.DetailLevel.ADVANCED) { + this.inner = io.prometheus.client.Summary + .build(name, name) + .quantile(0.5, 0.05) // Add 50th percentile (= median) with 5% tolerated error + .quantile(0.9, 0.01) // Add 90th percentile with 1% tolerated error + .quantile(0.99, 0.001) // Add 99th percentile with 0.1% tolerated error + .register(collectorRegistry); + } else { + this.inner = io.prometheus.client.Summary + .build(name, name) + .quantile(0.5, 0.05) // Add 50th percentile (= median) with 5% tolerated error + .register(collectorRegistry); + } + } + + @Override + public void add(long delta) { + reportMetrics(() -> observe(delta)); + } + + private void observe(final long delta) { + try { + inner.observe(delta); + } catch (final IllegalArgumentException err) { + LOG.error("invalid delta {} for metric {}", delta, name, err); + } + } + } + + private class PrometheusLabelledSummary implements SummarySet { + + private final io.prometheus.client.Summary inner; + private final String name; + + public PrometheusLabelledSummary(String name, MetricsContext.DetailLevel level) { + this.name = name; + if (level == MetricsContext.DetailLevel.ADVANCED) { + this.inner = io.prometheus.client.Summary + .build(name, name) + .labelNames(LABELS) + .quantile(0.5, 0.05) // Add 50th percentile (= median) with 5% tolerated error + .quantile(0.9, 0.01) // Add 90th percentile with 1% tolerated error + .quantile(0.99, 0.001) // Add 99th percentile with 0.1% tolerated error + .register(collectorRegistry); + } else { + this.inner = io.prometheus.client.Summary + .build(name, name) + .labelNames(LABELS) + .quantile(0.5, 0.05) // Add 50th percentile (= median) with 5% tolerated error + .register(collectorRegistry); + } + } + + @Override + public void add(String key, long value) { + reportMetrics(() -> observe(key, value)); + } + + private void observe(final String key, final long value) { + try { + inner.labels(key).observe(value); + } catch (final IllegalArgumentException err) { + LOG.error("invalid value {} for metric {} with key {}", value, name, key, err); + } + } + + } + + class MetricsServletImpl extends MetricsServlet { + + @Override + protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws IOException { + // little trick: update the Gauges before serving data + // from Prometheus CollectorRegistry + sampleGauges(); + // serve data using Prometheus built in client. + super.doGet(req, resp); + } + } + + private Optional createExecutor() { + if (numWorkerThreads < 1) { + LOG.info("Executor service was not created as numWorkerThreads {} is less than 1", numWorkerThreads); + return Optional.empty(); + } + + final BlockingQueue queue = new LinkedBlockingQueue<>(maxQueueSize); + final ThreadPoolExecutor executor = new ThreadPoolExecutor(numWorkerThreads, + numWorkerThreads, + 0L, + TimeUnit.MILLISECONDS, + queue, new PrometheusWorkerThreadFactory()); + LOG.info("Executor service was created with numWorkerThreads {} and maxQueueSize {}", + numWorkerThreads, + maxQueueSize); + return Optional.of(executor); + } + + private void shutdownExecutor() { + if (executorOptional.isPresent()) { + LOG.info("Shutdown executor service with timeout {}", workerShutdownTimeoutMs); + final ExecutorService executor = executorOptional.get(); + executor.shutdown(); + try { + if (!executor.awaitTermination(workerShutdownTimeoutMs, TimeUnit.MILLISECONDS)) { + LOG.error("Not all the Prometheus worker threads terminated properly after {} timeout", + workerShutdownTimeoutMs); + executor.shutdownNow(); + } + } catch (final Exception e) { + LOG.error("Error occurred while terminating Prometheus worker threads", e); + executor.shutdownNow(); + } + } + } + + private static class PrometheusWorkerThreadFactory implements ThreadFactory { + private static final AtomicInteger workerCounter = new AtomicInteger(1); + + @Override + public Thread newThread(final Runnable runnable) { + final String threadName = "PrometheusMetricsProviderWorker-" + workerCounter.getAndIncrement(); + final Thread thread = new Thread(runnable, threadName); + thread.setDaemon(true); + return thread; + } + } + + private void reportMetrics(final Runnable task) { + if (executorOptional.isPresent()) { + try { + executorOptional.get().submit(task); + } catch (final RejectedExecutionException e) { + rateLogger.rateLimitLog("Prometheus metrics reporting task queue size exceeded the max", + String.valueOf(maxQueueSize)); + } + } else { + task.run(); + } + } +} diff --git a/jetty-upgrade/zookeeper-prometheus-metrics/src/main/java/org/apache/pulsar/metrics/prometheus/zookeeper/package-info.java b/jetty-upgrade/zookeeper-prometheus-metrics/src/main/java/org/apache/pulsar/metrics/prometheus/zookeeper/package-info.java new file mode 100644 index 0000000000000..59e2add5b97be --- /dev/null +++ b/jetty-upgrade/zookeeper-prometheus-metrics/src/main/java/org/apache/pulsar/metrics/prometheus/zookeeper/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +/** + * Prometheus metrics provider for ZooKeeper that is using Jetty 12. + */ +package org.apache.pulsar.metrics.prometheus.zookeeper; diff --git a/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/ExportJvmInfoTest.java b/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/ExportJvmInfoTest.java new file mode 100644 index 0000000000000..635fe1b0833a4 --- /dev/null +++ b/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/ExportJvmInfoTest.java @@ -0,0 +1,61 @@ +/* + * 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.metrics.prometheus.zookeeper; + +import static org.testng.AssertJUnit.assertEquals; +import io.prometheus.client.CollectorRegistry; +import java.util.Properties; +import org.testng.annotations.Test; + +/** + * Tests about Prometheus Metrics Provider. Please note that we are not testing + * Prometheus but our integration. + */ +public class ExportJvmInfoTest { + + @Test + public void exportInfo() throws Exception { + runTest(true); + } + + @Test + public void doNotExportInfo() throws Exception { + runTest(false); + } + + private void runTest(boolean exportJvmInfo) throws Exception { + CollectorRegistry.defaultRegistry.clear(); + PrometheusMetricsProvider provider = new PrometheusMetricsProvider(); + try { + Properties configuration = new Properties(); + configuration.setProperty("httpPort", "0"); // ephemeral port + configuration.setProperty("exportJvmInfo", exportJvmInfo + ""); + provider.configure(configuration); + provider.start(); + boolean[] found = {false}; + provider.dump((k, v) -> { + found[0] = found[0] || k.contains("heap"); + }); + assertEquals(exportJvmInfo, found[0]); + } finally { + provider.stop(); + } + } + +} diff --git a/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProviderConfigTest.java b/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProviderConfigTest.java new file mode 100644 index 0000000000000..f076a15db9380 --- /dev/null +++ b/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProviderConfigTest.java @@ -0,0 +1,66 @@ +/* + * 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.metrics.prometheus.zookeeper; + +import io.prometheus.client.CollectorRegistry; +import java.util.Properties; +import org.apache.zookeeper.metrics.MetricsProviderLifeCycleException; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class PrometheusMetricsProviderConfigTest { + + @Test + public void testInvalidPort() { + Assert.assertThrows(MetricsProviderLifeCycleException.class, () -> { + CollectorRegistry.defaultRegistry.clear(); + PrometheusMetricsProvider provider = new PrometheusMetricsProvider(); + Properties configuration = new Properties(); + configuration.setProperty("httpPort", "65536"); + configuration.setProperty("exportJvmInfo", "false"); + provider.configure(configuration); + provider.start(); + }); + } + + @Test + public void testInvalidAddr() { + Assert.assertThrows(MetricsProviderLifeCycleException.class, () -> { + CollectorRegistry.defaultRegistry.clear(); + PrometheusMetricsProvider provider = new PrometheusMetricsProvider(); + Properties configuration = new Properties(); + configuration.setProperty("httpHost", "master"); + provider.configure(configuration); + provider.start(); + }); + } + + @Test + public void testValidConfig() throws MetricsProviderLifeCycleException { + CollectorRegistry.defaultRegistry.clear(); + PrometheusMetricsProvider provider = new PrometheusMetricsProvider(); + Properties configuration = new Properties(); + configuration.setProperty("httpHost", "0.0.0.0"); + configuration.setProperty("httpPort", "0"); + provider.configure(configuration); + provider.start(); + } + +} diff --git a/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProviderTest.java b/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProviderTest.java new file mode 100644 index 0000000000000..55b993b3ab5e7 --- /dev/null +++ b/jetty-upgrade/zookeeper-prometheus-metrics/src/test/java/org/apache/pulsar/metrics/prometheus/zookeeper/PrometheusMetricsProviderTest.java @@ -0,0 +1,714 @@ +/* + * 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.metrics.prometheus.zookeeper; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertThrows; +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertSame; +import static org.testng.AssertJUnit.assertTrue; +import static org.testng.AssertJUnit.fail; +import io.prometheus.client.CollectorRegistry; +import java.io.IOException; +import java.io.PrintWriter; +import java.io.StringWriter; +import java.lang.reflect.Field; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicInteger; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import org.apache.zookeeper.metrics.Counter; +import org.apache.zookeeper.metrics.CounterSet; +import org.apache.zookeeper.metrics.Gauge; +import org.apache.zookeeper.metrics.GaugeSet; +import org.apache.zookeeper.metrics.MetricsContext; +import org.apache.zookeeper.metrics.Summary; +import org.apache.zookeeper.metrics.SummarySet; +import org.apache.zookeeper.server.util.QuotaMetricsUtils; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.ServerConnector; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +/** + * Tests about Prometheus Metrics Provider. Please note that we are not testing + * Prometheus but only our integration. + */ +public class PrometheusMetricsProviderTest { + + private static final String URL_FORMAT = "http://localhost:%d/metrics"; + private PrometheusMetricsProvider provider; + + @BeforeMethod + public void setup() throws Exception { + CollectorRegistry.defaultRegistry.clear(); + provider = new PrometheusMetricsProvider(); + Properties configuration = new Properties(); + configuration.setProperty("numWorkerThreads", "0"); // sync behavior for test + configuration.setProperty("httpHost", "127.0.0.1"); // local host for test + configuration.setProperty("httpPort", "0"); // ephemeral port + configuration.setProperty("exportJvmInfo", "false"); + provider.configure(configuration); + provider.start(); + } + + @AfterMethod(alwaysRun = true) + public void tearDown() { + if (provider != null) { + provider.stop(); + } + CollectorRegistry.defaultRegistry.clear(); + } + + @Test + public void testCounters() throws Exception { + Counter counter = provider.getRootContext().getCounter("cc"); + counter.add(10); + int[] count = {0}; + provider.dump((k, v) -> { + if (k.contains("_created")) { + return; + } + assertEquals("cc_total", k); + assertEquals(10, ((Number) v).intValue()); + count[0]++; + } + ); + assertEquals(1, count[0]); + count[0] = 0; + + // this is not allowed but it must not throw errors + counter.add(-1); + + provider.dump((k, v) -> { + if (k.contains("_created")) { + return; + } + assertEquals("cc_total", k); + assertEquals(10, ((Number) v).intValue()); + count[0]++; + } + ); + assertEquals(1, count[0]); + + // we always must get the same object + assertSame(counter, provider.getRootContext().getCounter("cc")); + + String res = callServlet(); + assertThat(res).contains("# TYPE cc_total counter"); + assertThat(res).contains("cc_total 10.0"); + } + + @Test + public void testCounterSet_single() throws Exception { + // create and register a CounterSet + final String name = QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE; + final CounterSet counterSet = provider.getRootContext().getCounterSet(name); + final String[] keys = {"ns1", "ns2"}; + final int count = 3; + + // update the CounterSet multiple times + for (int i = 0; i < count; i++) { + Arrays.asList(keys).forEach(key -> counterSet.inc(key)); + Arrays.asList(keys).forEach(key -> counterSet.add(key, 2)); + } + + // validate with dump call + final Map expectedMetricsMap = new HashMap<>(); + for (final String key : keys) { + expectedMetricsMap.put(String.format("%s_total{key=\"%s\"}", name, key), count * 3.0); + } + validateWithDump(expectedMetricsMap); + + // validate with servlet call + final List expectedNames = Collections.singletonList(String.format("# TYPE %s count", name)); + final List expectedMetrics = new ArrayList<>(); + for (final String key : keys) { + expectedMetrics.add(String.format("%s{key=\"%s\",} %s", name, key, count * 3.0)); + } + validateWithServletCall(expectedNames, expectedMetrics); + + // validate registering with same name, no overwriting + assertSame(counterSet, provider.getRootContext().getCounterSet(name)); + } + + @Test + public void testCounterSet_multiple() throws Exception { + final String name = QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE; + + final String[] names = new String[]{name + "_1", name + "_2"}; + final String[] keys = new String[]{"ns21", "ns22"}; + final int[] counts = new int[]{3, 5}; + + final int length = names.length; + final CounterSet[] counterSets = new CounterSet[length]; + + // create and register the CounterSets + for (int i = 0; i < length; i++) { + counterSets[i] = provider.getRootContext().getCounterSet(names[i]); + } + + // update each CounterSet multiple times + for (int i = 0; i < length; i++) { + for (int j = 0; j < counts[i]; j++) { + counterSets[i].inc(keys[i]); + } + } + + // validate with dump call + final Map expectedMetricsMap = new HashMap<>(); + for (int i = 0; i < length; i++) { + expectedMetricsMap.put(String.format("%s_total{key=\"%s\"}", names[i], keys[i]), counts[i] * 1.0); + } + validateWithDump(expectedMetricsMap); + + // validate with servlet call + final List expectedNames = new ArrayList<>(); + final List expectedMetrics = new ArrayList<>(); + for (int i = 0; i < length; i++) { + expectedNames.add(String.format("# TYPE %s count", names[i])); + expectedMetrics.add(String.format("%s{key=\"%s\",} %s", names[i], keys[i], counts[i] * 1.0)); + } + validateWithServletCall(expectedNames, expectedMetrics); + } + + @Test + public void testCounterSet_registerWithNullName() { + assertThrows(NullPointerException.class, + () -> provider.getRootContext().getCounterSet(null)); + } + + @Test + public void testCounterSet_negativeValue() { + // create and register a CounterSet + final String name = QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE; + final CounterSet counterSet = provider.getRootContext().getCounterSet(name); + + // add negative value and make sure no exception is thrown + counterSet.add("ns1", -1); + } + + @Test + public void testCounterSet_nullKey() { + // create and register a CounterSet + final String name = QuotaMetricsUtils.QUOTA_EXCEEDED_ERROR_PER_NAMESPACE; + final CounterSet counterSet = provider.getRootContext().getCounterSet(name); + + // increment the count with null key and make sure no exception is thrown + counterSet.inc(null); + counterSet.add(null, 2); + } + + @Test + public void testGauge() throws Exception { + int[] values = {78, -89}; + int[] callCounts = {0, 0}; + Gauge gauge0 = () -> { + callCounts[0]++; + return values[0]; + }; + Gauge gauge1 = () -> { + callCounts[1]++; + return values[1]; + }; + provider.getRootContext().registerGauge("gg", gauge0); + + int[] count = {0}; + provider.dump((k, v) -> { + assertEquals("gg", k); + assertEquals(values[0], ((Number) v).intValue()); + count[0]++; + } + ); + assertEquals(1, callCounts[0]); + assertEquals(0, callCounts[1]); + assertEquals(1, count[0]); + count[0] = 0; + String res2 = callServlet(); + assertThat(res2).contains("# TYPE gg gauge"); + assertThat(res2.contains("gg 78.0")); + + provider.getRootContext().unregisterGauge("gg"); + provider.dump((k, v) -> { + count[0]++; + } + ); + assertEquals(2, callCounts[0]); + assertEquals(0, callCounts[1]); + assertEquals(0, count[0]); + String res3 = callServlet(); + assertTrue(res3.isEmpty()); + + provider.getRootContext().registerGauge("gg", gauge1); + + provider.dump((k, v) -> { + assertEquals("gg", k); + assertEquals(values[1], ((Number) v).intValue()); + count[0]++; + } + ); + assertEquals(2, callCounts[0]); + assertEquals(1, callCounts[1]); + assertEquals(1, count[0]); + count[0] = 0; + + String res4 = callServlet(); + assertThat(res4.contains("# TYPE gg gauge")); + assertThat(res4.contains("gg -89.0")); + assertEquals(2, callCounts[0]); + // the servlet must sample the value again (from gauge1) + assertEquals(2, callCounts[1]); + + // override gauge, without unregister + provider.getRootContext().registerGauge("gg", gauge0); + + provider.dump((k, v) -> { + count[0]++; + } + ); + assertEquals(1, count[0]); + assertEquals(3, callCounts[0]); + assertEquals(2, callCounts[1]); + } + + @Test + public void testBasicSummary() throws Exception { + Summary summary = provider.getRootContext() + .getSummary("cc", MetricsContext.DetailLevel.BASIC); + summary.add(10); + summary.add(10); + int[] count = {0}; + provider.dump((k, v) -> { + count[0]++; + int value = ((Number) v).intValue(); + + switch (k) { + case "cc{quantile=\"0.5\"}": + assertEquals(10, value); + break; + case "cc_count": + assertEquals(2, value); + break; + case "cc_sum": + assertEquals(20, value); + break; + case "cc_created": + break; + default: + fail("unexpected key " + k); + break; + } + } + ); + assertEquals(4, count[0]); + count[0] = 0; + + // we always must get the same object + assertSame(summary, provider.getRootContext() + .getSummary("cc", MetricsContext.DetailLevel.BASIC)); + + try { + provider.getRootContext() + .getSummary("cc", MetricsContext.DetailLevel.ADVANCED); + fail("Can't get the same summary with a different DetailLevel"); + } catch (IllegalArgumentException err) { + assertThat(err.getMessage()).contains("Already registered"); + } + + String res = callServlet(); + assertThat(res).contains("# TYPE cc summary"); + assertThat(res).contains("cc_sum 20.0"); + assertThat(res).contains("cc_count 2.0"); + assertThat(res).contains("cc{quantile=\"0.5\",} 10.0"); + } + + @Test + public void testAdvancedSummary() throws Exception { + Summary summary = provider.getRootContext() + .getSummary("cc", MetricsContext.DetailLevel.ADVANCED); + summary.add(10); + summary.add(10); + int[] count = {0}; + provider.dump((k, v) -> { + count[0]++; + int value = ((Number) v).intValue(); + + switch (k) { + case "cc{quantile=\"0.5\"}": + assertEquals(10, value); + break; + case "cc{quantile=\"0.9\"}": + assertEquals(10, value); + break; + case "cc{quantile=\"0.99\"}": + assertEquals(10, value); + break; + case "cc_count": + assertEquals(2, value); + break; + case "cc_sum": + assertEquals(20, value); + break; + case "cc_created": + break; + default: + fail("unexpected key " + k); + break; + } + } + ); + assertEquals(6, count[0]); + count[0] = 0; + + // we always must get the same object + assertSame(summary, provider.getRootContext() + .getSummary("cc", MetricsContext.DetailLevel.ADVANCED)); + + try { + provider.getRootContext() + .getSummary("cc", MetricsContext.DetailLevel.BASIC); + fail("Can't get the same summary with a different DetailLevel"); + } catch (IllegalArgumentException err) { + assertThat(err.getMessage()).contains("Already registered"); + } + + String res = callServlet(); + assertThat(res).contains("# TYPE cc summary"); + assertThat(res).contains("cc_sum 20.0"); + assertThat(res).contains("cc_count 2.0"); + assertThat(res).contains("cc{quantile=\"0.5\",} 10.0"); + assertThat(res).contains("cc{quantile=\"0.9\",} 10.0"); + assertThat(res).contains("cc{quantile=\"0.99\",} 10.0"); + } + + /** + * Using TRACE method to visit metrics provider, the response should be 403 forbidden. + */ + @Test + public void testTraceCall() throws IOException, IllegalAccessException, NoSuchFieldException { + Field privateServerField = provider.getClass().getDeclaredField("server"); + privateServerField.setAccessible(true); + Server server = (Server) privateServerField.get(provider); + int port = ((ServerConnector) server.getConnectors()[0]).getLocalPort(); + + String metricsUrl = String.format(URL_FORMAT, port); + HttpURLConnection conn = (HttpURLConnection) new URL(metricsUrl).openConnection(); + conn.setRequestMethod("TRACE"); + conn.connect(); + assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode()); + } + + @Test + public void testSummary_asyncAndExceedMaxQueueSize() throws Exception { + final Properties config = new Properties(); + config.setProperty("numWorkerThreads", "1"); + config.setProperty("maxQueueSize", "1"); + config.setProperty("httpPort", "0"); // ephemeral port + config.setProperty("exportJvmInfo", "false"); + + PrometheusMetricsProvider metricsProvider = null; + try { + metricsProvider = new PrometheusMetricsProvider(); + metricsProvider.configure(config); + metricsProvider.start(); + final Summary summary = + metricsProvider.getRootContext().getSummary("cc", MetricsContext.DetailLevel.ADVANCED); + + // make sure no error is thrown + for (int i = 0; i < 10; i++) { + summary.add(10); + } + } finally { + if (metricsProvider != null) { + metricsProvider.stop(); + } + } + } + + @Test + public void testSummarySet() throws Exception { + final String name = "ss"; + final String[] keys = {"ns1", "ns2"}; + final double count = 3.0; + + // create and register a SummarySet + final SummarySet summarySet = provider.getRootContext() + .getSummarySet(name, MetricsContext.DetailLevel.BASIC); + + // update the SummarySet multiple times + for (int i = 0; i < count; i++) { + Arrays.asList(keys).forEach(key -> summarySet.add(key, 1)); + } + + // validate with dump call + final Map expectedMetricsMap = new HashMap<>(); + for (final String key : keys) { + expectedMetricsMap.put(String.format("%s{key=\"%s\",quantile=\"0.5\"}", name, key), 1.0); + expectedMetricsMap.put(String.format("%s_count{key=\"%s\"}", name, key), count); + expectedMetricsMap.put(String.format("%s_sum{key=\"%s\"}", name, key), count); + } + validateWithDump(expectedMetricsMap); + + // validate with servlet call + final List expectedNames = Collections.singletonList(String.format("# TYPE %s summary", name)); + final List expectedMetrics = new ArrayList<>(); + for (final String key : keys) { + expectedMetrics.add(String.format("%s{key=\"%s\",quantile=\"0.5\",} %s", name, key, 1.0)); + expectedMetrics.add(String.format("%s_count{key=\"%s\",} %s", name, key, count)); + expectedMetrics.add(String.format("%s_sum{key=\"%s\",} %s", name, key, count)); + } + validateWithServletCall(expectedNames, expectedMetrics); + + // validate registering with same name, no overwriting + assertSame(summarySet, provider.getRootContext() + .getSummarySet(name, MetricsContext.DetailLevel.BASIC)); + + // validate registering with different DetailLevel, not allowed + try { + provider.getRootContext() + .getSummarySet(name, MetricsContext.DetailLevel.ADVANCED); + fail("Can't get the same summarySet with a different DetailLevel"); + } catch (final IllegalArgumentException e) { + assertThat(e.getMessage()).contains("Already registered"); + } + } + + private String callServlet() throws ServletException, IOException { + // we are not performing an HTTP request + // but we are calling directly the servlet + StringWriter writer = new StringWriter(); + HttpServletResponse response = mock(HttpServletResponse.class); + when(response.getWriter()).thenReturn(new PrintWriter(writer)); + HttpServletRequest req = mock(HttpServletRequest.class); + provider.getServlet().doGet(req, response); + String res = writer.toString(); + return res; + } + + @Test + public void testGaugeSet_singleGaugeSet() throws Exception { + final String name = QuotaMetricsUtils.QUOTA_BYTES_LIMIT_PER_NAMESPACE; + final Number[] values = {10.0, 100.0}; + final String[] keys = {"ns11", "ns12"}; + final Map metricsMap = new HashMap<>(); + for (int i = 0; i < values.length; i++) { + metricsMap.put(keys[i], values[i]); + } + final AtomicInteger callCount = new AtomicInteger(0); + + // create and register GaugeSet + createAndRegisterGaugeSet(name, metricsMap, callCount); + + // validate with dump call + final Map expectedMetricsMap = new HashMap<>(); + for (int i = 0; i < values.length; i++) { + expectedMetricsMap.put(String.format("%s{key=\"%s\"}", name, keys[i]), values[i]); + } + validateWithDump(expectedMetricsMap); + assertEquals(1, callCount.get()); + + // validate with servlet call + final List expectedNames = Collections.singletonList(String.format("# TYPE %s gauge", name)); + final List expectedMetrics = new ArrayList<>(); + for (int i = 0; i < values.length; i++) { + expectedMetrics.add(String.format("%s{key=\"%s\",} %s", name, keys[i], values[i])); + } + validateWithServletCall(expectedNames, expectedMetrics); + assertEquals(2, callCount.get()); + + // unregister the GaugeSet + callCount.set(0); + provider.getRootContext().unregisterGaugeSet(name); + + // validate with dump call + validateWithDump(Collections.emptyMap()); + assertEquals(0, callCount.get()); + + // validate with servlet call + validateWithServletCall(new ArrayList<>(), new ArrayList<>()); + assertEquals(0, callCount.get()); + } + + @Test + public void testGaugeSet_multipleGaugeSets() throws Exception { + final String[] names = new String[]{ + QuotaMetricsUtils.QUOTA_COUNT_LIMIT_PER_NAMESPACE, + QuotaMetricsUtils.QUOTA_COUNT_USAGE_PER_NAMESPACE + }; + + final Number[] values = new Number[]{20.0, 200.0}; + final String[] keys = new String[]{"ns21", "ns22"}; + final int count = names.length; + final AtomicInteger[] callCounts = new AtomicInteger[count]; + + // create and register the GaugeSets + for (int i = 0; i < count; i++) { + final Map metricsMap = new HashMap<>(); + metricsMap.put(keys[i], values[i]); + callCounts[i] = new AtomicInteger(0); + createAndRegisterGaugeSet(names[i], metricsMap, callCounts[i]); + } + + // validate with dump call + final Map expectedMetricsMap = new HashMap<>(); + for (int i = 0; i < count; i++) { + expectedMetricsMap.put(String.format("%s{key=\"%s\"}", names[i], keys[i]), values[i]); + } + validateWithDump(expectedMetricsMap); + for (int i = 0; i < count; i++) { + assertEquals(1, callCounts[i].get()); + } + + // validate with servlet call + final List expectedNames = new ArrayList<>(); + final List expectedMetrics = new ArrayList<>(); + for (int i = 0; i < count; i++) { + expectedNames.add(String.format("# TYPE %s gauge", names[i])); + expectedMetrics.add(String.format("%s{key=\"%s\",} %s", names[i], keys[i], values[i])); + } + validateWithServletCall(expectedNames, expectedMetrics); + for (int i = 0; i < count; i++) { + assertEquals(2, callCounts[i].get()); + } + + // unregister the GaugeSets + for (int i = 0; i < count; i++) { + callCounts[i].set(0); + provider.getRootContext().unregisterGaugeSet(names[i]); + } + + // validate with dump call + validateWithDump(Collections.emptyMap()); + for (int i = 0; i < count; i++) { + assertEquals(0, callCounts[i].get()); + } + + // validate with servlet call + validateWithServletCall(new ArrayList<>(), new ArrayList<>()); + for (int i = 0; i < count; i++) { + assertEquals(0, callCounts[i].get()); + } + } + + @Test + public void testGaugeSet_overwriteRegister() { + final String[] names = new String[]{ + QuotaMetricsUtils.QUOTA_COUNT_LIMIT_PER_NAMESPACE, + QuotaMetricsUtils.QUOTA_COUNT_USAGE_PER_NAMESPACE + }; + + final int count = names.length; + final Number[] values = new Number[]{30.0, 300.0}; + final String[] keys = new String[]{"ns31", "ns32"}; + final AtomicInteger[] callCounts = new AtomicInteger[count]; + + // create and register the GaugeSets + for (int i = 0; i < count; i++) { + final Map metricsMap = new HashMap<>(); + metricsMap.put(keys[i], values[i]); + callCounts[i] = new AtomicInteger(0); + // use the same name so the first GaugeSet got overwrite + createAndRegisterGaugeSet(names[0], metricsMap, callCounts[i]); + } + + // validate with dump call to make sure the second GaugeSet overwrites the first + final Map expectedMetricsMap = new HashMap<>(); + expectedMetricsMap.put(String.format("%s{key=\"%s\"}", names[0], keys[1]), values[1]); + validateWithDump(expectedMetricsMap); + assertEquals(0, callCounts[0].get()); + assertEquals(1, callCounts[1].get()); + } + + @Test + public void testGaugeSet_nullKey() { + final String name = QuotaMetricsUtils.QUOTA_COUNT_LIMIT_PER_NAMESPACE; + final Map metricsMap = new HashMap<>(); + metricsMap.put(null, 10.0); + + final AtomicInteger callCount = new AtomicInteger(0); + + // create and register GaugeSet + createAndRegisterGaugeSet(name, metricsMap, callCount); + + // validate with dump call + assertThrows(IllegalArgumentException.class, () -> provider.dump(new HashMap<>()::put)); + + // validate with servlet call + assertThrows(IllegalArgumentException.class, this::callServlet); + } + + @Test + public void testGaugeSet_registerWithNullGaugeSet() { + assertThrows(NullPointerException.class, + () -> provider.getRootContext().registerGaugeSet("name", null)); + + assertThrows(NullPointerException.class, + () -> provider.getRootContext().registerGaugeSet(null, HashMap::new)); + } + + @Test + public void testGaugeSet_unregisterNull() { + assertThrows(NullPointerException.class, + () -> provider.getRootContext().unregisterGaugeSet(null)); + } + + private void createAndRegisterGaugeSet(final String name, + final Map metricsMap, + final AtomicInteger callCount) { + final GaugeSet gaugeSet = () -> { + callCount.addAndGet(1); + return metricsMap; + }; + provider.getRootContext().registerGaugeSet(name, gaugeSet); + } + + private void validateWithDump(final Map expectedMetrics) { + final Map returnedMetrics = new HashMap<>(); + provider.dump((key, value) -> { + if (!key.contains("_created{")) { + returnedMetrics.put(key, value); + } + }); + assertThat(returnedMetrics).isEqualTo(expectedMetrics); + } + + private void validateWithServletCall(final List expectedNames, + final List expectedMetrics) throws Exception { + final String response = callServlet(); + if (expectedNames.isEmpty() && expectedMetrics.isEmpty()) { + assertTrue(response.isEmpty()); + } else { + expectedNames.forEach(name -> assertThat(response.contains(name))); + expectedMetrics.forEach(metric -> assertThat(response.contains(metric))); + } + } +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/pom.xml b/jetty-upgrade/zookeeper-with-patched-admin/pom.xml new file mode 100644 index 0000000000000..bf309c74b5da3 --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/pom.xml @@ -0,0 +1,105 @@ + + + + 4.0.0 + + org.apache.pulsar + jetty-upgrade + 4.2.0-SNAPSHOT + + + zookeeper-with-patched-admin + Apache Pulsar :: patched Apache ZooKeeper with Admin API using Jetty 12.1.x + Apache ZooKeeper with patched admin servlet using Jetty 12.1.x API + + + + org.apache.zookeeper + zookeeper + + + org.apache.zookeeper + zookeeper-jute + + + org.eclipse.jetty + jetty-server + provided + + + org.eclipse.jetty.ee8 + jetty-ee8-servlet + provided + + + com.github.spotbugs + spotbugs-annotations + provided + true + + + com.fasterxml.jackson.core + jackson-databind + provided + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + ${shadePluginPhase} + + shade + + + true + true + true + + + org.apache.zookeeper:zookeeper + + + + + org.apache.zookeeper:zookeeper + + ** + + + org/apache/zookeeper/server/admin/** + + + + + + + + + + \ No newline at end of file diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AdminServer.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AdminServer.java new file mode 100644 index 0000000000000..095b4e0ddf064 --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AdminServer.java @@ -0,0 +1,53 @@ +/* + * 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.zookeeper.server.admin; + +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.server.ZooKeeperServer; + +/** + * Interface for an embedded admin server that runs Commands. There is only one + * functional implementation, JettyAdminServer. DummyAdminServer, which does + * nothing, is used when we do not wish to run a server. + */ +@InterfaceAudience.Public +public interface AdminServer { + + void start() throws AdminServerException; + + void shutdown() throws AdminServerException; + + void setZooKeeperServer(ZooKeeperServer zkServer); + + @InterfaceAudience.Public + class AdminServerException extends Exception { + + private static final long serialVersionUID = 1L; + + public AdminServerException(String message, Throwable cause) { + super(message, cause); + } + + public AdminServerException(Throwable cause) { + super(cause); + } + + } + +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AdminServerFactory.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AdminServerFactory.java new file mode 100644 index 0000000000000..cdf83486e69bc --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AdminServerFactory.java @@ -0,0 +1,64 @@ +/* + * 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.zookeeper.server.admin; + +import java.lang.reflect.InvocationTargetException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Factory class for creating an AdminServer. + */ +public class AdminServerFactory { + + private static final Logger LOG = LoggerFactory.getLogger(AdminServerFactory.class); + + /** + * This method encapsulates the logic for whether we should use a + * JettyAdminServer (i.e., the AdminServer is enabled) or a DummyAdminServer + * (i.e., the AdminServer is disabled). It uses reflection when attempting + * to create a JettyAdminServer, rather than referencing the class directly, + * so that it's ok to omit Jetty from the classpath if a user doesn't wish + * to pull in Jetty with ZooKeeper. + */ + public static AdminServer createAdminServer() { + if (!"false".equals(System.getProperty("zookeeper.admin.enableServer"))) { + try { + Class jettyAdminServerC = Class.forName("org.apache.zookeeper.server.admin.JettyAdminServer"); + Object adminServer = jettyAdminServerC.getConstructor().newInstance(); + return (AdminServer) adminServer; + + } catch (ClassNotFoundException e) { + LOG.warn("Unable to start JettyAdminServer", e); + } catch (InstantiationException e) { + LOG.warn("Unable to start JettyAdminServer", e); + } catch (IllegalAccessException e) { + LOG.warn("Unable to start JettyAdminServer", e); + } catch (InvocationTargetException e) { + LOG.warn("Unable to start JettyAdminServer", e); + } catch (NoSuchMethodException e) { + LOG.warn("Unable to start JettyAdminServer", e); + } catch (NoClassDefFoundError e) { + LOG.warn("Unable to load jetty, not starting JettyAdminServer", e); + } + } + return new DummyAdminServer(); + } + +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AuthRequest.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AuthRequest.java new file mode 100644 index 0000000000000..421d4f92931fb --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/AuthRequest.java @@ -0,0 +1,64 @@ +/* + * 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.zookeeper.server.admin; + +import org.apache.zookeeper.ZooDefs; + +/** + * Class representing auth data for performing ACL check on admin server commands. + * + * For example, SnapshotCommand requires {@link ZooDefs.Perms.ALL} permission on + * the root node. + * + */ +public class AuthRequest { + private final int permission; + private final String path; + + /** + * @param permission the required permission for auth check + * @param path the ZNode path for auth check + */ + public AuthRequest(final int permission, final String path) { + this.permission = permission; + this.path = path; + } + + /** + * @return permission + */ + public int getPermission() { + return permission; + } + + /** + * @return ZNode path + */ + public String getPath() { + return path; + } + + @Override + public String toString() { + return "AuthRequest{" + + "permission=" + permission + + ", path='" + path + '\'' + + '}'; + } +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/Command.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/Command.java new file mode 100644 index 0000000000000..8a8089e2d6039 --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/Command.java @@ -0,0 +1,90 @@ +/* + * 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.zookeeper.server.admin; + +import java.io.InputStream; +import java.util.Map; +import java.util.Set; +import org.apache.zookeeper.server.ZooKeeperServer; + +/** + * Interface implemented by all commands runnable by JettyAdminServer. + * + * @see CommandBase + * @see Commands + * @see JettyAdminServer + */ +public interface Command { + + /** + * The set of all names that can be used to refer to this command (e.g., + * "configuration", "config", and "conf"). + */ + Set getNames(); + + /** + * The name that is returned with the command response and that appears in + * the list of all commands. This should be a member of the set returned by + * getNames(). + */ + String getPrimaryName(); + + /** + * @return true if the command requires an active ZooKeeperServer or a + * synced peer in order to resolve + */ + boolean isServerRequired(); + + /** + * @return AuthRequest associated to the command. Null means auth check is not required. + */ + AuthRequest getAuthRequest(); + + /** + * Run this command for HTTP GET request. Commands take a ZooKeeperServer, String-valued keyword + * arguments and return a CommandResponse object containing any information + * constituting the response to the command. Commands are responsible for + * parsing keyword arguments and performing any error handling if necessary. + * Errors should be reported by setting the "error" entry of the returned + * map with an appropriate message rather than throwing an exception. + * + * @param zkServer ZooKeeper server + * @param kwargs keyword -> argument value mapping + * @return CommandResponse representing response to command containing at minimum: + * - "command" key containing the command's primary name + * - "error" key containing a String error message or null if no error + */ + CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs); + + /** + * Run this command for HTTP POST. Commands take a ZooKeeperServer and InputStream and + * return a CommandResponse object containing any information + * constituting the response to the command. Commands are responsible for + * parsing keyword arguments and performing any error handling if necessary. + * Errors should be reported by setting the "error" entry of the returned + * map with an appropriate message rather than throwing an exception. + * + * @param zkServer ZooKeeper server + * @param inputStream InputStream from request + * @return CommandResponse representing response to command containing at minimum: + * - "command" key containing the command's primary name + * - "error" key containing a String error message or null if no error + */ + CommandResponse runPost(ZooKeeperServer zkServer, InputStream inputStream); +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandBase.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandBase.java new file mode 100644 index 0000000000000..f6cb35066c384 --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandBase.java @@ -0,0 +1,83 @@ +/* + * 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.zookeeper.server.admin; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public abstract class CommandBase implements Command { + + private final String primaryName; + private final Set names; + private final boolean serverRequired; + private final AuthRequest authRequest; + + /** + * @param names The possible names of this command, with the primary name first. + */ + protected CommandBase(List names) { + this(names, true); + } + + protected CommandBase(List names, boolean serverRequired) { + this(names, serverRequired, null); + } + + protected CommandBase(List names, boolean serverRequired, AuthRequest authRequest) { + if (authRequest != null && !serverRequired) { + throw new IllegalArgumentException("An active server is required for auth check"); + } + this.primaryName = names.get(0); + this.names = new HashSet<>(names); + this.serverRequired = serverRequired; + this.authRequest = authRequest; + } + + @Override + public String getPrimaryName() { + return primaryName; + } + + @Override + public Set getNames() { + return names; + } + + + @Override + public boolean isServerRequired() { + return serverRequired; + } + + @Override + public AuthRequest getAuthRequest() { + return authRequest; + } + + /** + * @return A response with the command set to the primary name and the + * error set to null (these are the two entries that all command + * responses are required to include). + */ + protected CommandResponse initializeResponse() { + return new CommandResponse(primaryName); + } + +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandOutputter.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandOutputter.java new file mode 100644 index 0000000000000..2571605c5d522 --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandOutputter.java @@ -0,0 +1,48 @@ +/* + * 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.zookeeper.server.admin; + +import java.io.OutputStream; +import java.io.PrintWriter; + +/** + * CommandOutputters are used to format the responses from Commands. + * + * @see Command + * @see JettyAdminServer + */ +public interface CommandOutputter { + + /** + * The MIME type of this output (e.g., "application/json"). + */ + String getContentType(); + + /** + * Print out data as output. + */ + default void output(CommandResponse response, PrintWriter pw) { + } + + /** + * Stream out data as output. + */ + default void output(final CommandResponse response, final OutputStream os) { + } +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandResponse.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandResponse.java new file mode 100644 index 0000000000000..35ba6707650c5 --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/CommandResponse.java @@ -0,0 +1,190 @@ +/* + * 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.zookeeper.server.admin; + +import java.io.InputStream; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; +import javax.servlet.http.HttpServletResponse; + +/** + * A response from running a {@link Command}. + */ +public class CommandResponse { + + /** + * The key in the map returned by {@link #toMap()} for the command name. + */ + public static final String KEY_COMMAND = "command"; + /** + * The key in the map returned by {@link #toMap()} for the error string. + */ + public static final String KEY_ERROR = "error"; + + private final String command; + private final String error; + private final Map data; + private final Map headers; + private int statusCode; + private InputStream inputStream; + + /** + * Creates a new response with no error string. + * + * @param command command name + */ + public CommandResponse(String command) { + this(command, null, HttpServletResponse.SC_OK); + } + + /** + * Creates a new response. + * + * @param command command name + * @param error error string (may be null) + * @param statusCode http status code + */ + public CommandResponse(String command, String error, int statusCode) { + this(command, error, statusCode, null); + } + + + /** + * Creates a new response. + * + * @param command command name + * @param error error string (may be null) + * @param statusCode http status code + * @param inputStream inputStream to send out data (may be null) + */ + public CommandResponse(final String command, final String error, final int statusCode, + final InputStream inputStream) { + this.command = command; + this.error = error; + data = new LinkedHashMap<>(); + headers = new HashMap<>(); + this.statusCode = statusCode; + this.inputStream = inputStream; + } + + /** + * Gets the command name. + * + * @return command name + */ + public String getCommand() { + return command; + } + + /** + * Gets the error string (may be null). + * + * @return error string + */ + public String getError() { + return error; + } + + /** + * Gets the http status code. + * + * @return http status code + */ + public int getStatusCode() { + return statusCode; + } + + /** + * Sets the http status code. + */ + public void setStatusCode(int statusCode) { + this.statusCode = statusCode; + } + + /** + * Gets the InputStream (may be null). + * + * @return InputStream + */ + public InputStream getInputStream() { + return inputStream; + } + + /** + * Sets the InputStream. + */ + public void setInputStream(final InputStream inputStream) { + this.inputStream = inputStream; + } + + /** + * Adds a key/value pair to this response. + * + * @param key key + * @param value value + * @return prior value for key, or null if none + */ + public Object put(String key, Object value) { + return data.put(key, value); + } + + /** + * Adds all key/value pairs in the given map to this response. + * + * @param m map of key/value pairs + */ + public void putAll(Map m) { + data.putAll(m); + } + + /** + * Adds a header to this response. + * + * @param name name of the header + * @param value value of the header + */ + public void addHeader(final String name, final String value) { + headers.put(name, value); + } + + /** + * Returns all headers. + * + * @return map representation of all headers + */ + public Map getHeaders() { + return headers; + } + + /** + * Converts this response to a map. The returned map is mutable, and + * changes to it do not reflect back into this response. + * + * @return map representation of response + */ + public Map toMap() { + Map m = new LinkedHashMap<>(data); + m.put(KEY_COMMAND, command); + m.put(KEY_ERROR, error); + m.putAll(data); + return m; + } + +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/Commands.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/Commands.java new file mode 100644 index 0000000000000..2b4b89a422f34 --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/Commands.java @@ -0,0 +1,1168 @@ +/* + * 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.zookeeper.server.admin; + +import static org.apache.zookeeper.server.persistence.FileSnap.SNAPSHOT_FILE_PREFIX; +import com.fasterxml.jackson.annotation.JsonProperty; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.io.File; +import java.io.FileInputStream; +import java.io.InputStream; +import java.net.InetSocketAddress; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import org.apache.zookeeper.Environment; +import org.apache.zookeeper.Environment.Entry; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.Version; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Id; +import org.apache.zookeeper.server.DataNode; +import org.apache.zookeeper.server.DataTree; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.ServerMetrics; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.ZooTrace; +import org.apache.zookeeper.server.auth.ProviderRegistry; +import org.apache.zookeeper.server.auth.ServerAuthenticationProvider; +import org.apache.zookeeper.server.persistence.SnapshotInfo; +import org.apache.zookeeper.server.persistence.Util; +import org.apache.zookeeper.server.quorum.Follower; +import org.apache.zookeeper.server.quorum.FollowerZooKeeperServer; +import org.apache.zookeeper.server.quorum.Leader; +import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer; +import org.apache.zookeeper.server.quorum.MultipleAddresses; +import org.apache.zookeeper.server.quorum.QuorumPeer; +import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType; +import org.apache.zookeeper.server.quorum.QuorumZooKeeperServer; +import org.apache.zookeeper.server.quorum.ReadOnlyZooKeeperServer; +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; +import org.apache.zookeeper.server.util.RateLimiter; +import org.apache.zookeeper.server.util.ZxidUtils; +import org.eclipse.jetty.http.HttpStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Class containing static methods for registering and running Commands, as well + * as default Command definitions. + * + * @see Command + * @see JettyAdminServer + */ +public class Commands { + + static final Logger LOG = LoggerFactory.getLogger(Commands.class); + // VisibleForTesting + static final String ADMIN_RATE_LIMITER_INTERVAL = "zookeeper.admin.rateLimiterIntervalInMS"; + private static final long rateLimiterInterval = + Integer.parseInt(System.getProperty(ADMIN_RATE_LIMITER_INTERVAL, "300000")); + // VisibleForTesting + static final String AUTH_INFO_SEPARATOR = " "; + // VisibleForTesting + static final String ROOT_PATH = "/"; + + /** + * Maps command names to Command instances. + */ + private static Map commands = new HashMap<>(); + private static Set primaryNames = new HashSet<>(); + + /** + * Registers the given command. Registered commands can be run by passing + * any of their names to runCommand. + */ + public static void registerCommand(Command command) { + for (String name : command.getNames()) { + Command prev = commands.put(name, command); + if (prev != null) { + LOG.warn("Re-registering command {} (primary name = {})", name, command.getPrimaryName()); + } + } + primaryNames.add(command.getPrimaryName()); + } + + /** + * Run the registered command with name cmdName. Commands should not produce + * any exceptions; any (anticipated) errors should be reported in the + * "error" entry of the returned map. Likewise, if no command with the given + * name is registered, this will be noted in the "error" entry. + * + * @param cmdName + * @param zkServer + * @param kwargs String-valued keyword arguments to the command from HTTP GET request + * (may be null if command requires no additional arguments) + * @param authInfo auth info for auth check + * (null if command requires no auth check) + * @param request HTTP request + * @return Map representing response to command containing at minimum: + * - "command" key containing the command's primary name + * - "error" key containing a String error message or null if no error + */ + public static CommandResponse runGetCommand( + String cmdName, + ZooKeeperServer zkServer, + Map kwargs, + String authInfo, + HttpServletRequest request) { + return runCommand(cmdName, zkServer, kwargs, null, authInfo, request, true); + } + + /** + * Run the registered command with name cmdName. Commands should not produce + * any exceptions; any (anticipated) errors should be reported in the + * "error" entry of the returned map. Likewise, if no command with the given + * name is registered, this will be noted in the "error" entry. + * + * @param cmdName + * @param zkServer + * @param inputStream InputStream from HTTP POST request + * @return Map representing response to command containing at minimum: + * - "command" key containing the command's primary name + * - "error" key containing a String error message or null if no error + */ + public static CommandResponse runPostCommand( + String cmdName, + ZooKeeperServer zkServer, + InputStream inputStream, + String authInfo, + HttpServletRequest request) { + return runCommand(cmdName, zkServer, null, inputStream, authInfo, request, false); + } + + private static CommandResponse runCommand( + String cmdName, + ZooKeeperServer zkServer, + Map kwargs, + InputStream inputStream, + String authInfo, + HttpServletRequest request, + boolean isGet) { + Command command = getCommand(cmdName); + if (command == null) { + // set the status code to 200 to keep the current behavior of existing commands + LOG.warn("Unknown command"); + return new CommandResponse(cmdName, "Unknown command: " + cmdName, HttpServletResponse.SC_OK); + } + if (command.isServerRequired() && (zkServer == null || !zkServer.isRunning())) { + // set the status code to 200 to keep the current behavior of existing commands + LOG.warn("This ZooKeeper instance is not currently serving requests for command"); + return new CommandResponse(cmdName, "This ZooKeeper instance is not currently serving requests", + HttpServletResponse.SC_OK); + } + + final AuthRequest authRequest = command.getAuthRequest(); + if (authRequest != null) { + if (authInfo == null) { + LOG.warn("Auth info is missing for command"); + return new CommandResponse(cmdName, "Auth info is missing for the command", + HttpServletResponse.SC_UNAUTHORIZED); + } + try { + final List ids = handleAuthentication(request, authInfo); + handleAuthorization(zkServer, ids, authRequest.getPermission(), authRequest.getPath()); + } catch (final KeeperException.AuthFailedException e) { + return new CommandResponse(cmdName, "Not authenticated", HttpServletResponse.SC_UNAUTHORIZED); + } catch (final KeeperException.NoAuthException e) { + return new CommandResponse(cmdName, "Not authorized", HttpServletResponse.SC_FORBIDDEN); + } catch (final Exception e) { + LOG.warn("Error occurred during auth for command", e); + return new CommandResponse(cmdName, "Error occurred during auth", + HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + } + } + return isGet ? command.runGet(zkServer, kwargs) : command.runPost(zkServer, inputStream); + } + + private static List handleAuthentication(final HttpServletRequest request, final String authInfo) + throws KeeperException.AuthFailedException { + final String[] authData = authInfo.split(AUTH_INFO_SEPARATOR); + // for IP and x509, auth info only contains the schema and Auth Id will be extracted from HTTP request + if (authData.length != 1 && authData.length != 2) { + LOG.warn("Invalid auth info length"); + throw new KeeperException.AuthFailedException(); + } + + final String schema = authData[0]; + final ServerAuthenticationProvider authProvider = ProviderRegistry.getServerProvider(schema); + if (authProvider != null) { + try { + final byte[] auth = authData.length == 2 ? authData[1].getBytes(StandardCharsets.UTF_8) : null; + final List ids = authProvider.handleAuthentication(request, auth); + if (ids.isEmpty()) { + LOG.warn("Auth Id list is empty"); + throw new KeeperException.AuthFailedException(); + } + return ids; + } catch (final RuntimeException e) { + LOG.warn("Caught runtime exception from AuthenticationProvider", e); + throw new KeeperException.AuthFailedException(); + } + } else { + LOG.warn("Auth provider not found for schema"); + throw new KeeperException.AuthFailedException(); + } + } + + /** + * Grant or deny authorization for a command by matching + * request-provided credentials with the ACLs present on a node. + * + * @param zkServer the ZooKeeper server object. + * @param ids the credentials extracted from the Authorization header. + * @param perm the set of permission bits required by the command. + * @param path the ZooKeeper node path whose ACLs should be used + * to satisfy the perm bits. + * @throws KeeperException.NoAuthException if one or more perm + * bits could not be satisfied. + */ + private static void handleAuthorization(final ZooKeeperServer zkServer, + final List ids, + final int perm, + final String path) + throws KeeperException.NoNodeException, KeeperException.NoAuthException { + final DataNode dataNode = zkServer.getZKDatabase().getNode(path); + if (dataNode == null) { + throw new KeeperException.NoNodeException(path); + } + final List acls = zkServer.getZKDatabase().aclForNode(dataNode); + // Check the individual bits of perm. + final int bitWidth = Integer.SIZE - Integer.numberOfLeadingZeros(perm); + for (int b = 0; b < bitWidth; b++) { + final int permBit = 1 << b; + if ((perm & permBit) != 0) { + zkServer.checkACL(null, acls, permBit, ids, path, null); + } + } + } + + /** + * Returns the primary names of all registered commands. + */ + public static Set getPrimaryNames() { + return primaryNames; + } + + /** + * Returns the commands registered under cmdName with registerCommand, or + * null if no command is registered with that name. + */ + public static Command getCommand(String cmdName) { + return commands.get(cmdName); + } + + static { + registerCommand(new CnxnStatResetCommand()); + registerCommand(new ConfCommand()); + registerCommand(new ConsCommand()); + registerCommand(new DigestCommand()); + registerCommand(new DirsCommand()); + registerCommand(new DumpCommand()); + registerCommand(new EnvCommand()); + registerCommand(new GetTraceMaskCommand()); + registerCommand(new InitialConfigurationCommand()); + registerCommand(new IsroCommand()); + registerCommand(new LastSnapshotCommand()); + registerCommand(new LeaderCommand()); + registerCommand(new MonitorCommand()); + registerCommand(new ObserverCnxnStatResetCommand()); + registerCommand(new RestoreCommand()); + registerCommand(new RuokCommand()); + registerCommand(new SetTraceMaskCommand()); + registerCommand(new SnapshotCommand()); + registerCommand(new SrvrCommand()); + registerCommand(new StatCommand()); + registerCommand(new StatResetCommand()); + registerCommand(new SyncedObserverConsCommand()); + registerCommand(new SystemPropertiesCommand()); + registerCommand(new VotingViewCommand()); + registerCommand(new WatchCommand()); + registerCommand(new WatchesByPathCommand()); + registerCommand(new WatchSummaryCommand()); + registerCommand(new ZabStateCommand()); + } + + /** + * Reset all connection statistics. + */ + public static class CnxnStatResetCommand extends GetCommand { + + public CnxnStatResetCommand() { + super(Arrays.asList("connection_stat_reset", "crst")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + zkServer.getServerCnxnFactory().resetAllConnectionStats(); + return response; + + } + + } + + /** + * Server configuration parameters. + * + * @see ZooKeeperServer#getConf() + */ + public static class ConfCommand extends GetCommand { + + public ConfCommand() { + super(Arrays.asList("configuration", "conf", "config")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + response.putAll(zkServer.getConf().toMap()); + return response; + } + + } + + /** + * Information on client connections to server. Returned Map contains: + * - "connections": list of connection info objects + * + * @see org.apache.zookeeper.server.ServerCnxn#getConnectionInfo(boolean) + */ + public static class ConsCommand extends GetCommand { + + public ConsCommand() { + super(Arrays.asList("connections", "cons")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + ServerCnxnFactory serverCnxnFactory = zkServer.getServerCnxnFactory(); + if (serverCnxnFactory != null) { + response.put("connections", serverCnxnFactory.getAllConnectionInfo(false)); + } else { + response.put("connections", Collections.emptyList()); + } + ServerCnxnFactory secureServerCnxnFactory = zkServer.getSecureServerCnxnFactory(); + if (secureServerCnxnFactory != null) { + response.put("secure_connections", secureServerCnxnFactory.getAllConnectionInfo(false)); + } else { + response.put("secure_connections", Collections.emptyList()); + } + return response; + } + + } + + /** + * Information on ZK datadir and snapdir size in bytes. + */ + public static class DirsCommand extends GetCommand { + + public DirsCommand() { + super(Arrays.asList("dirs")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + response.put("datadir_size", zkServer.getDataDirSize()); + response.put("logdir_size", zkServer.getLogDirSize()); + return response; + } + + } + + /** + * Information on session expirations and ephemerals. Returned map contains: + * - "expiry_time_to_session_ids": Map<Long, Set<Long>> + * time -> sessions IDs of sessions that expire at time + * - "session_id_to_ephemeral_paths": Map<Long, Set<String>> + * session ID -> ephemeral paths created by that session + * + * @see ZooKeeperServer#getSessionExpiryMap() + * @see ZooKeeperServer#getEphemerals() + */ + public static class DumpCommand extends GetCommand { + + public DumpCommand() { + super(Arrays.asList("dump")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + response.put("expiry_time_to_session_ids", zkServer.getSessionExpiryMap()); + response.put("session_id_to_ephemeral_paths", zkServer.getEphemerals()); + return response; + } + + } + + /** + * All defined environment variables. + */ + public static class EnvCommand extends GetCommand { + + public EnvCommand() { + super(Arrays.asList("environment", "env", "envi"), false); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + for (Entry e : Environment.list()) { + response.put(e.getKey(), e.getValue()); + } + return response; + } + + } + + /** + * Digest histories for every specific number of txns. + */ + public static class DigestCommand extends GetCommand { + + public DigestCommand() { + super(Arrays.asList("hash")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + response.put("digests", zkServer.getZKDatabase().getDataTree().getDigestLog()); + return response; + } + + } + + /** + * The current trace mask. Returned map contains: + * - "tracemask": Long + */ + public static class GetTraceMaskCommand extends GetCommand { + + public GetTraceMaskCommand() { + super(Arrays.asList("get_trace_mask", "gtmk"), false); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + response.put("tracemask", ZooTrace.getTextTraceLevel()); + return response; + } + + } + + public static class InitialConfigurationCommand extends GetCommand { + + public InitialConfigurationCommand() { + super(Arrays.asList("initial_configuration", "icfg")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + response.put("initial_configuration", zkServer.getInitialConfig()); + return response; + } + + } + + /** + * Is this server in read-only mode. Returned map contains: + * - "is_read_only": Boolean + */ + public static class IsroCommand extends GetCommand { + + public IsroCommand() { + super(Arrays.asList("is_read_only", "isro")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + response.put("read_only", zkServer instanceof ReadOnlyZooKeeperServer); + return response; + } + + } + + /** + * Command returns information of the last snapshot that zookeeper server + * has finished saving to disk. During the time between the server starts up + * and it finishes saving its first snapshot, the command returns the zxid + * and last modified time of the snapshot file used for restoration at + * server startup. Returned map contains: + * - "zxid": String + * - "timestamp": Long + */ + public static class LastSnapshotCommand extends GetCommand { + + public LastSnapshotCommand() { + super(Arrays.asList("last_snapshot", "lsnp")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + SnapshotInfo info = zkServer.getTxnLogFactory().getLastSnapshotInfo(); + response.put("zxid", Long.toHexString(info == null ? -1L : info.zxid)); + response.put("timestamp", info == null ? -1L : info.timestamp); + return response; + } + + } + + /** + * Returns the leader status of this instance and the leader host string. + */ + public static class LeaderCommand extends GetCommand { + + public LeaderCommand() { + super(Arrays.asList("leader", "lead")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + if (zkServer instanceof QuorumZooKeeperServer) { + response.put("is_leader", zkServer instanceof LeaderZooKeeperServer); + QuorumPeer peer = ((QuorumZooKeeperServer) zkServer).self; + response.put("leader_id", peer.getLeaderId()); + String leaderAddress = peer.getLeaderAddress(); + response.put("leader_ip", leaderAddress != null ? leaderAddress : ""); + } else { + response.put("error", "server is not initialized"); + } + return response; + } + + } + + /** + * Some useful info for monitoring. Returned map contains: + * - "version": String + * server version + * - "avg_latency": Long + * - "max_latency": Long + * - "min_latency": Long + * - "packets_received": Long + * - "packets_sents": Long + * - "num_alive_connections": Integer + * - "outstanding_requests": Long + * number of unprocessed requests + * - "server_state": "leader", "follower", or "standalone" + * - "znode_count": Integer + * - "watch_count": Integer + * - "ephemerals_count": Integer + * - "approximate_data_size": Long + * - "open_file_descriptor_count": Long (unix only) + * - "max_file_descriptor_count": Long (unix only) + * - "fsync_threshold_exceed_count": Long + * - "non_mtls_conn_count": Long + * - "non_mtls_remote_conn_count": Long + * - "non_mtls_local_conn_count": Long + * - "followers": Integer (leader only) + * - "synced_followers": Integer (leader only) + * - "pending_syncs": Integer (leader only) + */ + public static class MonitorCommand extends GetCommand { + + public MonitorCommand() { + super(Arrays.asList("monitor", "mntr"), false); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + zkServer.dumpMonitorValues(response::put); + ServerMetrics.getMetrics().getMetricsProvider().dump(response::put); + return response; + + } + + } + + /** + * Reset all observer connection statistics. + */ + public static class ObserverCnxnStatResetCommand extends GetCommand { + + public ObserverCnxnStatResetCommand() { + super(Arrays.asList("observer_connection_stat_reset", "orst")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + if (zkServer instanceof LeaderZooKeeperServer) { + Leader leader = ((LeaderZooKeeperServer) zkServer).getLeader(); + leader.resetObserverConnectionStats(); + } else if (zkServer instanceof FollowerZooKeeperServer) { + Follower follower = ((FollowerZooKeeperServer) zkServer).getFollower(); + follower.resetObserverConnectionStats(); + } + return response; + } + + } + + /** + * Restore from snapshot on the current server. + * + * Returned map contains: + * - "last_zxid": String + */ + public static class RestoreCommand extends PostCommand { + static final String RESPONSE_DATA_LAST_ZXID = "last_zxid"; + static final String ADMIN_RESTORE_ENABLED = "zookeeper.admin.restore.enabled"; + + private RateLimiter rateLimiter; + + public RestoreCommand() { + super(Arrays.asList("restore", "rest"), true, new AuthRequest(ZooDefs.Perms.ALL, ROOT_PATH)); + rateLimiter = new RateLimiter(1, rateLimiterInterval, TimeUnit.MILLISECONDS); + } + + @Override + public CommandResponse runPost(final ZooKeeperServer zkServer, final InputStream inputStream) { + final CommandResponse response = initializeResponse(); + + // check feature flag + final boolean restoreEnabled = Boolean.parseBoolean(System.getProperty(ADMIN_RESTORE_ENABLED, "true")); + if (!restoreEnabled) { + response.setStatusCode(HttpServletResponse.SC_SERVICE_UNAVAILABLE); + LOG.warn("Restore command is disabled"); + return response; + } + + if (!zkServer.isSerializeLastProcessedZxidEnabled()) { + response.setStatusCode(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + LOG.warn("Restore command requires serializeLastProcessedZxidEnable flag is set to true"); + return response; + } + + if (inputStream == null) { + response.setStatusCode(HttpServletResponse.SC_BAD_REQUEST); + LOG.warn("InputStream from restore request is null"); + return response; + } + + // check rate limiting + if (!rateLimiter.allow()) { + response.setStatusCode(HttpStatus.TOO_MANY_REQUESTS_429); + ServerMetrics.getMetrics().RESTORE_RATE_LIMITED_COUNT.add(1); + LOG.warn("Restore request was rate limited"); + return response; + } + + // restore from snapshot InputStream + try { + final long lastZxid = zkServer.restoreFromSnapshot(inputStream); + response.put(RESPONSE_DATA_LAST_ZXID, lastZxid); + } catch (final Exception e) { + response.setStatusCode(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + ServerMetrics.getMetrics().RESTORE_ERROR_COUNT.add(1); + LOG.warn("Exception occurred when restore snapshot via the restore command", e); + } + return response; + } + } + + /** + * No-op command, check if the server is running. + */ + public static class RuokCommand extends GetCommand { + + public RuokCommand() { + super(Arrays.asList("ruok")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + return initializeResponse(); + } + + } + + /** + * Sets the trace mask. Required arguments: + * - "traceMask": Long + * Returned Map contains: + * - "tracemask": Long + */ + public static class SetTraceMaskCommand extends GetCommand { + + public SetTraceMaskCommand() { + super(Arrays.asList("set_trace_mask", "stmk"), false); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + long traceMask; + if (!kwargs.containsKey("traceMask")) { + response.put("error", "setTraceMask requires long traceMask argument"); + return response; + } + try { + traceMask = Long.parseLong(kwargs.get("traceMask")); + } catch (NumberFormatException e) { + response.put("error", "setTraceMask requires long traceMask argument, got " + kwargs.get("traceMask")); + return response; + } + + ZooTrace.setTextTraceLevel(traceMask); + response.put("tracemask", traceMask); + return response; + } + + } + + /** + * Take a snapshot of current server and stream out the data. + * + * Argument: + * - "streaming": optional String to indicate whether streaming out data + * + * Returned snapshot as stream if streaming is true and metadata of the snapshot + * - "last_zxid": String + * - "snapshot_size": String + */ + public static class SnapshotCommand extends GetCommand { + static final String REQUEST_QUERY_PARAM_STREAMING = "streaming"; + + static final String RESPONSE_HEADER_LAST_ZXID = "last_zxid"; + static final String RESPONSE_HEADER_SNAPSHOT_SIZE = "snapshot_size"; + + static final String ADMIN_SNAPSHOT_ENABLED = "zookeeper.admin.snapshot.enabled"; + + private final RateLimiter rateLimiter; + + public SnapshotCommand() { + super(Arrays.asList("snapshot", "snap"), true, new AuthRequest(ZooDefs.Perms.ALL, ROOT_PATH)); + rateLimiter = new RateLimiter(1, rateLimiterInterval, TimeUnit.MICROSECONDS); + } + + @SuppressFBWarnings(value = "OBL_UNSATISFIED_OBLIGATION", + justification = "FileInputStream is passed to CommandResponse and closed in StreamOutputter") + @Override + public CommandResponse runGet(final ZooKeeperServer zkServer, final Map kwargs) { + final CommandResponse response = initializeResponse(); + + // check feature flag + final boolean snapshotEnabled = Boolean.parseBoolean(System.getProperty(ADMIN_SNAPSHOT_ENABLED, "true")); + if (!snapshotEnabled) { + response.setStatusCode(HttpServletResponse.SC_SERVICE_UNAVAILABLE); + LOG.warn("Snapshot command is disabled"); + return response; + } + + if (!zkServer.isSerializeLastProcessedZxidEnabled()) { + response.setStatusCode(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + LOG.warn("Snapshot command requires serializeLastProcessedZxidEnable flag is set to true"); + return response; + } + + // check rate limiting + if (!rateLimiter.allow()) { + response.setStatusCode(HttpStatus.TOO_MANY_REQUESTS_429); + ServerMetrics.getMetrics().SNAPSHOT_RATE_LIMITED_COUNT.add(1); + LOG.warn("Snapshot request was rate limited"); + return response; + } + + // check the streaming query param + boolean streaming = true; + if (kwargs.containsKey(REQUEST_QUERY_PARAM_STREAMING)) { + streaming = Boolean.parseBoolean(kwargs.get(REQUEST_QUERY_PARAM_STREAMING)); + } + + // take snapshot and stream out data if needed + try { + final File snapshotFile = zkServer.takeSnapshot(false, false); + final long lastZxid = Util.getZxidFromName(snapshotFile.getName(), SNAPSHOT_FILE_PREFIX); + response.addHeader(RESPONSE_HEADER_LAST_ZXID, "0x" + ZxidUtils.zxidToString(lastZxid)); + + final long size = snapshotFile.length(); + response.addHeader(RESPONSE_HEADER_SNAPSHOT_SIZE, String.valueOf(size)); + + if (size == 0) { + response.setStatusCode(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + ServerMetrics.getMetrics().SNAPSHOT_ERROR_COUNT.add(1); + LOG.warn("Snapshot file {} is empty", snapshotFile); + } else if (streaming) { + response.setInputStream(new FileInputStream(snapshotFile)); + } + } catch (final Exception e) { + response.setStatusCode(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + ServerMetrics.getMetrics().SNAPSHOT_ERROR_COUNT.add(1); + LOG.warn("Exception occurred when taking the snapshot via the snapshot admin command", e); + } + return response; + } + } + + /** + * Server information. Returned map contains: + * - "version": String + * version of server + * - "read_only": Boolean + * is server in read-only mode + * - "server_stats": ServerStats object + * - "node_count": Integer + */ + public static class SrvrCommand extends GetCommand { + + public SrvrCommand() { + super(Arrays.asList("server_stats", "srvr")); + } + + // Allow subclasses (e.g. StatCommand) to specify their own names + protected SrvrCommand(List names) { + super(names); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + LOG.info("running stat"); + response.put("version", Version.getFullVersion()); + response.put("read_only", zkServer instanceof ReadOnlyZooKeeperServer); + response.put("server_stats", zkServer.serverStats()); + response.put("client_response", zkServer.serverStats().getClientResponseStats()); + if (zkServer instanceof LeaderZooKeeperServer) { + Leader leader = ((LeaderZooKeeperServer) zkServer).getLeader(); + response.put("proposal_stats", leader.getProposalStats()); + } + response.put("node_count", zkServer.getZKDatabase().getNodeCount()); + return response; + } + + } + + /** + * Same as SrvrCommand but has extra "connections" entry. + */ + public static class StatCommand extends SrvrCommand { + + public StatCommand() { + super(Arrays.asList("stats", "stat")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = super.runGet(zkServer, kwargs); + + final Iterable> connections; + if (zkServer.getServerCnxnFactory() != null) { + connections = zkServer.getServerCnxnFactory().getAllConnectionInfo(true); + } else { + connections = Collections.emptyList(); + } + response.put("connections", connections); + + final Iterable> secureConnections; + if (zkServer.getSecureServerCnxnFactory() != null) { + secureConnections = zkServer.getSecureServerCnxnFactory().getAllConnectionInfo(true); + } else { + secureConnections = Collections.emptyList(); + } + response.put("secure_connections", secureConnections); + return response; + } + + } + + /** + * Resets server statistics. + */ + public static class StatResetCommand extends GetCommand { + + public StatResetCommand() { + super(Arrays.asList("stat_reset", "srst")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + zkServer.serverStats().reset(); + return response; + } + + } + + /** + * Information on observer connections to server. Returned Map contains: + * - "synced_observers": Integer (leader/follower only) + * - "observers": list of observer learner handler info objects (leader/follower only) + * + * @see org.apache.zookeeper.server.quorum.LearnerHandler#getLearnerHandlerInfo() + */ + public static class SyncedObserverConsCommand extends GetCommand { + + public SyncedObserverConsCommand() { + super(Arrays.asList("observers", "obsr")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + + CommandResponse response = initializeResponse(); + + if (zkServer instanceof LeaderZooKeeperServer) { + Leader leader = ((LeaderZooKeeperServer) zkServer).getLeader(); + + response.put("synced_observers", leader.getObservingLearners().size()); + response.put("observers", leader.getObservingLearnersInfo()); + return response; + } else if (zkServer instanceof FollowerZooKeeperServer) { + Follower follower = ((FollowerZooKeeperServer) zkServer).getFollower(); + Integer syncedObservers = follower.getSyncedObserverSize(); + if (syncedObservers != null) { + response.put("synced_observers", syncedObservers); + response.put("observers", follower.getSyncedObserversInfo()); + return response; + } + } + + response.put("synced_observers", 0); + response.put("observers", Collections.emptySet()); + return response; + } + + } + + /** + * All defined system properties. + */ + public static class SystemPropertiesCommand extends GetCommand { + + public SystemPropertiesCommand() { + super(Arrays.asList("system_properties", "sysp"), false); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + Properties systemProperties = System.getProperties(); + SortedMap sortedSystemProperties = new TreeMap<>(); + systemProperties.forEach((k, v) -> sortedSystemProperties.put(k.toString(), v.toString())); + response.putAll(sortedSystemProperties); + return response; + } + + } + + /** + * Returns the current ensemble configuration information. + * It provides list of current voting members in the ensemble. + */ + public static class VotingViewCommand extends GetCommand { + + public VotingViewCommand() { + super(Arrays.asList("voting_view")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + if (zkServer instanceof QuorumZooKeeperServer) { + QuorumPeer peer = ((QuorumZooKeeperServer) zkServer).self; + Map votingView = peer.getVotingView().entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> new QuorumServerView(e.getValue()))); + response.put("current_config", votingView); + } else { + response.put("current_config", Collections.emptyMap()); + } + return response; + } + + @SuppressFBWarnings(value = "URF_UNREAD_FIELD", justification = "class is used only for JSON serialization") + private static class QuorumServerView { + + @JsonProperty + private List serverAddresses; + + @JsonProperty + private List electionAddresses; + + @JsonProperty + private String clientAddress; + + @JsonProperty + private String learnerType; + + public QuorumServerView(QuorumPeer.QuorumServer quorumServer) { + this.serverAddresses = getMultiAddressString(quorumServer.addr); + this.electionAddresses = getMultiAddressString(quorumServer.electionAddr); + this.learnerType = quorumServer.type.equals(LearnerType.PARTICIPANT) ? "participant" : "observer"; + this.clientAddress = getAddressString(quorumServer.clientAddr); + } + + private static List getMultiAddressString(MultipleAddresses multipleAddresses) { + if (multipleAddresses == null) { + return Collections.emptyList(); + } + + return multipleAddresses.getAllAddresses().stream() + .map(QuorumServerView::getAddressString) + .collect(Collectors.toList()); + } + + private static String getAddressString(InetSocketAddress address) { + if (address == null) { + return ""; + } + return String.format("%s:%d", QuorumPeer.QuorumServer.delimitedHostString(address), address.getPort()); + } + } + + } + + /** + * Watch information aggregated by session. Returned Map contains: + * - "session_id_to_watched_paths": Map<Long, Set<String>> session ID -> watched paths + * + * @see DataTree#getWatches() + * @see DataTree#getWatches() + */ + public static class WatchCommand extends GetCommand { + + public WatchCommand() { + super(Arrays.asList("watches", "wchc")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + DataTree dt = zkServer.getZKDatabase().getDataTree(); + CommandResponse response = initializeResponse(); + response.put("session_id_to_watched_paths", dt.getWatches().toMap()); + return response; + } + + } + + /** + * Watch information aggregated by path. Returned Map contains: + * - "path_to_session_ids": Map<String, Set<Long>> path -> session IDs of sessions watching path + * + * @see DataTree#getWatchesByPath() + */ + public static class WatchesByPathCommand extends GetCommand { + + public WatchesByPathCommand() { + super(Arrays.asList("watches_by_path", "wchp")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + DataTree dt = zkServer.getZKDatabase().getDataTree(); + CommandResponse response = initializeResponse(); + response.put("path_to_session_ids", dt.getWatchesByPath().toMap()); + return response; + } + + } + + /** + * Summarized watch information. + * + * @see DataTree#getWatchesSummary() + */ + public static class WatchSummaryCommand extends GetCommand { + + public WatchSummaryCommand() { + super(Arrays.asList("watch_summary", "wchs")); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + DataTree dt = zkServer.getZKDatabase().getDataTree(); + CommandResponse response = initializeResponse(); + response.putAll(dt.getWatchesSummary().toMap()); + return response; + } + + } + + /** + * Returns the current phase of Zab protocol that peer is running. + * It can be in one of these phases: ELECTION, DISCOVERY, SYNCHRONIZATION, BROADCAST + */ + public static class ZabStateCommand extends GetCommand { + + public ZabStateCommand() { + super(Arrays.asList("zabstate"), false); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + CommandResponse response = initializeResponse(); + if (zkServer instanceof QuorumZooKeeperServer) { + QuorumPeer peer = ((QuorumZooKeeperServer) zkServer).self; + QuorumPeer.ZabState zabState = peer.getZabState(); + QuorumVerifier qv = peer.getQuorumVerifier(); + + QuorumPeer.QuorumServer voter = qv.getVotingMembers().get(peer.getMyId()); + boolean voting = ( + voter != null + && voter.addr.equals(peer.getQuorumAddress()) + && voter.electionAddr.equals(peer.getElectionAddress()) + ); + response.put("myid", zkServer.getConf().getServerId()); + response.put("is_leader", zkServer instanceof LeaderZooKeeperServer); + response.put("quorum_address", peer.getQuorumAddress()); + response.put("election_address", peer.getElectionAddress()); + response.put("client_address", peer.getClientAddress()); + response.put("voting", voting); + long lastProcessedZxid = zkServer.getZKDatabase().getDataTreeLastProcessedZxid(); + response.put("last_zxid", "0x" + ZxidUtils.zxidToString(lastProcessedZxid)); + response.put("zab_epoch", ZxidUtils.getEpochFromZxid(lastProcessedZxid)); + response.put("zab_counter", ZxidUtils.getCounterFromZxid(lastProcessedZxid)); + response.put("zabstate", zabState.name().toLowerCase()); + } else { + response.put("voting", false); + response.put("zabstate", ""); + } + return response; + } + + } + + private Commands() { + } + +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/DummyAdminServer.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/DummyAdminServer.java new file mode 100644 index 0000000000000..4750ed2215f0d --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/DummyAdminServer.java @@ -0,0 +1,44 @@ +/* + * 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.zookeeper.server.admin; + +import org.apache.zookeeper.server.ZooKeeperServer; + +/** + * An AdminServer that does nothing. + * + * We use this class when we wish to disable the AdminServer. (This way we only + * have to consider whether the server is enabled when we create the + * AdminServer, which is handled by AdminServerFactory.) + */ +public class DummyAdminServer implements AdminServer { + + @Override + public void start() throws AdminServerException { + } + + @Override + public void shutdown() throws AdminServerException { + } + + @Override + public void setZooKeeperServer(ZooKeeperServer zkServer) { + } + +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/GetCommand.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/GetCommand.java new file mode 100644 index 0000000000000..90673a93066ba --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/GetCommand.java @@ -0,0 +1,47 @@ +/* + * 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.zookeeper.server.admin; + +import java.io.InputStream; +import java.util.List; +import org.apache.zookeeper.server.ZooKeeperServer; + +/** + * Command that represents HTTP GET request. + */ + +public abstract class GetCommand extends CommandBase { + + protected GetCommand(List names) { + super(names); + } + + protected GetCommand(List names, boolean serverRequired) { + super(names, serverRequired); + } + + protected GetCommand(List names, boolean serverRequired, AuthRequest authRequest) { + super(names, serverRequired, authRequest); + } + + @Override + public CommandResponse runPost(ZooKeeperServer zkServer, InputStream inputStream) { + return null; + } +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/JettyAdminServer.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/JettyAdminServer.java new file mode 100644 index 0000000000000..1a487c00fd03c --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/JettyAdminServer.java @@ -0,0 +1,397 @@ +/* + * 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.zookeeper.server.admin; + +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.security.KeyStore; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import org.apache.zookeeper.common.QuorumX509Util; +import org.apache.zookeeper.common.SecretUtils; +import org.apache.zookeeper.common.X509Util; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.auth.IPAuthenticationProvider; +import org.eclipse.jetty.ee8.nested.ServletConstraint; +import org.eclipse.jetty.ee8.security.ConstraintMapping; +import org.eclipse.jetty.ee8.security.ConstraintSecurityHandler; +import org.eclipse.jetty.ee8.servlet.ServletContextHandler; +import org.eclipse.jetty.ee8.servlet.ServletHolder; +import org.eclipse.jetty.http.HttpHeader; +import org.eclipse.jetty.http.HttpVersion; +import org.eclipse.jetty.server.DetectorConnectionFactory; +import org.eclipse.jetty.server.HttpConfiguration; +import org.eclipse.jetty.server.HttpConnectionFactory; +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.util.ssl.SslContextFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class encapsulates a Jetty server for running Commands. + * + * Given the default settings, start a ZooKeeper server and visit + * http://hostname:8080/commands for links to all registered commands. Visiting + * http://hostname:8080/commands/commandname will execute the associated + * Command and return the result in the body of the response. Any keyword + * arguments to the command are specified with URL parameters (e.g., + * http://localhost:8080/commands/set_trace_mask?traceMask=306). + * + * @see Commands + * @see CommandOutputter + */ +public class JettyAdminServer implements AdminServer { + + static final Logger LOG = LoggerFactory.getLogger(JettyAdminServer.class); + + public static final int DEFAULT_PORT = 8080; + public static final int DEFAULT_IDLE_TIMEOUT = 30000; + public static final String DEFAULT_COMMAND_URL = "/commands"; + private static final String DEFAULT_ADDRESS = "0.0.0.0"; + public static final int DEFAULT_STS_MAX_AGE = 1 * 24 * 60 * 60; // seconds in a day + public static final int DEFAULT_HTTP_VERSION = 11; // based on HttpVersion.java in jetty + + private final Server server; + private final String address; + private final int port; + private final int idleTimeout; + private final String commandUrl; + private ZooKeeperServer zkServer; + + public JettyAdminServer() throws AdminServerException, IOException, GeneralSecurityException { + this( + System.getProperty("zookeeper.admin.serverAddress", DEFAULT_ADDRESS), + Integer.getInteger("zookeeper.admin.serverPort", DEFAULT_PORT), + Integer.getInteger("zookeeper.admin.idleTimeout", DEFAULT_IDLE_TIMEOUT), + System.getProperty("zookeeper.admin.commandURL", DEFAULT_COMMAND_URL), + Integer.getInteger("zookeeper.admin.httpVersion", DEFAULT_HTTP_VERSION), + Boolean.getBoolean("zookeeper.admin.portUnification"), + Boolean.getBoolean("zookeeper.admin.forceHttps"), + Boolean.getBoolean("zookeeper.admin.needClientAuth")); + } + + public JettyAdminServer( + String address, + int port, + int timeout, + String commandUrl, + int httpVersion, + boolean portUnification, + boolean forceHttps, + boolean needClientAuth) throws IOException, GeneralSecurityException { + + this.port = port; + this.idleTimeout = timeout; + this.commandUrl = commandUrl; + this.address = address; + + server = new Server(); + ServerConnector connector = null; + + if (!portUnification && !forceHttps) { + connector = new ServerConnector(server); + } else { + SecureRequestCustomizer customizer = new SecureRequestCustomizer(); + customizer.setStsMaxAge(DEFAULT_STS_MAX_AGE); + customizer.setStsIncludeSubDomains(true); + + HttpConfiguration config = new HttpConfiguration(); + config.setSecureScheme("https"); + config.addCustomizer(customizer); + + try (QuorumX509Util x509Util = new QuorumX509Util()) { + String privateKeyType = System.getProperty(x509Util.getSslKeystoreTypeProperty(), ""); + String privateKeyPath = System.getProperty(x509Util.getSslKeystoreLocationProperty(), ""); + String privateKeyPassword = getPasswordFromSystemPropertyOrFile( + x509Util.getSslKeystorePasswdProperty(), + x509Util.getSslKeystorePasswdPathProperty()); + + String certAuthType = System.getProperty(x509Util.getSslTruststoreTypeProperty(), ""); + String certAuthPath = System.getProperty(x509Util.getSslTruststoreLocationProperty(), ""); + String certAuthPassword = getPasswordFromSystemPropertyOrFile( + x509Util.getSslTruststorePasswdProperty(), + x509Util.getSslTruststorePasswdPathProperty()); + KeyStore keyStore = null, trustStore = null; + + try { + keyStore = X509Util.loadKeyStore(privateKeyPath, privateKeyPassword, privateKeyType); + trustStore = X509Util.loadTrustStore(certAuthPath, certAuthPassword, certAuthType); + LOG.info("Successfully loaded private key from {}", privateKeyPath); + LOG.info("Successfully loaded certificate authority from {}", certAuthPath); + } catch (Exception e) { + LOG.error("Failed to load authentication certificates for admin server.", e); + throw e; + } + + SslContextFactory.Server sslContextFactory = new SslContextFactory.Server(); + sslContextFactory.setKeyStore(keyStore); + sslContextFactory.setKeyStorePassword(privateKeyPassword); + sslContextFactory.setTrustStore(trustStore); + sslContextFactory.setTrustStorePassword(certAuthPassword); + sslContextFactory.setNeedClientAuth(needClientAuth); + + SslConnectionFactory sslConnectionFactory = new SslConnectionFactory(sslContextFactory, + HttpVersion.fromVersion(httpVersion).asString()); + if (forceHttps) { + connector = new ServerConnector(server, + sslConnectionFactory, + new HttpConnectionFactory(config)); + } else { + connector = new ServerConnector( + server, + new DetectorConnectionFactory(sslConnectionFactory), + new HttpConnectionFactory(config)); + connector.setDefaultProtocol(sslConnectionFactory.getProtocol()); + } + } + } + + connector.setHost(address); + connector.setPort(port); + connector.setIdleTimeout(idleTimeout); + + server.addConnector(connector); + + ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS); + context.setContextPath("/*"); + constrainTraceMethod(context); + server.setHandler(context); + + context.addServlet(new ServletHolder(new CommandServlet()), commandUrl + "/*"); + } + + /** + * Start the embedded Jetty server. + */ + @Override + public void start() throws AdminServerException { + try { + server.start(); + } catch (Exception e) { + // Server.start() only throws Exception, so let's at least wrap it + // in an identifiable subclass + String message = String.format( + "Problem starting AdminServer on address %s, port %d and command URL %s", + address, + port, + commandUrl); + throw new AdminServerException(message, e); + } + LOG.info("Started AdminServer on address {}, port {} and command URL {}", address, port, commandUrl); + } + + /** + * Stop the embedded Jetty server. + * + * This is not very important except for tests where multiple + * JettyAdminServers are started and may try to bind to the same ports if + * previous servers aren't shut down. + */ + @Override + public void shutdown() throws AdminServerException { + try { + server.stop(); + } catch (Exception e) { + String message = String.format( + "Problem stopping AdminServer on address %s, port %d and command URL %s", + address, + port, + commandUrl); + throw new AdminServerException(message, e); + } + } + + /** + * Set the ZooKeeperServer that will be used to run Commands. + * + * It is not necessary to set the ZK server before calling + * AdminServer.start(), and the ZK server can be set to null when, e.g., + * that server is being shut down. If the ZK server is not set or set to + * null, the AdminServer will still be able to issue Commands, but they will + * return an error until a ZK server is set. + */ + @Override + public void setZooKeeperServer(ZooKeeperServer zkServer) { + this.zkServer = zkServer; + } + + private class CommandServlet extends HttpServlet { + + private static final long serialVersionUID = 1L; + + @Override + protected void doGet( + HttpServletRequest request, + HttpServletResponse response) throws ServletException, IOException { + // Capture the command name from the URL + String cmd = request.getPathInfo(); + if (cmd == null || cmd.equals("/")) { + // No command specified, print links to all commands instead + for (String link : commandLinks()) { + response.getWriter().println(link); + response.getWriter().println("
"); + } + return; + } + // Strip leading "/" + cmd = cmd.substring(1); + + // Extract keyword arguments to command from request parameters + @SuppressWarnings("unchecked") Map parameterMap = request.getParameterMap(); + Map kwargs = new HashMap<>(); + for (Map.Entry entry : parameterMap.entrySet()) { + kwargs.put(entry.getKey(), entry.getValue()[0]); + } + final String authInfo = request.getHeader(HttpHeader.AUTHORIZATION.asString()); + + // Run the command + final CommandResponse cmdResponse = Commands.runGetCommand(cmd, zkServer, kwargs, authInfo, request); + response.setStatus(cmdResponse.getStatusCode()); + + final Map headers = cmdResponse.getHeaders(); + for (final Map.Entry header : headers.entrySet()) { + response.addHeader(header.getKey(), header.getValue()); + } + final String clientIP = IPAuthenticationProvider.getClientIPAddress(request); + if (cmdResponse.getInputStream() == null) { + // Format and print the output of the command + CommandOutputter outputter = new JsonOutputter(clientIP); + response.setContentType(outputter.getContentType()); + outputter.output(cmdResponse, response.getWriter()); + } else { + // Stream out the output of the command + CommandOutputter outputter = new StreamOutputter(clientIP); + response.setContentType(outputter.getContentType()); + outputter.output(cmdResponse, response.getOutputStream()); + } + } + + /** + * Serves HTTP POST requests. It reads request payload as raw data. + * It's up to each command to process the payload accordingly. + * For example, RestoreCommand uses the payload InputStream directly + * to read snapshot data. + */ + @Override + protected void doPost(final HttpServletRequest request, + final HttpServletResponse response) throws ServletException, IOException { + final String cmdName = extractCommandNameFromURL(request, response); + if (cmdName != null) { + final String authInfo = request.getHeader(HttpHeader.AUTHORIZATION.asString()); + final CommandResponse cmdResponse = + Commands.runPostCommand(cmdName, zkServer, request.getInputStream(), authInfo, request); + final String clientIP = IPAuthenticationProvider.getClientIPAddress(request); + sendJSONResponse(response, cmdResponse, clientIP); + } + } + + /** + * Extracts the command name from URL if it exists otherwise null. + */ + private String extractCommandNameFromURL(final HttpServletRequest request, + final HttpServletResponse response) throws IOException { + String cmd = request.getPathInfo(); + if (cmd == null || cmd.equals("/")) { + printCommandLinks(response); + return null; + } + // Strip leading "/" + return cmd.substring(1); + } + + /** + * Prints the list of URLs to each registered command as response. + */ + private void printCommandLinks(final HttpServletResponse response) throws IOException { + for (final String link : commandLinks()) { + response.getWriter().println(link); + response.getWriter().println("
"); + } + } + + /** + * Send JSON string as the response. + */ + private void sendJSONResponse(final HttpServletResponse response, + final CommandResponse cmdResponse, + final String clientIP) throws IOException { + final CommandOutputter outputter = new JsonOutputter(clientIP); + + response.setStatus(cmdResponse.getStatusCode()); + response.setContentType(outputter.getContentType()); + outputter.output(cmdResponse, response.getWriter()); + } + } + + /** + * Returns a list of URLs to each registered Command. + */ + private List commandLinks() { + return Commands.getPrimaryNames().stream().sorted() + .map(command -> String.format("%s", commandUrl + "/" + command, command)) + .collect(Collectors.toList()); + } + + /** + * Add constraint to a given context to disallow TRACE method. + * + * @param ctxHandler the context to modify + */ + private void constrainTraceMethod(ServletContextHandler ctxHandler) { + ServletConstraint c = new ServletConstraint(); + c.setAuthenticate(true); + + ConstraintMapping cmt = new ConstraintMapping(); + cmt.setConstraint(c); + cmt.setMethod("TRACE"); + cmt.setPathSpec("/*"); + + ConstraintSecurityHandler securityHandler = new ConstraintSecurityHandler(); + securityHandler.setConstraintMappings(new ConstraintMapping[]{cmt}); + + ctxHandler.setSecurityHandler(securityHandler); + } + + /** + * Returns the password specified by the given property or stored in the file specified by the + * given path property. If both are specified, the password stored in the file will be returned. + * + * @param propertyName the name of the property + * @param pathPropertyName the name of the path property + * @return password value + */ + private String getPasswordFromSystemPropertyOrFile(final String propertyName, + final String pathPropertyName) { + String value = System.getProperty(propertyName, ""); + final String pathValue = System.getProperty(pathPropertyName, ""); + if (!pathValue.isEmpty()) { + value = String.valueOf(SecretUtils.readSecret(pathValue)); + } + return value; + } +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/JsonOutputter.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/JsonOutputter.java new file mode 100644 index 0000000000000..631f43508c78b --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/JsonOutputter.java @@ -0,0 +1,69 @@ +/* + * 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.zookeeper.server.admin; + +import com.fasterxml.jackson.core.JsonGenerationException; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.PropertyNamingStrategies; +import com.fasterxml.jackson.databind.SerializationFeature; +import java.io.IOException; +import java.io.PrintWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JsonOutputter implements CommandOutputter { + + static final Logger LOG = LoggerFactory.getLogger(JsonOutputter.class); + + public static final String ERROR_RESPONSE = "{\"error\": \"Exception writing command response to JSON\"}"; + + private ObjectMapper mapper; + private final String clientIP; + + public JsonOutputter(final String clientIP) { + mapper = new ObjectMapper(); + mapper.configure(SerializationFeature.WRITE_ENUMS_USING_TO_STRING, true); + mapper.configure(SerializationFeature.INDENT_OUTPUT, true); + mapper.setPropertyNamingStrategy(PropertyNamingStrategies.SNAKE_CASE); + this.clientIP = clientIP; + } + + @Override + public String getContentType() { + return "application/json"; + } + + @Override + public void output(CommandResponse response, PrintWriter pw) { + try { + mapper.writeValue(pw, response.toMap()); + } catch (JsonGenerationException e) { + LOG.warn("Exception writing command response to JSON:", e); + pw.write(ERROR_RESPONSE); + } catch (JsonMappingException e) { + LOG.warn("Exception writing command response to JSON:", e); + pw.write(ERROR_RESPONSE); + } catch (IOException e) { + LOG.warn("Exception writing command response as JSON to {}", clientIP, e); + pw.write(ERROR_RESPONSE); + } + } + +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/PostCommand.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/PostCommand.java new file mode 100644 index 0000000000000..23795ed34ae7f --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/PostCommand.java @@ -0,0 +1,37 @@ +/* + * 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. + */ +/** + * Command that represents HTTP POST request + */ +package org.apache.zookeeper.server.admin; + +import java.util.List; +import java.util.Map; +import org.apache.zookeeper.server.ZooKeeperServer; + +public abstract class PostCommand extends CommandBase { + protected PostCommand(List names, boolean serverRequired, AuthRequest authRequest) { + super(names, serverRequired, authRequest); + } + + @Override + public CommandResponse runGet(ZooKeeperServer zkServer, Map kwargs) { + return null; + } +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/StreamOutputter.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/StreamOutputter.java new file mode 100644 index 0000000000000..f4d9ec5c13dd7 --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/StreamOutputter.java @@ -0,0 +1,52 @@ +/* + * 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.zookeeper.server.admin; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.zookeeper.common.IOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A class for streaming data out. + */ +public class StreamOutputter implements CommandOutputter { + private static final Logger LOG = LoggerFactory.getLogger(StreamOutputter.class); + private final String clientIP; + + public StreamOutputter(final String clientIP) { + this.clientIP = clientIP; + } + + @Override + public String getContentType() { + return "application/octet-stream"; + } + + @Override + public void output(final CommandResponse response, final OutputStream os) { + try (final InputStream is = response.getInputStream()) { + IOUtils.copyBytes(is, os, 1024, true); + } catch (final IOException e) { + LOG.warn("Exception streaming out data to {}", clientIP, e); + } + } +} diff --git a/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/package-info.java b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/package-info.java new file mode 100644 index 0000000000000..85e5a06e70604 --- /dev/null +++ b/jetty-upgrade/zookeeper-with-patched-admin/src/main/java/org/apache/zookeeper/server/admin/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +/** + * Admin server for ZooKeeper that is using Jetty 12. + */ +package org.apache.zookeeper.server.admin; diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index b9a70441679dd..ef6a03d107040 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -36,23 +36,12 @@ org.apache.bookkeeper bookkeeper-server
- - org.apache.bookkeeper.stats - prometheus-metrics-provider + org.apache.pulsar + zookeeper-with-patched-admin + ${project.version} - - org.apache.bookkeeper.stats - codahale-metrics-provider - ${bookkeeper.version} - - - amqp-client - com.rabbitmq - - - com.google.protobuf diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 1cdeb415b7c21..08885739b99b5 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -69,6 +69,7 @@ import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutorService; @@ -150,7 +151,6 @@ import org.apache.pulsar.metadata.impl.FaultInjectionMetadataStore; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; -import org.eclipse.jetty.util.BlockingArrayQueue; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.DataProvider; @@ -3339,8 +3339,8 @@ public void testAddEntryResponseTimeout() throws Exception { private static class CollectCtxAddEntryCallback implements AddEntryCallback { - public List addCompleteCtxList = new BlockingArrayQueue<>(); - public List addFailedCtxList = new BlockingArrayQueue<>(); + public List addCompleteCtxList = new CopyOnWriteArrayList<>(); + public List addFailedCtxList = new CopyOnWriteArrayList<>(); @Override public void addComplete(Position position, ByteBuf entryData, Object ctx) { diff --git a/pom.xml b/pom.xml index 3f4587dc2445c..4d91aa1912a88 100644 --- a/pom.xml +++ b/pom.xml @@ -191,7 +191,9 @@ flexible messaging model and an intuitive client API. 5.7.1 4.1.130.Final 0.0.26.Final - 9.4.58.v20250814 + 12.1.5 + + 9.4.58.v20250814 2.5.2 2.42 1.10.62 @@ -204,6 +206,7 @@ flexible messaging model and an intuitive client API. 1.78.1 1.81 + 1.81 1.78.1 2.0.10 2.0.1 @@ -282,7 +285,7 @@ flexible messaging model and an intuitive client API. 1.2.2 2.3.3 2.0.2 - 5.12.1 + 5.18.1 23.0.0 0.9.6 5.3.1 @@ -555,6 +558,11 @@ flexible messaging model and an intuitive client API. io.netty netty-* + + + org.apache.zookeeper + zookeeper + @@ -716,12 +724,6 @@ flexible messaging model and an intuitive client API. - - org.apache.bookkeeper.stats - prometheus-metrics-provider - ${bookkeeper.version} - - com.google.re2j re2j @@ -746,18 +748,6 @@ flexible messaging model and an intuitive client API. ${jose4j.version} - - org.eclipse.jetty - jetty-server - ${jetty.version} - - - - org.eclipse.jetty - jetty-alpn-conscrypt-server - ${jetty.version} - - org.conscrypt conscrypt-openjdk-uber @@ -765,17 +755,17 @@ flexible messaging model and an intuitive client API. - org.eclipse.jetty - jetty-bom - ${jetty.version} + io.netty + netty-bom + ${netty.version} pom import - io.netty - netty-bom - ${netty.version} + com.fasterxml.jackson + jackson-bom + ${jackson.version} pom import @@ -885,36 +875,6 @@ flexible messaging model and an intuitive client API. ${commons-codec.version} - - org.glassfish.jersey.core - jersey-server - ${jersey.version} - - - - org.glassfish.jersey.core - jersey-client - ${jersey.version} - - - - org.glassfish.jersey.inject - jersey-hk2 - ${jersey.version} - - - - org.glassfish.jersey.containers - jersey-container-servlet-core - ${jersey.version} - - - - org.glassfish.jersey.containers - jersey-container-servlet - ${jersey.version} - - jakarta.ws.rs jakarta.ws.rs-api @@ -922,26 +882,23 @@ flexible messaging model and an intuitive client API. - org.glassfish.jersey.media - jersey-media-json-jackson - ${jersey.version} - - - jakarta.activation - jakarta.activation-api - - + net.java.dev.jna + jna + ${jna.version} - - org.glassfish.jersey.media - jersey-media-multipart - ${jersey.version} + net.java.dev.jna + jna-platform + ${jna.version} - net.java.dev.jna - jna + jna-jpms + ${jna.version} + + + net.java.dev.jna + jna-platform-jpms ${jna.version} @@ -967,14 +924,6 @@ flexible messaging model and an intuitive client API. ${docker-java.version} - - com.fasterxml.jackson - jackson-bom - ${jackson.version} - pom - import - - it.unimi.dsi fastutil @@ -1042,6 +991,12 @@ flexible messaging model and an intuitive client API. ${bouncycastle.bcpkix-jdk18on.version} + + org.bouncycastle + bcutil-jdk18on + ${bouncycastle.bcutil-jdk18on.version} + + org.bouncycastle bcprov-ext-jdk18on @@ -1372,6 +1327,10 @@ flexible messaging model and an intuitive client API. commons-lang commons-lang + + org.apache.zookeeper + zookeeper + @@ -1752,6 +1711,54 @@ flexible messaging model and an intuitive client API. commons-configuration2 ${commons-configuration2.version} + + + org.glassfish.jersey + jersey-bom + ${jersey.version} + pom + import + + + + org.glassfish.jersey.media + jersey-media-json-jackson + ${jersey.version} + + + jakarta.activation + jakarta.activation-api + + + + + + org.eclipse.jetty + jetty-bom + ${jetty.version} + pom + import + + + + org.eclipse.jetty.ee8 + jetty-ee8 + ${jetty.version} + pom + import + + + + org.apache.pulsar + zookeeper-with-patched-admin + ${project.version} + + + org.apache.zookeeper + zookeeper + + + @@ -2674,6 +2681,7 @@ flexible messaging model and an intuitive client API. tests microbench + jetty-upgrade @@ -2732,6 +2740,7 @@ flexible messaging model and an intuitive client API. pulsar-package-management + jetty-upgrade diff --git a/pulsar-broker-auth-oidc/pom.xml b/pulsar-broker-auth-oidc/pom.xml index 14ec21bb58ee7..83715bff923a0 100644 --- a/pulsar-broker-auth-oidc/pom.xml +++ b/pulsar-broker-auth-oidc/pom.xml @@ -114,7 +114,7 @@ com.github.tomakehurst - wiremock-jre8 + wiremock-jre8-standalone ${wiremock.version} test diff --git a/pulsar-broker-common/pom.xml b/pulsar-broker-common/pom.xml index 50ce9f466c32f..644d098bcc65d 100644 --- a/pulsar-broker-common/pom.xml +++ b/pulsar-broker-common/pom.xml @@ -45,11 +45,6 @@ guava - - io.prometheus - simpleclient_jetty - - io.opentelemetry opentelemetry-api @@ -75,6 +70,26 @@ jjwt-jackson + + org.eclipse.jetty + jetty-server + + + + org.eclipse.jetty.compression + jetty-compression-server + + + + org.eclipse.jetty.compression + jetty-compression-gzip + + + + org.eclipse.jetty.ee8 + jetty-ee8-servlet + + org.bouncycastle diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java index 9e980cecb791f..a1e28c864b4ce 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java @@ -19,9 +19,11 @@ package org.apache.pulsar.broker.web; import java.util.List; +import org.eclipse.jetty.compression.gzip.GzipCompression; +import org.eclipse.jetty.compression.server.CompressionConfig; +import org.eclipse.jetty.compression.server.CompressionHandler; import org.eclipse.jetty.http.pathmap.PathSpecSet; import org.eclipse.jetty.server.Handler; -import org.eclipse.jetty.server.handler.gzip.GzipHandler; import org.eclipse.jetty.util.IncludeExclude; public class GzipHandlerUtil { @@ -32,10 +34,13 @@ public static Handler wrapWithGzipHandler(Handler innerHandler, List gzi wrappedHandler = innerHandler; } else { // add GZIP handler which is active when the request contains "Accept-Encoding: gzip" header - GzipHandler gzipHandler = new GzipHandler(); + CompressionHandler gzipHandler = new CompressionHandler(); + gzipHandler.putCompression(new GzipCompression()); gzipHandler.setHandler(innerHandler); if (gzipCompressionExcludedPaths != null && gzipCompressionExcludedPaths.size() > 0) { - gzipHandler.setExcludedPaths(gzipCompressionExcludedPaths.toArray(new String[0])); + CompressionConfig.Builder compressConfigBuilder = CompressionConfig.builder().defaults(); + gzipCompressionExcludedPaths.forEach(compressConfigBuilder::compressExcludePath); + gzipHandler.putConfiguration("/", compressConfigBuilder.build()); } wrappedHandler = gzipHandler; } 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 fc88647eb49ea..b8a777dbe8852 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 @@ -19,12 +19,14 @@ package org.apache.pulsar.broker.web; import java.net.InetSocketAddress; +import java.net.SocketAddress; 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.ConnectionMetaData; import org.eclipse.jetty.server.Connector; import org.eclipse.jetty.server.CustomRequestLog; import org.eclipse.jetty.server.ProxyConnectionFactory; @@ -33,6 +35,7 @@ import org.eclipse.jetty.server.Response; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.Slf4jRequestLogWriter; +import org.eclipse.jetty.util.Attributes; import org.eclipse.jetty.util.HostPort; import org.eclipse.jetty.util.component.ContainerLifeCycle; @@ -128,14 +131,18 @@ public void log(Request request, Response response) { delegate.log(request, response); StringBuilder sb = requestLogStringBuilder.get(); sb.append(" [R:"); - sb.append(request.getRemoteHost()); + String remoteAddr = Request.getRemoteAddr(request); + sb.append(remoteAddr); sb.append(':'); - sb.append(request.getRemotePort()); - InetSocketAddress realRemoteAddress = lookupRealAddress(request.getHttpChannel().getRemoteAddress()); + int remotePort = Request.getRemotePort(request); + sb.append(remotePort); + + InetSocketAddress realRemoteAddress = + lookupRealAddress(unwrap(request.getConnectionMetaData()).getRemoteSocketAddress()); if (realRemoteAddress != null) { String realRemoteHost = HostPort.normalizeHost(realRemoteAddress.getHostString()); int realRemotePort = realRemoteAddress.getPort(); - if (!realRemoteHost.equals(request.getRemoteHost()) || realRemotePort != request.getRemotePort()) { + if (!realRemoteHost.equals(remoteAddr) || realRemotePort != remotePort) { sb.append(" via "); sb.append(realRemoteHost); sb.append(':'); @@ -143,23 +150,26 @@ public void log(Request request, Response response) { } } sb.append("]->[L:"); - InetSocketAddress realLocalAddress = lookupRealAddress(request.getHttpChannel().getLocalAddress()); + InetSocketAddress realLocalAddress = lookupRealAddress(unwrap(request.getConnectionMetaData()) + .getLocalSocketAddress()); + String localAddr = Request.getLocalAddr(request); + int localPort = Request.getLocalPort(request); 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()) { + if (!realLocalHost.equals(localAddr) || realLocalPort != localPort) { sb.append(" dst "); - sb.append(request.getLocalAddr()); + sb.append(localAddr); sb.append(':'); - sb.append(request.getLocalPort()); + sb.append(localPort); } } else { - sb.append(request.getLocalAddr()); + sb.append(localAddr); sb.append(':'); - sb.append(request.getLocalPort()); + sb.append(localPort); } sb.append(']'); try { @@ -169,19 +179,27 @@ public void log(Request request, Response response) { } } - private InetSocketAddress lookupRealAddress(InetSocketAddress socketAddress) { - if (socketAddress == null) { + private ConnectionMetaData unwrap(ConnectionMetaData connectionMetaData) { + if (connectionMetaData instanceof Attributes) { + return (ConnectionMetaData) Attributes.unwrap((Attributes) connectionMetaData); + } + return connectionMetaData; + } + + private InetSocketAddress lookupRealAddress(SocketAddress socketAddress) { + if (socketAddress == null || !(socketAddress instanceof InetSocketAddress)) { return null; } + InetSocketAddress inetSocketAddress = (InetSocketAddress) socketAddress; if (proxyProtocolRealAddressMapping.isEmpty()) { - return socketAddress; + return inetSocketAddress; } - AddressEntry entry = proxyProtocolRealAddressMapping.get(new AddressKey(socketAddress.getHostString(), - socketAddress.getPort())); + AddressEntry entry = proxyProtocolRealAddressMapping.get(new AddressKey(inetSocketAddress.getHostString(), + inetSocketAddress.getPort())); if (entry != null) { return entry.realAddress; } else { - return socketAddress; + return inetSocketAddress; } } @@ -221,8 +239,12 @@ private void handleConnection(Connection connection, boolean increment) { ProxyConnectionFactory.ProxyEndPoint proxyEndPoint = (ProxyConnectionFactory.ProxyEndPoint) connection.getEndPoint(); EndPoint originalEndpoint = proxyEndPoint.unwrap(); - mapAddress(proxyEndPoint.getLocalAddress(), originalEndpoint.getLocalAddress(), increment); - mapAddress(proxyEndPoint.getRemoteAddress(), originalEndpoint.getRemoteAddress(), increment); + mapAddress((InetSocketAddress) proxyEndPoint.getLocalSocketAddress(), + (InetSocketAddress) originalEndpoint.getLocalSocketAddress(), + increment); + mapAddress((InetSocketAddress) proxyEndPoint.getRemoteSocketAddress(), + (InetSocketAddress) originalEndpoint.getRemoteSocketAddress(), + increment); } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServlet.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServlet.java index 1ecf4c4e53943..6d817bfb9bd47 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServlet.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServlet.java @@ -20,13 +20,19 @@ import com.google.common.annotations.Beta; import org.apache.pulsar.common.configuration.PulsarConfiguration; -import org.eclipse.jetty.servlet.ServletHolder; /** * The additional servlet interface for support additional servlet. */ @Beta public interface AdditionalServlet extends AutoCloseable { + /** + * The servlet implementation type enum. + * Currently, only {@link AdditionalServletType#JAVAX_SERVLET} is supported. + */ + enum AdditionalServletType { + JAVAX_SERVLET + } /** * load plugin config. @@ -43,11 +49,24 @@ public interface AdditionalServlet extends AutoCloseable { String getBasePath(); /** - * Get the servlet holder. + * Get the servlet type that this implementation uses. + */ + default AdditionalServletType getServletType() { + return AdditionalServletType.JAVAX_SERVLET; + } + + /** + * Retrieves the servlet instance for this additional servlet implementation. + *

+ * The returned object's type must be compatible with the servlet interface class + * specified by the {@link AdditionalServletType} returned from {@link #getServletType()}. + * For example, if {@link #getServletType()} returns {@link AdditionalServletType#JAVAX_SERVLET}, + * the returned object must implement {@code javax.servlet.Servlet}. + *

* - * @return the servlet holder + * @return the servlet instance implementing the appropriate servlet interface */ - ServletHolder getServletHolder(); + Object getServletInstance(); @Override void close(); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java index bc1f25c5af933..7fc261ecb7b18 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java @@ -24,7 +24,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.configuration.PulsarConfiguration; import org.apache.pulsar.common.nar.NarClassLoader; -import org.eclipse.jetty.servlet.ServletHolder; /** * An additional servlet with it's classloader. @@ -60,11 +59,22 @@ public String getBasePath() { } @Override - public ServletHolder getServletHolder() { + public AdditionalServletType getServletType() { ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); try { Thread.currentThread().setContextClassLoader(classLoader); - return servlet.getServletHolder(); + return servlet.getServletType(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); + } + } + + @Override + public Object getServletInstance() { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); + return servlet.getServletInstance(); } finally { Thread.currentThread().setContextClassLoader(prevClassLoader); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/metrics/JettyStatisticsCollector.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/metrics/JettyStatisticsCollector.java new file mode 100644 index 0000000000000..c4facc315153d --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/metrics/JettyStatisticsCollector.java @@ -0,0 +1,111 @@ +/* + * 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.jetty.metrics; + +import io.prometheus.client.Collector; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.eclipse.jetty.server.handler.StatisticsHandler; + +/** + * Collect Prometheus metrics from jetty's org.eclipse.jetty.server.handler.StatisticsHandler. + * + * This is ported from prometheus client_java 0.16 version of JettyStatisticsCollector which is using Jetty 9.x. + * This supports Jetty 12.x. + */ +public class JettyStatisticsCollector extends Collector { + private final StatisticsHandler statisticsHandler; + private static final List EMPTY_LIST = new ArrayList(); + + public JettyStatisticsCollector(StatisticsHandler statisticsHandler) { + this.statisticsHandler = statisticsHandler; + } + + @Override + public List collect() { + return Arrays.asList( + buildCounter("jetty_requests_total", "Number of requests", statisticsHandler.getRequests()), + buildGauge("jetty_requests_active", "Number of requests currently active", + statisticsHandler.getRequestsActive()), + buildGauge("jetty_requests_active_max", "Maximum number of requests that have been active at once", + statisticsHandler.getRequestsActiveMax()), + buildGauge("jetty_request_time_max_seconds", "Maximum time spent handling requests", + statisticsHandler.getRequestTimeMax() / 1000_000_000.0), + buildCounter("jetty_request_time_seconds_total", "Total time spent in all request handling", + statisticsHandler.getRequestTimeTotal() / 1000_000_000.0), + buildCounter("jetty_dispatched_total", "Number of dispatches", statisticsHandler.getHandleTotal()), + buildGauge("jetty_dispatched_active", "Number of dispatches currently active", + statisticsHandler.getHandleActive()), + buildGauge("jetty_dispatched_active_max", "Maximum number of active dispatches being handled", + statisticsHandler.getHandleActiveMax()), + buildGauge("jetty_dispatched_time_max", "Maximum time spent in dispatch handling", + statisticsHandler.getHandleTimeMax() / 1000_000_000.0), + buildCounter("jetty_dispatched_time_seconds_total", "Total time spent in dispatch handling", + statisticsHandler.getHandleTimeTotal() / 1000_000_000.0), + buildStatusCounter(), + buildGauge("jetty_stats_seconds", "Time in seconds stats have been collected for", + statisticsHandler.getStatisticsDuration().toNanos() / 1000_000_000.0), + buildCounter("jetty_responses_bytes_total", "Total number of bytes across all responses", + statisticsHandler.getBytesRead() + statisticsHandler.getBytesWritten()) + ); + } + + private static MetricFamilySamples buildGauge(String name, String help, double value) { + return new MetricFamilySamples( + name, + Type.GAUGE, + help, + Collections.singletonList(new MetricFamilySamples.Sample(name, EMPTY_LIST, EMPTY_LIST, value))); + } + + private static MetricFamilySamples buildCounter(String name, String help, double value) { + return new MetricFamilySamples( + name, + Type.COUNTER, + help, + Collections.singletonList(new MetricFamilySamples.Sample(name, EMPTY_LIST, EMPTY_LIST, value))); + } + + private MetricFamilySamples buildStatusCounter() { + String name = "jetty_responses_total"; + return new MetricFamilySamples( + name, + Type.COUNTER, + "Number of requests with response status", + Arrays.asList( + buildStatusSample(name, "1xx", statisticsHandler.getResponses1xx()), + buildStatusSample(name, "2xx", statisticsHandler.getResponses2xx()), + buildStatusSample(name, "3xx", statisticsHandler.getResponses3xx()), + buildStatusSample(name, "4xx", statisticsHandler.getResponses4xx()), + buildStatusSample(name, "5xx", statisticsHandler.getResponses5xx()) + ) + ); + } + + private static MetricFamilySamples.Sample buildStatusSample(String name, String status, double value) { + return new MetricFamilySamples.Sample( + name, + Collections.singletonList("code"), + Collections.singletonList(status), + value); + } +} \ No newline at end of file diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/metrics/package-info.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/metrics/package-info.java new file mode 100644 index 0000000000000..53cc77e93ea21 --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/metrics/package-info.java @@ -0,0 +1,19 @@ +/* + * 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.jetty.metrics; diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/JettySslContextFactory.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/JettySslContextFactory.java index 0ac1b78ca993f..f99df06b73c54 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/JettySslContextFactory.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/JettySslContextFactory.java @@ -68,6 +68,10 @@ public Server(String sslProviderString, PulsarSslFactory pulsarSslFactory, this.setWantClientAuth(true); this.setTrustAll(true); } + + // https://jetty.org/docs/jetty/12.1/operations-guide/protocols/index.html#ssl-sni + // Set to false for backwards compatibility with Jetty 9.x + setSniRequired(false); } @Override diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/plugin/servlet/MockAdditionalServlet.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/plugin/servlet/MockAdditionalServlet.java index 65cc10e0d3bb7..492b541d6f98b 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/plugin/servlet/MockAdditionalServlet.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/plugin/servlet/MockAdditionalServlet.java @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.web.plugin.servlet; import org.apache.pulsar.common.configuration.PulsarConfiguration; -import org.eclipse.jetty.servlet.ServletHolder; public class MockAdditionalServlet implements AdditionalServlet { @@ -34,7 +33,7 @@ public String getBasePath() { } @Override - public ServletHolder getServletHolder() { + public Object getServletInstance() { return null; } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryTest.java index 3a4664c920886..6d8f38fbb5093 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryTest.java @@ -66,7 +66,7 @@ public void testJettyTlsServerTls() throws Exception { PulsarSslFactory sslFactory = new DefaultPulsarSslFactory(); sslFactory.initialize(sslConfiguration); sslFactory.createInternalSslContext(); - SslContextFactory factory = JettySslContextFactory.createSslContextFactory(null, + SslContextFactory.Server factory = JettySslContextFactory.createSslContextFactory(null, sslFactory, true, null, null); ServerConnector connector = new ServerConnector(server, factory); @@ -109,7 +109,7 @@ public void testJettyTlsServerInvalidTlsProtocol() throws Exception { PulsarSslFactory sslFactory = new DefaultPulsarSslFactory(); sslFactory.initialize(sslConfiguration); sslFactory.createInternalSslContext(); - SslContextFactory factory = JettySslContextFactory.createSslContextFactory(null, + SslContextFactory.Server factory = JettySslContextFactory.createSslContextFactory(null, sslFactory, true, null, new HashSet() { { @@ -162,7 +162,7 @@ public void testJettyTlsServerInvalidCipher() throws Exception { PulsarSslFactory sslFactory = new DefaultPulsarSslFactory(); sslFactory.initialize(sslConfiguration); sslFactory.createInternalSslContext(); - SslContextFactory factory = JettySslContextFactory.createSslContextFactory(null, + SslContextFactory.Server factory = JettySslContextFactory.createSslContextFactory(null, sslFactory, true, new HashSet() { { diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index e72b2dbedda65..328b0d377d57a 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -169,7 +169,7 @@ com.github.tomakehurst - wiremock-jre8 + wiremock-jre8-standalone ${wiremock.version} test @@ -187,6 +187,18 @@ test
+ + org.eclipse.jetty.websocket + jetty-websocket-jetty-client + test + + + org.eclipse.jetty.ee8 + jetty-ee8-proxy + test + + + io.dropwizard.metrics @@ -247,13 +259,13 @@ - org.eclipse.jetty - jetty-servlet + org.eclipse.jetty.ee8 + jetty-ee8-servlet - org.eclipse.jetty - jetty-servlets + org.eclipse.jetty.ee8 + jetty-ee8-servlets @@ -280,14 +292,12 @@ org.glassfish.jersey.test-framework jersey-test-framework-core test - ${jersey.version} org.glassfish.jersey.test-framework.providers jersey-test-framework-provider-grizzly2 test - ${jersey.version} @@ -352,11 +362,6 @@ simpleclient - - io.prometheus - simpleclient_jetty - - io.prometheus simpleclient_hotspot 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 e61fbfac56622..d698a568d9f4a 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 @@ -62,8 +62,8 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import java.util.function.Supplier; +import javax.servlet.Servlet; import javax.servlet.ServletException; -import javax.websocket.DeploymentException; import lombok.AccessLevel; import lombok.Getter; import lombok.Setter; @@ -198,8 +198,8 @@ import org.apache.pulsar.websocket.WebSocketReaderServlet; import org.apache.pulsar.websocket.WebSocketService; import org.apache.pulsar.zookeeper.DefaultMetadataNodeSizeStats; -import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.websocket.servlet.WebSocketServlet; +import org.eclipse.jetty.ee8.servlet.ServletHolder; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServlet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1172,8 +1172,7 @@ private synchronized void createMetricsServlet() { private void addWebServerHandlers(WebService webService, PulsarPrometheusMetricsServlet metricsServlet, ServiceConfiguration config) - throws PulsarServerException, PulsarClientException, MalformedURLException, ServletException, - DeploymentException { + throws PulsarServerException, PulsarClientException, MalformedURLException, ServletException { Map attributeMap = new HashMap<>(); attributeMap.put(WebService.ATTRIBUTE_PULSAR_NAME, this); @@ -1239,9 +1238,37 @@ private void addBrokerAdditionalServlets(WebService webService, if (additionalServlet instanceof AdditionalServletWithPulsarService) { ((AdditionalServletWithPulsarService) additionalServlet).setPulsarService(this); } - webService.addServlet(servletWithClassLoader.getBasePath(), servletWithClassLoader.getServletHolder(), - config.isAuthenticationEnabled(), attributeMap); - LOG.info("Broker add additional servlet basePath {} ", servletWithClassLoader.getBasePath()); + switch (servletWithClassLoader.getServletType()) { + case JAVAX_SERVLET -> { + Object servletInstance = servletWithClassLoader.getServletInstance(); + if (!(servletInstance instanceof javax.servlet.Servlet)) { + LOG.error("AdditionalServletWithClassLoader {} has invalid servlet instance type {} which " + + "doesn't match {}. Skipping.", servletWithClassLoader, + servletInstance.getClass().getName(), servletWithClassLoader.getServletType()); + try { + servletWithClassLoader.close(); + } catch (Exception e) { + LOG.error("Failed to close servlet {}.", servletWithClassLoader, e); + } + continue; + } + ServletHolder servletHolder = + new ServletHolder((Servlet) servletInstance); + webService.addServlet(servletWithClassLoader.getBasePath(), servletHolder, + config.isAuthenticationEnabled(), attributeMap); + LOG.info("Broker add additional servlet basePath {} ", servletWithClassLoader.getBasePath()); + } + default -> { + LOG.error("AdditionalServletWithClassLoader {} has unsupported servlet type {}. Skipping.", + servletWithClassLoader, servletWithClassLoader.getServletType()); + try { + servletWithClassLoader.close(); + } catch (Exception e) { + LOG.error("Failed to close servlet {}.", servletWithClassLoader, e); + } + continue; + } + } } } } @@ -1249,35 +1276,31 @@ private void addBrokerAdditionalServlets(WebService webService, private void addWebSocketServiceHandler(WebService webService, Map attributeMap, ServiceConfiguration config) - throws PulsarServerException, PulsarClientException, MalformedURLException, ServletException, - DeploymentException { + throws PulsarServerException, PulsarClientException, MalformedURLException, ServletException { if (config.isWebSocketServiceEnabled()) { // Use local broker address to avoid different IP address when using a VIP for service discovery this.webSocketService = new WebSocketService(null, config); this.webSocketService.start(); - final WebSocketServlet producerWebSocketServlet = new WebSocketProducerServlet(webSocketService); - webService.addServlet(WebSocketProducerServlet.SERVLET_PATH, - new ServletHolder(producerWebSocketServlet), true, attributeMap); - webService.addServlet(WebSocketProducerServlet.SERVLET_PATH_V2, - new ServletHolder(producerWebSocketServlet), true, attributeMap); - - final WebSocketServlet consumerWebSocketServlet = new WebSocketConsumerServlet(webSocketService); - webService.addServlet(WebSocketConsumerServlet.SERVLET_PATH, - new ServletHolder(consumerWebSocketServlet), true, attributeMap); - webService.addServlet(WebSocketConsumerServlet.SERVLET_PATH_V2, - new ServletHolder(consumerWebSocketServlet), true, attributeMap); - - final WebSocketServlet readerWebSocketServlet = new WebSocketReaderServlet(webSocketService); - webService.addServlet(WebSocketReaderServlet.SERVLET_PATH, - new ServletHolder(readerWebSocketServlet), true, attributeMap); - webService.addServlet(WebSocketReaderServlet.SERVLET_PATH_V2, - new ServletHolder(readerWebSocketServlet), true, attributeMap); - - final WebSocketMultiTopicConsumerServlet multiTopicConsumerWebSocketServlet = - new WebSocketMultiTopicConsumerServlet(webSocketService); - webService.addServlet(WebSocketMultiTopicConsumerServlet.SERVLET_PATH, - new ServletHolder(multiTopicConsumerWebSocketServlet), true, attributeMap); + addWebSocketServlet(new WebSocketProducerServlet(webSocketService), attributeMap, + WebSocketProducerServlet.SERVLET_PATH, WebSocketProducerServlet.SERVLET_PATH_V2); + + addWebSocketServlet(new WebSocketConsumerServlet(webSocketService), attributeMap, + WebSocketConsumerServlet.SERVLET_PATH, WebSocketConsumerServlet.SERVLET_PATH_V2); + + addWebSocketServlet(new WebSocketReaderServlet(webSocketService), attributeMap, + WebSocketReaderServlet.SERVLET_PATH, + WebSocketReaderServlet.SERVLET_PATH_V2); + + addWebSocketServlet(new WebSocketMultiTopicConsumerServlet(webSocketService), attributeMap, + WebSocketMultiTopicConsumerServlet.SERVLET_PATH); + } + } + + private void addWebSocketServlet(JettyWebSocketServlet webSocketServlet, Map attributeMap, + String... servletPaths) { + for (String servletPath : servletPaths) { + webService.addWebSocketServlet(servletPath, webSocketServlet, attributeMap); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java index faabad8f8fe58..7593c6697259c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java @@ -35,7 +35,7 @@ import javax.servlet.http.HttpServletResponse; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; -import org.eclipse.jetty.server.HttpOutput; +import org.eclipse.jetty.ee8.nested.HttpOutput; @Slf4j public class PulsarPrometheusMetricsServlet extends PrometheusMetricsServlet { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/ExceptionHandler.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/ExceptionHandler.java index 205e02ed75a2e..adf6123d72cf3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/ExceptionHandler.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/ExceptionHandler.java @@ -19,8 +19,6 @@ package org.apache.pulsar.broker.web; import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import javax.servlet.ServletResponse; import javax.servlet.http.HttpServletResponse; import javax.ws.rs.core.Response; @@ -28,11 +26,6 @@ import org.apache.pulsar.common.intercept.InterceptException; import org.apache.pulsar.common.policies.data.ErrorData; import org.apache.pulsar.common.util.ObjectMapperFactory; -import org.eclipse.jetty.http.HttpField; -import org.eclipse.jetty.http.HttpFields; -import org.eclipse.jetty.http.HttpHeader; -import org.eclipse.jetty.http.HttpVersion; -import org.eclipse.jetty.http.MetaData; /** * Exception handler for handle exception. @@ -41,29 +34,17 @@ public class ExceptionHandler { public void handle(ServletResponse response, Exception ex) throws IOException { + HttpServletResponse httpServletResponse = (HttpServletResponse) response; if (ex instanceof InterceptException) { - if (response instanceof org.eclipse.jetty.server.Response) { - String errorData = ObjectMapperFactory - .getMapper().writer().writeValueAsString(new ErrorData(ex.getMessage())); - byte[] errorBytes = errorData.getBytes(StandardCharsets.UTF_8); - int errorCode = ((InterceptException) ex).getErrorCode(); - HttpFields httpFields = new HttpFields(); - HttpField httpField = new HttpField(HttpHeader.CONTENT_TYPE, "application/json;charset=utf-8"); - httpFields.add(httpField); - MetaData.Response info = new MetaData.Response(HttpVersion.HTTP_1_1, errorCode, httpFields); - info.setHttpVersion(HttpVersion.HTTP_1_1); - info.setReason(errorData); - info.setStatus(errorCode); - info.setContentLength(errorBytes.length); - ((org.eclipse.jetty.server.Response) response).getHttpChannel().sendResponse(info, - ByteBuffer.wrap(errorBytes), - true); - } else { - ((HttpServletResponse) response).sendError(((InterceptException) ex).getErrorCode(), - ex.getMessage()); - } + byte[] errorBytes = ObjectMapperFactory + .getMapper().writer().writeValueAsBytes(new ErrorData(ex.getMessage())); + int errorCode = ((InterceptException) ex).getErrorCode(); + httpServletResponse.setStatus(errorCode); + httpServletResponse.setContentType("application/json;charset=utf-8"); + httpServletResponse.setContentLength(errorBytes.length); + httpServletResponse.getOutputStream().write(errorBytes); } else { - ((HttpServletResponse) response).sendError(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), + httpServletResponse.sendError(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), ex.getMessage()); } } 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 96e8a516a6cad..d3037cdd44bc7 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 @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.web; import io.prometheus.client.CollectorRegistry; -import io.prometheus.client.jetty.JettyStatisticsCollector; import java.io.IOException; import java.util.ArrayList; import java.util.EnumSet; @@ -45,31 +44,33 @@ import org.apache.pulsar.broker.intercept.BrokerInterceptors; import org.apache.pulsar.common.util.PulsarSslConfiguration; import org.apache.pulsar.common.util.PulsarSslFactory; +import org.apache.pulsar.jetty.metrics.JettyStatisticsCollector; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.ee8.nested.ContextHandler; +import org.eclipse.jetty.ee8.nested.ResourceHandler; +import org.eclipse.jetty.ee8.servlet.FilterHolder; +import org.eclipse.jetty.ee8.servlet.ServletContextHandler; +import org.eclipse.jetty.ee8.servlet.ServletHolder; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServlet; +import org.eclipse.jetty.ee8.websocket.server.config.JettyWebSocketServletContainerInitializer; +import org.eclipse.jetty.http.UriCompliance; 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.NetworkConnectionLimit; 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; -import org.eclipse.jetty.server.handler.HandlerCollection; -import org.eclipse.jetty.server.handler.RequestLogHandler; -import org.eclipse.jetty.server.handler.ResourceHandler; +import org.eclipse.jetty.server.handler.QoSHandler; import org.eclipse.jetty.server.handler.StatisticsHandler; -import org.eclipse.jetty.servlet.FilterHolder; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.QoSFilter; -import org.eclipse.jetty.util.resource.Resource; +import org.eclipse.jetty.util.resource.ResourceFactory; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.glassfish.jersey.media.multipart.MultiPartFeature; import org.glassfish.jersey.server.ResourceConfig; @@ -124,12 +125,14 @@ public WebService(PulsarService pulsar) throws PulsarServerException { new WebExecutorThreadPoolStats(pulsar.getOpenTelemetry().getMeter(), webServiceExecutor); this.server = new Server(webServiceExecutor); if (config.getMaxHttpServerConnections() > 0) { - server.addBean(new ConnectionLimit(config.getMaxHttpServerConnections(), server)); + server.addBean(new NetworkConnectionLimit(config.getMaxHttpServerConnections(), server)); } + server.setStopTimeout(config.getBrokerShutdownTimeoutMs()); List connectors = new ArrayList<>(); Optional port = config.getWebServicePort(); HttpConfiguration httpConfig = new HttpConfiguration(); + httpConfig.setUriCompliance(UriCompliance.LEGACY); if (config.isWebServiceTrustXForwardedFor()) { httpConfig.addCustomizer(new ForwardedRequestCustomizer()); } @@ -164,7 +167,7 @@ public WebService(PulsarService pulsar) throws PulsarServerException { config.getTlsCertRefreshCheckDurationSec(), TimeUnit.SECONDS); } - SslContextFactory sslCtxFactory = + SslContextFactory.Server sslCtxFactory = JettySslContextFactory.createSslContextFactory(config.getWebServiceTlsProvider(), this.sslFactory, config.isTlsRequireTrustedClientCertOnConnect(), config.getTlsCiphers(), config.getTlsProtocols()); @@ -177,7 +180,9 @@ public WebService(PulsarService pulsar) throws PulsarServerException { // 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()); + // disable SNI host check for backwards compatibility with Jetty 9.x + boolean sniHostCheck = false; + httpConfig.addCustomizer(new SecureRequestCustomizer(sniHostCheck)); } httpsConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpsConnector.setPort(tlsPort.get()); @@ -238,12 +243,6 @@ private static class FilterInitializer { FilterInitializer(PulsarService pulsarService) { ServiceConfiguration config = pulsarService.getConfiguration(); - if (config.getMaxConcurrentHttpRequests() > 0) { - FilterHolder filterHolder = new FilterHolder(QoSFilter.class); - filterHolder.setInitParameter("maxRequests", String.valueOf(config.getMaxConcurrentHttpRequests())); - filterHolders.add(filterHolder); - } - if (config.isHttpRequestsLimitEnabled()) { filterHolders.add(new FilterHolder( new RateLimitingFilter(config.getHttpRequestsMaxPerSecond(), @@ -354,39 +353,47 @@ public void addServlet(String path, ServletHolder servletHolder, boolean require } filterInitializer.addFilters(servletContextHandler, requiresAuthentication); - handlers.add(servletContextHandler); + handlers.add(servletContextHandler.get()); + } + + public void addWebSocketServlet(String path, JettyWebSocketServlet webSocketServlet, + Map attributeMap) { + ServletContextHandler servletContextHandler = new ServletContextHandler(ServletContextHandler.SESSIONS); + servletContextHandler.setContextPath(path); + JettyWebSocketServletContainerInitializer.configure(servletContextHandler, null); + ServletHolder servletHolder = new ServletHolder(webSocketServlet); + servletContextHandler.addServlet(servletHolder, MATCH_ALL); + handlers.add(servletContextHandler.get()); } public void addStaticResources(String basePath, String resourcePath) { ContextHandler capHandler = new ContextHandler(); capHandler.setContextPath(basePath); ResourceHandler resHandler = new ResourceHandler(); - resHandler.setBaseResource(Resource.newClassPathResource(resourcePath)); + ResourceFactory resourceFactory = ResourceFactory.root(); + resHandler.setBaseResource(resourceFactory.newClassLoaderResource(resourcePath, true)); resHandler.setEtags(true); resHandler.setCacheControl(WebService.HANDLER_CACHE_CONTROL); capHandler.setHandler(resHandler); - handlers.add(capHandler); + handlers.add(capHandler.get()); } public void start() throws PulsarServerException { try { - RequestLogHandler requestLogHandler = new RequestLogHandler(); 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); + server.setRequestLog(requestLogger); ContextHandlerCollection contexts = new ContextHandlerCollection(); - contexts.setHandlers(handlers.toArray(new Handler[handlers.size()])); + contexts.setHandlers(handlers); Handler handlerForContexts = GzipHandlerUtil.wrapWithGzipHandler(contexts, pulsar.getConfig().getHttpServerGzipCompressionExcludedPaths()); - HandlerCollection handlerCollection = new HandlerCollection(); - handlerCollection.setHandlers(new Handler[] {handlerForContexts, new DefaultHandler(), requestLogHandler}); + Handler.Collection handlerCollection = new Handler.Sequence(); + handlerCollection.setHandlers(handlerForContexts, new DefaultHandler()); // Metrics handler StatisticsHandler stats = new StatisticsHandler(); @@ -398,7 +405,14 @@ public void start() throws PulsarServerException { // Already registered. Eg: in unit tests } - server.setHandler(stats); + Handler serverHandler = stats; + if (pulsar.getConfiguration().getMaxConcurrentHttpRequests() > 0) { + QoSHandler qoSHandler = new QoSHandler(serverHandler); + qoSHandler.setMaxRequestCount(pulsar.getConfiguration().getMaxConcurrentHttpRequests()); + serverHandler = qoSHandler; + } + server.setHandler(serverHandler); + server.start(); if (httpConnector != null) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java index 68826372b7bd6..50663af4f4fcd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java @@ -32,7 +32,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; -import org.eclipse.jetty.server.HttpOutput; +import org.eclipse.jetty.ee8.nested.HttpOutput; public class PrometheusMetricsTestUtil { public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerAdditionalServletTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerAdditionalServletTest.java index dc21a6a69335b..3457be30107b1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerAdditionalServletTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerAdditionalServletTest.java @@ -37,8 +37,7 @@ import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServletWithPulsarService; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlets; import org.apache.pulsar.common.configuration.PulsarConfiguration; -import org.eclipse.jetty.server.Request; -import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.ee8.nested.Request; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -75,7 +74,9 @@ private void mockAdditionalServlet(PulsarService pulsar) { AdditionalServlet brokerAdditionalServlet = Mockito.mock(AdditionalServlet.class); Mockito.when(brokerAdditionalServlet.getBasePath()).thenReturn(BASE_PATH); - Mockito.when(brokerAdditionalServlet.getServletHolder()).thenReturn(new ServletHolder(servlet)); + Mockito.when(brokerAdditionalServlet.getServletInstance()).thenReturn(servlet); + Mockito.when(brokerAdditionalServlet.getServletType()) + .thenReturn(AdditionalServlet.AdditionalServletType.JAVAX_SERVLET); AdditionalServletWithPulsarService brokerAdditionalServletWithPulsarService = new AdditionalServletWithPulsarService() { @@ -96,8 +97,8 @@ public String getBasePath() { } @Override - public ServletHolder getServletHolder() { - return new ServletHolder(new WithPulsarServiceServlet(pulsarService)); + public Object getServletInstance() { + return new WithPulsarServiceServlet(pulsarService); } @Override @@ -146,7 +147,7 @@ public ServletConfig getServletConfig() { @Override public void service(ServletRequest servletRequest, ServletResponse servletResponse) throws ServletException, IOException { - log.info("[service] path: {}", ((Request) servletRequest).getOriginalURI()); + log.info("[service] path: {}", ((Request) servletRequest).getHttpURI()); String value = servletRequest.getParameterMap().get(QUERY_PARAM)[0]; ServletOutputStream servletOutputStream = servletResponse.getOutputStream(); servletResponse.setContentLength(value.getBytes().length); @@ -177,7 +178,7 @@ public WithPulsarServiceServlet(PulsarService pulsar) { @Override public void service(ServletRequest servletRequest, ServletResponse servletResponse) throws ServletException, IOException { - log.info("[service] path: {}", ((Request) servletRequest).getOriginalURI()); + log.info("[service] path: {}", ((Request) servletRequest).getHttpURI()); String value = pulsarService == null ? "null" : PulsarService.class.getName(); ServletOutputStream servletOutputStream = servletResponse.getOutputStream(); servletResponse.setContentLength(value.getBytes().length); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java index 1ec8f160dfb6c..3a99dbc251f10 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java @@ -29,6 +29,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import javax.servlet.ServletContext; import javax.ws.rs.container.AsyncResponse; import javax.ws.rs.container.TimeoutHandler; import org.apache.pulsar.broker.admin.v2.PersistentTopics; @@ -69,7 +70,7 @@ protected void setup() throws Exception { admin.namespaces().createNamespace("prop/ns-abc"); admin.namespaces().setNamespaceReplicationClusters("prop/ns-abc", Set.of("test")); persistentTopics = spy(PersistentTopics.class); - persistentTopics.setServletContext(new MockServletContext()); + persistentTopics.setServletContext(mock(ServletContext.class)); persistentTopics.setPulsar(pulsar); doReturn(false).when(persistentTopics).isRequestHttps(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java index 91fe43f8b396a..b951549247a6f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java @@ -46,6 +46,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; +import javax.servlet.ServletContext; import javax.ws.rs.container.AsyncResponse; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; @@ -137,7 +138,7 @@ public void setup() throws Exception { doNothing().when(properties).validateSuperUserAccess(); namespaces = spy(Namespaces.class); - namespaces.setServletContext(new MockServletContext()); + namespaces.setServletContext(mock(ServletContext.class)); namespaces.setPulsar(pulsar); doReturn("test").when(namespaces).clientAppId(); doReturn(Set.of("use", "usw", "usc", "global")).when(namespaces).clusters(); @@ -154,7 +155,7 @@ public void setup() throws Exception { uriField.setAccessible(true); persistentTopics = spy(PersistentTopics.class); - persistentTopics.setServletContext(new MockServletContext()); + persistentTopics.setServletContext(mock(ServletContext.class)); persistentTopics.setPulsar(pulsar); doReturn("test").when(persistentTopics).clientAppId(); doReturn("persistent").when(persistentTopics).domain(); @@ -164,11 +165,11 @@ public void setup() throws Exception { doNothing().when(persistentTopics).validateAdminAccessForTenant("prop-xyz"); resourceQuotas = spy(ResourceQuotas.class); - resourceQuotas.setServletContext(new MockServletContext()); + resourceQuotas.setServletContext(mock(ServletContext.class)); resourceQuotas.setPulsar(pulsar); brokerStats = spy(BrokerStats.class); - brokerStats.setServletContext(new MockServletContext()); + brokerStats.setServletContext(mock(ServletContext.class)); brokerStats.setPulsar(pulsar); doReturn(false).when(persistentTopics).isRequestHttps(); @@ -177,7 +178,7 @@ public void setup() throws Exception { doReturn(mock(AuthenticationDataHttps.class)).when(persistentTopics).clientAuthData(); schemasResource = spy(SchemasResource.class); - schemasResource.setServletContext(new MockServletContext()); + schemasResource.setServletContext(mock(ServletContext.class)); schemasResource.setPulsar(pulsar); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/MockServletContext.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/MockServletContext.java deleted file mode 100644 index 7ae63c500df6c..0000000000000 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/MockServletContext.java +++ /dev/null @@ -1,279 +0,0 @@ -/* - * 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 java.io.InputStream; -import java.net.MalformedURLException; -import java.net.URL; -import java.util.Enumeration; -import java.util.EventListener; -import java.util.Map; -import java.util.Set; -import javax.servlet.Filter; -import javax.servlet.FilterRegistration; -import javax.servlet.RequestDispatcher; -import javax.servlet.Servlet; -import javax.servlet.ServletContext; -import javax.servlet.ServletException; -import javax.servlet.ServletRegistration; -import javax.servlet.ServletRegistration.Dynamic; -import javax.servlet.SessionCookieConfig; -import javax.servlet.SessionTrackingMode; -import javax.servlet.descriptor.JspConfigDescriptor; -import org.eclipse.jetty.util.AttributesMap; - -public class MockServletContext extends AttributesMap implements ServletContext { - - @Override - public String getContextPath() { - return null; - } - - @Override - public ServletContext getContext(String uripath) { - return null; - } - - @Override - public int getMajorVersion() { - return 0; - } - - @Override - public int getMinorVersion() { - return 0; - } - - @Override - public int getEffectiveMajorVersion() { - return 0; - } - - @Override - public int getEffectiveMinorVersion() { - return 0; - } - - @Override - public String getMimeType(String file) { - return null; - } - - @Override - public Set getResourcePaths(String path) { - return null; - } - - @Override - public URL getResource(String path) throws MalformedURLException { - return null; - } - - @Override - public InputStream getResourceAsStream(String path) { - return null; - } - - @Override - public RequestDispatcher getRequestDispatcher(String path) { - return null; - } - - @Override - public RequestDispatcher getNamedDispatcher(String name) { - return null; - } - - @Override - @Deprecated - public Servlet getServlet(String name) throws ServletException { - return null; - } - - @Override - @Deprecated - public Enumeration getServlets() { - return null; - } - - @Override - @Deprecated - public Enumeration getServletNames() { - return null; - } - - @Override - public void log(String msg) { - } - - @Override - @Deprecated - public void log(Exception exception, String msg) { - } - - @Override - public void log(String message, Throwable throwable) { - - } - - @Override - public String getRealPath(String path) { - return null; - } - - @Override - public String getServerInfo() { - return null; - } - - @Override - public String getInitParameter(String name) { - return null; - } - - @Override - public Enumeration getInitParameterNames() { - return null; - } - - @Override - public boolean setInitParameter(String name, String value) { - return false; - } - - @Override - public String getServletContextName() { - return null; - } - - @Override - public Dynamic addServlet(String servletName, String className) { - return null; - } - - @Override - public Dynamic addServlet(String servletName, Servlet servlet) { - return null; - } - - @Override - public Dynamic addServlet(String servletName, Class servletClass) { - return null; - } - - @Override - public T createServlet(Class clazz) throws ServletException { - return null; - } - - @Override - public ServletRegistration getServletRegistration(String servletName) { - return null; - } - - @Override - public Map getServletRegistrations() { - return null; - } - - @Override - public javax.servlet.FilterRegistration.Dynamic addFilter(String filterName, String className) { - return null; - } - - @Override - public javax.servlet.FilterRegistration.Dynamic addFilter(String filterName, Filter filter) { - return null; - } - - @Override - public javax.servlet.FilterRegistration.Dynamic addFilter(String filterName, Class filterClass) { - return null; - } - - @Override - public T createFilter(Class clazz) throws ServletException { - return null; - } - - @Override - public FilterRegistration getFilterRegistration(String filterName) { - return null; - } - - @Override - public Map getFilterRegistrations() { - return null; - } - - @Override - public SessionCookieConfig getSessionCookieConfig() { - return null; - } - - @Override - public void setSessionTrackingModes(Set sessionTrackingModes) { - } - - @Override - public Set getDefaultSessionTrackingModes() { - return null; - } - - @Override - public Set getEffectiveSessionTrackingModes() { - return null; - } - - @Override - public void addListener(String className) { - } - - @Override - public void addListener(T t) { - } - - @Override - public void addListener(Class listenerClass) { - } - - @Override - public T createListener(Class clazz) throws ServletException { - return null; - } - - @Override - public JspConfigDescriptor getJspConfigDescriptor() { - return null; - } - - @Override - public ClassLoader getClassLoader() { - return null; - } - - @Override - public void declareRoles(String... roleNames) { - } - - @Override - public String getVirtualServerName() { - return null; - } -} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java index 2f5b99bfd9de1..da3ced47efcd3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java @@ -57,6 +57,7 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; +import javax.servlet.ServletContext; import javax.ws.rs.BadRequestException; import javax.ws.rs.ClientErrorException; import javax.ws.rs.WebApplicationException; @@ -211,7 +212,7 @@ private void initAndStartBroker() throws Exception { super.internalSetup(); namespaces = spy(Namespaces.class); - namespaces.setServletContext(new MockServletContext()); + namespaces.setServletContext(mock(ServletContext.class)); namespaces.setPulsar(pulsar); doReturn(false).when(namespaces).isRequestHttps(); doReturn("test").when(namespaces).clientAppId(); @@ -1230,7 +1231,7 @@ public void testValidateTopicOwnership() throws Exception { ownership.set(pulsar.getNamespaceService(), mockOwnershipCache); TopicName topicName = TopicName.get(testNs.getPersistentTopicName("my-topic")); PersistentTopics topics = spy(PersistentTopics.class); - topics.setServletContext(new MockServletContext()); + topics.setServletContext(mock(ServletContext.class)); topics.setPulsar(pulsar); doReturn(false).when(topics).isRequestHttps(); doReturn("test").when(topics).clientAppId(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesV2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesV2Test.java index 596cfa3f396cf..fcd603434111c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesV2Test.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesV2Test.java @@ -32,6 +32,7 @@ import java.util.List; import java.util.Objects; import java.util.Set; +import javax.servlet.ServletContext; import javax.ws.rs.core.Response; import javax.ws.rs.core.UriInfo; import org.apache.pulsar.broker.admin.v2.Namespaces; @@ -95,7 +96,7 @@ public void setup() throws Exception { super.internalSetup(); namespaces = spy(Namespaces.class); - namespaces.setServletContext(new MockServletContext()); + namespaces.setServletContext(mock(ServletContext.class)); namespaces.setPulsar(pulsar); doReturn(false).when(namespaces).isRequestHttps(); doReturn("test").when(namespaces).clientAppId(); 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 752ce1d6d9cd3..281c2ce52c166 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 @@ -48,6 +48,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import javax.servlet.ServletContext; import javax.ws.rs.InternalServerErrorException; import javax.ws.rs.WebApplicationException; import javax.ws.rs.container.AsyncResponse; @@ -143,7 +144,7 @@ protected void setup() throws Exception { conf.setTopicLevelPoliciesEnabled(false); super.internalSetup(); persistentTopics = spy(PersistentTopics.class); - persistentTopics.setServletContext(new MockServletContext()); + persistentTopics.setServletContext(mock(ServletContext.class)); persistentTopics.setPulsar(pulsar); doReturn(false).when(persistentTopics).isRequestHttps(); doReturn(null).when(persistentTopics).originalPrincipal(); @@ -153,7 +154,7 @@ protected void setup() throws Exception { doReturn(mock(AuthenticationDataHttps.class)).when(persistentTopics).clientAuthData(); extPersistentTopics = spy(ExtPersistentTopics.class); - extPersistentTopics.setServletContext(new MockServletContext()); + extPersistentTopics.setServletContext(mock(ServletContext.class)); extPersistentTopics.setPulsar(pulsar); doReturn(false).when(extPersistentTopics).isRequestHttps(); doReturn(null).when(extPersistentTopics).originalPrincipal(); @@ -163,7 +164,7 @@ protected void setup() throws Exception { doReturn(mock(AuthenticationDataHttps.class)).when(extPersistentTopics).clientAuthData(); nonPersistentTopic = spy(NonPersistentTopics.class); - nonPersistentTopic.setServletContext(new MockServletContext()); + nonPersistentTopic.setServletContext(mock(ServletContext.class)); nonPersistentTopic.setPulsar(pulsar); namespaceResources = mock(NamespaceResources.class); doReturn(false).when(nonPersistentTopic).isRequestHttps(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ResourceGroupsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ResourceGroupsTest.java index b8b4f0ea1601f..c1fa24ae87556 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ResourceGroupsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ResourceGroupsTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.admin; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.fail; @@ -27,6 +28,7 @@ import java.util.Iterator; import java.util.List; import java.util.Set; +import javax.servlet.ServletContext; import org.apache.pulsar.broker.admin.v2.ResourceGroups; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.web.RestException; @@ -51,7 +53,7 @@ public class ResourceGroupsTest extends MockedPulsarServiceBaseTest { protected void setup() throws Exception { super.internalSetup(); resourcegroups = spy(ResourceGroups.class); - resourcegroups.setServletContext(new MockServletContext()); + resourcegroups.setServletContext(mock(ServletContext.class)); resourcegroups.setPulsar(pulsar); doReturn(false).when(resourcegroups).isRequestHttps(); doReturn("test").when(resourcegroups).clientAppId(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java index 6c690e267b03e..ab67388193e20 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java @@ -29,6 +29,9 @@ import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.fasterxml.jackson.databind.ObjectReader; +import java.io.IOException; +import java.io.InputStream; import java.lang.reflect.Field; import java.lang.reflect.Method; import java.util.ArrayList; @@ -45,6 +48,7 @@ import java.util.concurrent.TimeUnit; import javax.ws.rs.BadRequestException; import javax.ws.rs.client.Entity; +import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; @@ -89,6 +93,7 @@ import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; import org.apache.pulsar.common.policies.data.DispatchRate; +import org.apache.pulsar.common.policies.data.ErrorData; import org.apache.pulsar.common.policies.data.HierarchyTopicPolicies; import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode; import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; @@ -104,6 +109,7 @@ import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; @@ -4043,18 +4049,37 @@ public void testSetNonBacklogQuotType() throws Exception { .header("Content-Type", "application/json") .post(Entity.json(backlogQuotaWithNonPolicy)); assertEquals(response1.getStatus(), 400); - assertTrue(response1.getStatusInfo().getReasonPhrase().contains("policy cannot be null")); + assertTrue(resolveErrorReason(response1).contains("policy cannot be null")); // Topic level. Response response2 = httpClient.target(hostAndPort).path(topicPath).request() .header("Content-Type", "application/json") .post(Entity.json(backlogQuotaWithNonPolicy)); assertEquals(response2.getStatus(), 400); - assertTrue(response2.getStatusInfo().getReasonPhrase().contains("policy cannot be null")); + assertTrue(resolveErrorReason(response2).contains("policy cannot be null")); // cleanup. httpClient.close(); } + String resolveErrorReason(Response response) { + ErrorData errorData = null; + if (response.hasEntity() && response.getMediaType().isCompatible(MediaType.APPLICATION_JSON_TYPE)) { + Object responseBody = response.getEntity(); + ObjectReader reader = + ObjectMapperFactory.getMapper().getObjectMapper().reader(); + try { + errorData = reader.readValue((InputStream) responseBody, ErrorData.class); + } catch (IOException e) { + // ignore + if (log.isDebugEnabled()) { + log.debug("Failed to parse error response: {}", response); + } + } + } + return errorData != null ? errorData.reason : response.getStatusInfo().getReasonPhrase(); + } + + @Test public void testSetSubRateWithSub() throws Exception { String topic = "persistent://" + myNamespace + "/testSetSubRateWithSub"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/PulsarLookupProxy.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/PulsarLookupProxy.java index 9e92676830d6e..c493ee165a510 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/PulsarLookupProxy.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/cache/PulsarLookupProxy.java @@ -23,12 +23,12 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import lombok.SneakyThrows; -import org.eclipse.jetty.client.api.Response; -import org.eclipse.jetty.proxy.ProxyServlet; +import org.eclipse.jetty.client.Response; +import org.eclipse.jetty.ee8.proxy.ProxyServlet; +import org.eclipse.jetty.ee8.servlet.ServletContextHandler; +import org.eclipse.jetty.ee8.servlet.ServletHolder; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.util.Callback; /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorTest.java index 65fb7f398b30d..72f1c4f50094d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorTest.java @@ -49,6 +49,7 @@ import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; import org.testng.Assert; +import org.testng.ITestResult; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -109,17 +110,18 @@ private CounterBrokerInterceptor getCounterBrokerInterceptor() { @Override protected void cleanup() throws Exception { - teardown(); + teardown(null); } @AfterMethod(alwaysRun = true) - public void teardown() throws Exception { + public void teardown(ITestResult testResult) throws Exception { this.listeners.close(); - - verify(listener1, times(1)).close(); - verify(listener2, times(1)).close(); - verify(ncl1, times(1)).close(); - verify(ncl2, times(1)).close(); + if (testResult != null && testResult.getStatus() == ITestResult.SUCCESS) { + verify(listener1, times(1)).close(); + verify(listener2, times(1)).close(); + verify(ncl1, times(1)).close(); + verify(ncl2, times(1)).close(); + } super.internalCleanup(); } @@ -212,13 +214,13 @@ public void testBeforeSendMessage() throws PulsarClientException { @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING) - .topic("test-before-send-message") - .create(); + .topic("test-before-send-message") + .create(); Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic("test-before-send-message") - .subscriptionName("test") - .subscribe(); + .topic("test-before-send-message") + .subscriptionName("test") + .subscribe(); assertEquals(counterBrokerInterceptor.getMessageProducedCount(), 0); assertEquals(counterBrokerInterceptor.getMessageDispatchCount(), 0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/CounterBrokerInterceptor.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/CounterBrokerInterceptor.java index a7302a907a0f0..a13a3671b2d1d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/CounterBrokerInterceptor.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/CounterBrokerInterceptor.java @@ -252,7 +252,11 @@ public void onWebserviceResponse(ServletRequest request, ServletResponse respons } if (response instanceof Response) { Response res = (Response) response; - responseList.add(new ResponseEvent(res.getHttpChannel().getRequest().getRequestURI(), res.getStatus())); + responseList.add(new ResponseEvent(res.getRequest().getHttpURI().getPath(), res.getStatus())); + } else if (response instanceof org.eclipse.jetty.ee8.nested.Response) { + org.eclipse.jetty.ee8.nested.Response res = (org.eclipse.jetty.ee8.nested.Response) response; + responseList.add( + new ResponseEvent(res.getHttpChannel().getRequest().getHttpURI().getPath(), res.getStatus())); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 56a08eac2096f..0a8f6601ad5f1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -853,7 +853,7 @@ public void testTlsAuthAllowInsecure() throws Exception { fail("should fail"); } catch (Exception e) { - assertTrue(e.getMessage().contains("Unauthorized")); + assertTrue(e.getMessage().contains("Authentication required")); } finally { pulsarClient.close(); } @@ -916,7 +916,7 @@ public void testTlsAuthDisallowInsecure() throws Exception { fail("should fail"); } catch (Exception e) { - assertTrue(e.getMessage().contains("Unauthorized")); + assertTrue(e.getMessage().contains("Authentication required")); } finally { pulsarClient.close(); } @@ -934,7 +934,7 @@ public void testTlsAuthDisallowInsecure() throws Exception { .subscribe(); fail("should fail"); } catch (Exception e) { - assertTrue(e.getMessage().contains("Unauthorized")); + assertTrue(e.getMessage().contains("Authentication required")); } finally { pulsarClient.close(); } @@ -978,7 +978,7 @@ public void testTlsAuthUseTrustCert() throws Exception { .subscribe(); fail("should fail"); } catch (Exception e) { - assertTrue(e.getMessage().contains("Unauthorized")); + assertTrue(e.getMessage().contains("Authentication required")); } finally { pulsarClient.close(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/ExceptionHandlerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/ExceptionHandlerTest.java index 3c0c2f4e68774..660f4bce28c3e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/ExceptionHandlerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/ExceptionHandlerTest.java @@ -23,8 +23,6 @@ import javax.servlet.http.HttpServletResponse; import lombok.SneakyThrows; import org.apache.pulsar.common.intercept.InterceptException; -import org.eclipse.jetty.server.HttpChannel; -import org.eclipse.jetty.server.Response; import org.mockito.Mockito; import org.testng.annotations.Test; @@ -41,21 +39,18 @@ public void testHandle() { String internal = "internal exception"; String illegal = "illegal argument exception "; ExceptionHandler handler = new ExceptionHandler(); - HttpServletResponse response = Mockito.mock(HttpServletResponse.class); + + HttpServletResponse response = Mockito.mock(HttpServletResponse.class, Mockito.RETURNS_DEEP_STUBS); handler.handle(response, new InterceptException(PRECONDITION_FAILED_412, restriction)); - Mockito.verify(response).sendError(PRECONDITION_FAILED_412, restriction); + Mockito.verify(response).setStatus(PRECONDITION_FAILED_412); + Mockito.verify(response.getOutputStream()).write(("{\"reason\":\"" + restriction + "\"}").getBytes()); handler.handle(response, new InterceptException(INTERNAL_SERVER_ERROR_500, internal)); - Mockito.verify(response).sendError(INTERNAL_SERVER_ERROR_500, internal); + Mockito.verify(response).setStatus(INTERNAL_SERVER_ERROR_500); + Mockito.verify(response.getOutputStream()).write(("{\"reason\":\"" + internal + "\"}").getBytes()); handler.handle(response, new IllegalArgumentException(illegal)); Mockito.verify(response).sendError(INTERNAL_SERVER_ERROR_500, illegal); - - Response response2 = Mockito.mock(Response.class); - HttpChannel httpChannel = Mockito.mock(HttpChannel.class); - Mockito.when(response2.getHttpChannel()).thenReturn(httpChannel); - handler.handle(response2, new InterceptException(PRECONDITION_FAILED_412, restriction)); - Mockito.verify(httpChannel).sendResponse(Mockito.any(), Mockito.any(), Mockito.anyBoolean()); } } 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 index 42186e988a256..3fc4cdb9f18c3 100644 --- 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 @@ -27,9 +27,10 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.assertj.core.api.ThrowingConsumer; import org.awaitility.Awaitility; +import org.eclipse.jetty.client.ContentResponse; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.client.ProxyProtocolClientConnectionFactory.V2; -import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.http.HttpField; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -45,7 +46,8 @@ public class WebServiceOriginalClientIPTest extends MockedPulsarServiceBaseTest @Override protected void setup() throws Exception { super.internalSetup(); - httpClient = new HttpClient(new SslContextFactory(true)); + httpClient = new HttpClient(); + httpClient.setSslContextFactory(new SslContextFactory.Client(true)); httpClient.start(); } @@ -81,7 +83,7 @@ public void testClientIPIsPickedFromXForwardedForHeaderAndLogged(boolean tlsEnab performLoggingTest(consoleCaptor -> { // Send a GET request to the metrics URL ContentResponse response = httpClient.newRequest(metricsUrl) - .header("X-Forwarded-For", "11.22.33.44:12345") + .headers(hdrs -> hdrs.ensureField(new HttpField("X-Forwarded-For", "11.22.33.44:12345"))) .send(); // Validate the response @@ -100,7 +102,7 @@ public void testClientIPIsPickedFromForwardedHeaderAndLogged(boolean tlsEnabled) performLoggingTest(consoleCaptor -> { // Send a GET request to the metrics URL ContentResponse response = httpClient.newRequest(metricsUrl) - .header("Forwarded", "for=11.22.33.44:12345") + .headers(hdrs -> hdrs.ensureField(new HttpField("Forwarded", "for=11.22.33.44:12345"))) .send(); // Validate the response diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoaderTest.java index 48b1135f10c69..6f7879bda4f45 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoaderTest.java @@ -25,10 +25,10 @@ import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; +import javax.servlet.Servlet; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.common.configuration.PulsarConfiguration; import org.apache.pulsar.common.nar.NarClassLoader; -import org.eclipse.jetty.servlet.ServletHolder; import org.testng.annotations.Test; @@ -55,10 +55,11 @@ public void testWrapper() { // test getServlet assertEquals(wrapper.getServlet(), servlet); // test getServletHolder - ServletHolder servletHolder = new ServletHolder(); - when(servlet.getServletHolder()).thenReturn(servletHolder); - assertEquals(wrapper.getServletHolder(), servletHolder); - verify(servlet, times(1)).getServletHolder(); + Servlet servletInstance = mock(Servlet.class); + when(servlet.getServletInstance()).thenReturn(servletInstance); + when(servlet.getServletType()).thenReturn(AdditionalServlet.AdditionalServletType.JAVAX_SERVLET); + assertEquals(wrapper.getServletInstance(), servletInstance); + verify(servlet, times(1)).getServletInstance(); } @Test @@ -77,7 +78,7 @@ public String getBasePath() { } @Override - public ServletHolder getServletHolder() { + public Object getServletInstance() { assertEquals(Thread.currentThread().getContextClassLoader(), narLoader); return null; } @@ -101,7 +102,7 @@ public void close() { additionalServletWithClassLoader.loadConfig(conf); assertEquals(Thread.currentThread().getContextClassLoader(), curClassLoader); // test getServletHolder - assertNull(additionalServletWithClassLoader.getServletHolder()); + assertNull(additionalServletWithClassLoader.getServletInstance()); assertEquals(Thread.currentThread().getContextClassLoader(), curClassLoader); // test getServlet assertEquals(additionalServletWithClassLoader.getServlet(), servlet); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/plugin/servlet/MockAdditionalServletWithClassLoader.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/plugin/servlet/MockAdditionalServletWithClassLoader.java index f65a98fd3f18f..cf186f9527be3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/plugin/servlet/MockAdditionalServletWithClassLoader.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/plugin/servlet/MockAdditionalServletWithClassLoader.java @@ -20,7 +20,6 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.common.configuration.PulsarConfiguration; -import org.eclipse.jetty.servlet.ServletHolder; public class MockAdditionalServletWithClassLoader implements AdditionalServletWithPulsarService{ @Override @@ -34,7 +33,7 @@ public String getBasePath() { } @Override - public ServletHolder getServletHolder() { + public Object getServletInstance() { return null; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MockBrokerService.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MockBrokerService.java index b7e0ee429034a..6e2db9ade0104 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MockBrokerService.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MockBrokerService.java @@ -23,6 +23,7 @@ import io.netty.bootstrap.ServerBootstrap; import io.netty.buffer.ByteBuf; import io.netty.channel.Channel; +import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInitializer; import io.netty.channel.EventLoopGroup; @@ -69,9 +70,10 @@ import org.apache.pulsar.common.protocol.PulsarDecoder; import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.util.netty.EventLoopUtil; -import org.eclipse.jetty.server.Request; +import org.eclipse.jetty.ee8.nested.AbstractHandler; +import org.eclipse.jetty.ee8.nested.ContextHandler; +import org.eclipse.jetty.ee8.nested.Request; import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.server.handler.AbstractHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,7 +95,7 @@ private class GenericResponseHandler extends AbstractHandler { private final Pattern multiPartPattern = Pattern.compile(".*/multi-part-.*"); @Override - public void handle(String s, Request baseRequest, HttpServletRequest request, HttpServletResponse response) + public void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException { String responseString; log.info("Received HTTP request {}", baseRequest.getRequestURI()); @@ -294,7 +296,7 @@ protected final void handlePong(CommandPong pong) { public MockBrokerService() { server = new Server(0); - server.setHandler(new GenericResponseHandler()); + server.setHandler(new ContextHandler("/", new GenericResponseHandler())); } public void start() { @@ -337,7 +339,7 @@ public void startMockBrokerService() throws Exception { @Override public void initChannel(SocketChannel ch) throws Exception { FrameDecoderUtil.addFrameDecoder(ch.pipeline(), maxMessageSize); - ch.pipeline().addLast("handler", new MockServerCnx()); + ch.pipeline().addLast("handler", (ChannelHandler) new MockServerCnx()); } }); // Bind and start to accept incoming connections. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ClientSideEncryptionWssConsumer.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ClientSideEncryptionWssConsumer.java index ab8372cbfb058..b16ec3a5f1590 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ClientSideEncryptionWssConsumer.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ClientSideEncryptionWssConsumer.java @@ -34,14 +34,17 @@ import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.websocket.data.ConsumerMessage; import org.eclipse.jetty.websocket.api.Session; -import org.eclipse.jetty.websocket.api.WebSocketAdapter; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketClose; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketError; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketMessage; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketOpen; import org.eclipse.jetty.websocket.api.annotations.WebSocket; import org.eclipse.jetty.websocket.client.ClientUpgradeRequest; import org.eclipse.jetty.websocket.client.WebSocketClient; @Slf4j -@WebSocket(maxTextMessageSize = 64 * 1024) -public class ClientSideEncryptionWssConsumer extends WebSocketAdapter implements Closeable { +@WebSocket +public class ClientSideEncryptionWssConsumer implements Closeable { private Session session; private final CryptoKeyReader cryptoKeyReader; @@ -69,8 +72,8 @@ public ClientSideEncryptionWssConsumer(String webSocketProxyHost, int webSocketP public void start() throws Exception { wssClient = new WebSocketClient(); wssClient.start(); - session = wssClient.connect(this, buildConnectURL(), new ClientUpgradeRequest()).get(); - assertTrue(session.isOpen()); + Session ses = wssClient.connect(this, new ClientUpgradeRequest(buildConnectURL())).get(); + assertTrue(ses.isOpen()); } private URI buildConnectURL() throws PulsarClientException.CryptoException { @@ -93,24 +96,24 @@ public synchronized ConsumerMessage receive(int timeout, TimeUnit unit) throws E return msg; } - @Override + @OnWebSocketClose public void onWebSocketClose(int statusCode, String reason) { log.info("Connection closed: {} - {}", statusCode, reason); this.session = null; } - @Override + @OnWebSocketOpen public void onWebSocketConnect(Session session) { log.info("Got connect: {}", session); this.session = session; } - @Override + @OnWebSocketError public void onWebSocketError(Throwable cause) { log.error("Received an error", cause); } - @Override + @OnWebSocketMessage public void onWebSocketText(String text) { try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ClientSideEncryptionWssProducer.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ClientSideEncryptionWssProducer.java index 9188bac7b3755..b2a4e8a97ec3d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ClientSideEncryptionWssProducer.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ClientSideEncryptionWssProducer.java @@ -42,15 +42,19 @@ import org.apache.pulsar.common.api.proto.MessageIdData; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.websocket.data.ProducerMessage; +import org.eclipse.jetty.websocket.api.Callback; import org.eclipse.jetty.websocket.api.Session; -import org.eclipse.jetty.websocket.api.WebSocketAdapter; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketClose; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketError; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketMessage; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketOpen; import org.eclipse.jetty.websocket.api.annotations.WebSocket; import org.eclipse.jetty.websocket.client.ClientUpgradeRequest; import org.eclipse.jetty.websocket.client.WebSocketClient; @Slf4j -@WebSocket(maxTextMessageSize = 64 * 1024) -public class ClientSideEncryptionWssProducer extends WebSocketAdapter implements Closeable { +@WebSocket +public class ClientSideEncryptionWssProducer implements Closeable { private Session session; private volatile CompletableFuture sendFuture; @@ -80,8 +84,8 @@ public ClientSideEncryptionWssProducer(String webSocketProxyHost, int webSocketP public void start() throws Exception { wssClient = new WebSocketClient(); wssClient.start(); - session = wssClient.connect(this, buildConnectURL(), new ClientUpgradeRequest()).get(); - assertTrue(session.isOpen()); + Session ses = wssClient.connect(this, new ClientUpgradeRequest(buildConnectURL())).get(); + assertTrue(ses.isOpen()); } private URI buildConnectURL() throws PulsarClientException.CryptoException { @@ -130,7 +134,7 @@ public synchronized MessageIdData sendMessage(ProducerMessage msg) throws Except // Do send. sendFuture = new CompletableFuture<>(); String jsonMsg = ObjectMapperFactory.getMapper().writer().writeValueAsString(msg); - this.session.getRemote().sendString(jsonMsg); + this.session.sendText(jsonMsg, Callback.NOOP); // Wait for response. executor.schedule(() -> { synchronized (ClientSideEncryptionWssProducer.this) { @@ -142,7 +146,7 @@ public synchronized MessageIdData sendMessage(ProducerMessage msg) throws Except return sendFuture.get(); } - @Override + @OnWebSocketClose public void onWebSocketClose(int statusCode, String reason) { log.info("Connection closed: {} - {}", statusCode, reason); this.session = null; @@ -151,18 +155,18 @@ public void onWebSocketClose(int statusCode, String reason) { } } - @Override + @OnWebSocketOpen public void onWebSocketConnect(Session session) { log.info("Got connect: {}", session); this.session = session; } - @Override + @OnWebSocketError public void onWebSocketError(Throwable cause) { log.error("Received an error", cause); } - @Override + @OnWebSocketMessage public void onWebSocketText(String text) { try { ResponseOfSend responseOfSend = diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthenticationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthenticationTest.java index 77632fbd418c5..b7f45c8c196e3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthenticationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthenticationTest.java @@ -127,13 +127,13 @@ private void checkSocket() throws Exception { SimpleProducerSocket produceSocket = new SimpleProducerSocket(); consumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); - Future consumerFuture = consumeClient.connect(consumeSocket, consumeUri, consumeRequest); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); + Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); log.info("Connecting to : {}", consumeUri); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); produceClient.start(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); Assert.assertTrue(consumerFuture.get().isOpen()); Assert.assertTrue(producerFuture.get().isOpen()); @@ -188,13 +188,13 @@ public void statsTest() throws Exception { try { consumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); - Future consumerFuture = consumeClient.connect(consumeSocket, consumeUri, consumeRequest); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); + Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); Assert.assertTrue(consumerFuture.get().isOpen()); produceClient.start(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); Assert.assertTrue(producerFuture.get().isOpen()); Awaitility.await().untilAsserted(() -> Assert.assertTrue( diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyEncryptionPublishConsumeTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyEncryptionPublishConsumeTest.java index d694e9596af67..e4aa3dec7eab6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyEncryptionPublishConsumeTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyEncryptionPublishConsumeTest.java @@ -118,15 +118,15 @@ public void socketTest() throws Exception { try { consumeClient1.start(); consumeClient2.start(); - ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(); - ClientUpgradeRequest consumeRequest2 = new ClientUpgradeRequest(); - Future consumerFuture1 = consumeClient1.connect(consumeSocket1, consumeUri, consumeRequest1); - Future consumerFuture2 = consumeClient2.connect(consumeSocket2, consumeUri, consumeRequest2); + ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(consumeUri); + ClientUpgradeRequest consumeRequest2 = new ClientUpgradeRequest(consumeUri); + Future consumerFuture1 = consumeClient1.connect(consumeSocket1, consumeRequest1); + Future consumerFuture2 = consumeClient2.connect(consumeSocket2, consumeRequest2); log.info("Connecting to : {}", consumeUri); readClient.start(); - ClientUpgradeRequest readRequest = new ClientUpgradeRequest(); - Future readerFuture = readClient.connect(readSocket, readUri, readRequest); + ClientUpgradeRequest readRequest = new ClientUpgradeRequest(readUri); + Future readerFuture = readClient.connect(readSocket, readRequest); log.info("Connecting to : {}", readUri); // let it connect @@ -137,9 +137,9 @@ public void socketTest() throws Exception { // Also make sure subscriptions and reader are already created Thread.sleep(500); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); produceClient.start(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); assertTrue(producerFuture.get().isOpen()); Awaitility.await().untilAsserted(() -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyIdleTimeoutTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyIdleTimeoutTest.java index ed05a99a22cff..b6926ae7bbb99 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyIdleTimeoutTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyIdleTimeoutTest.java @@ -95,8 +95,8 @@ public void testIdleTimeout() throws Exception { try { produceClient.start(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); assertThat(producerFuture).succeedsWithin(2, SECONDS); Session session = producerFuture.get(); Awaitility.await().during(5, SECONDS).untilAsserted(() -> assertThat(session.isOpen()).isFalse()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPingTest.java index 813d978f52a7f..632128b37a8c0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPingTest.java @@ -97,8 +97,8 @@ public void testPing() throws Exception { try { produceClient.start(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); assertThat(producerFuture).succeedsWithin(2, SECONDS); Session session = producerFuture.get(); Awaitility.await().during(5, SECONDS).untilAsserted(() -> assertThat(session.isOpen()).isTrue()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTest.java index 30db9ee43be9c..f1255a8c97a7a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTest.java @@ -35,6 +35,7 @@ import com.google.gson.reflect.TypeToken; import java.net.URI; import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -68,7 +69,7 @@ import org.apache.pulsar.websocket.stats.ProxyTopicStat.ProducerStats; import org.awaitility.Awaitility; import org.eclipse.jetty.websocket.api.Session; -import org.eclipse.jetty.websocket.api.UpgradeException; +import org.eclipse.jetty.websocket.api.exceptions.UpgradeException; import org.eclipse.jetty.websocket.client.ClientUpgradeRequest; import org.eclipse.jetty.websocket.client.WebSocketClient; import org.glassfish.jersey.client.ClientConfig; @@ -144,15 +145,15 @@ public void socketTest() throws Exception { try { consumeClient1.start(); consumeClient2.start(); - ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(); - ClientUpgradeRequest consumeRequest2 = new ClientUpgradeRequest(); - Future consumerFuture1 = consumeClient1.connect(consumeSocket1, consumeUri, consumeRequest1); - Future consumerFuture2 = consumeClient2.connect(consumeSocket2, consumeUri, consumeRequest2); + ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(consumeUri); + ClientUpgradeRequest consumeRequest2 = new ClientUpgradeRequest(consumeUri); + Future consumerFuture1 = consumeClient1.connect(consumeSocket1, consumeRequest1); + Future consumerFuture2 = consumeClient2.connect(consumeSocket2, consumeRequest2); log.info("Connecting to : {}", consumeUri); readClient.start(); - ClientUpgradeRequest readRequest = new ClientUpgradeRequest(); - Future readerFuture = readClient.connect(readSocket, readUri, readRequest); + ClientUpgradeRequest readRequest = new ClientUpgradeRequest(readUri); + Future readerFuture = readClient.connect(readSocket, readRequest); log.info("Connecting to : {}", readUri); // let it connect @@ -163,9 +164,9 @@ public void socketTest() throws Exception { // Also make sure subscriptions and reader are already created Thread.sleep(500); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); produceClient.start(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); assertTrue(producerFuture.get().isOpen()); int retry = 0; @@ -214,16 +215,16 @@ public void socketTestEndOfTopic() throws Exception { try { consumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); - Future consumerFuture = consumeClient.connect(consumeSocket, consumeUri, consumeRequest); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); + Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); log.info("Connecting to : {}", consumeUri); // let it connect assertTrue(consumerFuture.get().isOpen()); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); produceClient.start(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); assertTrue(producerFuture.get().isOpen()); // Send 30 message in total. produceSocket.sendMessage(20); @@ -284,8 +285,8 @@ public void unsubscribeTest() throws Exception { try { // setup a consumer consumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); - Future consumerFuture = consumeClient.connect(consumeSocket, consumeUri, consumeRequest); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); + Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); consumerFuture.get(); List subs = admin.topics().getSubscriptions(topic); assertEquals(subs.size(), 1); @@ -314,8 +315,8 @@ public void emptySubscriptionConsumerTest() { try { consumeClient1.start(); - ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(); - Future consumerFuture1 = consumeClient1.connect(consumeSocket1, consumeUri, consumeRequest1); + ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(consumeUri); + Future consumerFuture1 = consumeClient1.connect(consumeSocket1, consumeRequest1); consumerFuture1.get(); fail("should fail: empty subscription"); } catch (Exception e) { @@ -341,15 +342,15 @@ public void conflictingConsumerTest() throws Exception { try { consumeClient1.start(); - ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(); - Future consumerFuture1 = consumeClient1.connect(consumeSocket1, consumeUri, consumeRequest1); + ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(consumeUri); + Future consumerFuture1 = consumeClient1.connect(consumeSocket1, consumeRequest1); consumerFuture1.get(); try { consumeClient2.start(); - ClientUpgradeRequest consumeRequest2 = new ClientUpgradeRequest(); + ClientUpgradeRequest consumeRequest2 = new ClientUpgradeRequest(consumeUri); Future consumerFuture2 = - consumeClient2.connect(consumeSocket2, consumeUri, consumeRequest2); + consumeClient2.connect(consumeSocket2, consumeRequest2); consumerFuture2.get(); fail("should fail: conflicting subscription name"); } catch (Exception e) { @@ -378,15 +379,15 @@ public void conflictingProducerTest() throws Exception { try { produceClient1.start(); - ClientUpgradeRequest produceRequest1 = new ClientUpgradeRequest(); - Future producerFuture1 = produceClient1.connect(produceSocket1, produceUri, produceRequest1); + ClientUpgradeRequest produceRequest1 = new ClientUpgradeRequest(produceUri); + Future producerFuture1 = produceClient1.connect(produceSocket1, produceRequest1); producerFuture1.get(); try { produceClient2.start(); - ClientUpgradeRequest produceRequest2 = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest2 = new ClientUpgradeRequest(produceUri); Future producerFuture2 = - produceClient2.connect(produceSocket2, produceUri, produceRequest2); + produceClient2.connect(produceSocket2, produceRequest2); producerFuture2.get(); fail("should fail: conflicting producer name"); } catch (Exception e) { @@ -434,8 +435,8 @@ public void producerBacklogQuotaExceededTest() throws Exception { // Create subscription try { consumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); - Future consumerFuture = consumeClient.connect(consumeSocket, consumeUri, consumeRequest); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); + Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); consumerFuture.get(); } finally { stopWebSocketClient(consumeClient); @@ -444,8 +445,8 @@ public void producerBacklogQuotaExceededTest() throws Exception { // Fill the backlog try { produceClient1.start(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - Future producerFuture = produceClient1.connect(produceSocket1, produceUri, produceRequest); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); + Future producerFuture = produceClient1.connect(produceSocket1, produceRequest); producerFuture.get(); produceSocket1.sendMessage(100); } finally { @@ -456,16 +457,15 @@ public void producerBacklogQuotaExceededTest() throws Exception { // New producer fails to connect try { + produceClient2.setIdleTimeout(Duration.ofSeconds(5)); produceClient2.start(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - Future producerFuture = produceClient2.connect(produceSocket2, produceUri, produceRequest); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); + Future producerFuture = produceClient2.connect(produceSocket2, produceRequest); producerFuture.get(); fail("should fail: backlog quota exceeded"); } catch (Exception e) { // Expected assertTrue(e.getCause() instanceof UpgradeException); - assertEquals(((UpgradeException) e.getCause()).getResponseStatusCode(), - HttpServletResponse.SC_SERVICE_UNAVAILABLE); } finally { stopWebSocketClient(produceClient2); admin.topics().skipAllMessages("persistent://" + topic, subscription); @@ -504,8 +504,8 @@ public void topicDoesNotExistTest() throws Exception { try { produceClient.start(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); producerFuture.get(); fail("should fail: topic does not exist"); } catch (Exception e) { @@ -518,8 +518,8 @@ public void topicDoesNotExistTest() throws Exception { try { consumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); - Future consumerFuture = consumeClient.connect(consumeSocket, consumeUri, consumeRequest); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); + Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); consumerFuture.get(); fail("should fail: topic does not exist"); } catch (Exception e) { @@ -548,8 +548,8 @@ public void producerFencedTest() throws Exception { try { produceClient.start(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); producerFuture.get(); fail("should fail: producer fenced"); } catch (Exception e) { @@ -577,8 +577,8 @@ public void topicTerminatedTest() throws Exception { try { produceClient.start(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); producerFuture.get(); fail("should fail: topic terminated"); } catch (Exception e) { @@ -621,13 +621,13 @@ public void testProxyStats() throws Exception { try { consumeClient1.start(); - ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(); - Future consumerFuture1 = consumeClient1.connect(consumeSocket1, consumeUri, consumeRequest1); + ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(consumeUri); + Future consumerFuture1 = consumeClient1.connect(consumeSocket1, consumeRequest1); log.info("Connecting to : {}", consumeUri); readClient.start(); - ClientUpgradeRequest readRequest = new ClientUpgradeRequest(); - Future readerFuture = readClient.connect(readSocket, readUri, readRequest); + ClientUpgradeRequest readRequest = new ClientUpgradeRequest(readUri); + Future readerFuture = readClient.connect(readSocket, readRequest); log.info("Connecting to : {}", readUri); assertTrue(consumerFuture1.get().isOpen()); @@ -635,9 +635,9 @@ public void testProxyStats() throws Exception { Thread.sleep(500); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); produceClient.start(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); // let it connect assertTrue(producerFuture.get().isOpen()); @@ -691,8 +691,8 @@ public void consumeMessagesInPartitionedTopicTest() throws Exception { try { produceClient.start(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); producerFuture.get(); produceSocket.sendMessage(100); } finally { @@ -703,8 +703,8 @@ public void consumeMessagesInPartitionedTopicTest() throws Exception { try { consumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); - Future consumerFuture = consumeClient.connect(consumeSocket, consumeUri, consumeRequest); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); + Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); consumerFuture.get(); } finally { stopWebSocketClient(consumeClient); @@ -735,19 +735,19 @@ public void socketPullModeTest() throws Exception { try { consumeClient1.start(); consumeClient2.start(); - ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(); - ClientUpgradeRequest consumeRequest2 = new ClientUpgradeRequest(); - Future consumerFuture1 = consumeClient1.connect(consumeSocket1, consumeUri, consumeRequest1); - Future consumerFuture2 = consumeClient2.connect(consumeSocket2, consumeUri, consumeRequest2); + ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(consumeUri); + ClientUpgradeRequest consumeRequest2 = new ClientUpgradeRequest(consumeUri); + Future consumerFuture1 = consumeClient1.connect(consumeSocket1, consumeRequest1); + Future consumerFuture2 = consumeClient2.connect(consumeSocket2, consumeRequest2); log.info("Connecting to : {}", consumeUri); // let it connect assertTrue(consumerFuture1.get().isOpen()); assertTrue(consumerFuture2.get().isOpen()); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); produceClient.start(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); assertTrue(producerFuture.get().isOpen()); produceSocket.sendMessage(100); @@ -808,20 +808,20 @@ public void nackMessageTest() throws Exception { try { consumeClient1.start(); consumeClient2.start(); - ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(); - ClientUpgradeRequest consumeRequest2 = new ClientUpgradeRequest(); + ClientUpgradeRequest consumeRequest1 = new ClientUpgradeRequest(URI.create(consumerUri)); + ClientUpgradeRequest consumeRequest2 = new ClientUpgradeRequest(URI.create(dlqUri)); Future consumerFuture1 = - consumeClient1.connect(consumeSocket1, URI.create(consumerUri), consumeRequest1); + consumeClient1.connect(consumeSocket1, consumeRequest1); Future consumerFuture2 = - consumeClient2.connect(consumeSocket2, URI.create(dlqUri), consumeRequest2); + consumeClient2.connect(consumeSocket2, consumeRequest2); assertTrue(consumerFuture1.get().isOpen()); assertTrue(consumerFuture2.get().isOpen()); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(URI.create(producerUri)); produceClient.start(); Future producerFuture = - produceClient.connect(produceSocket, URI.create(producerUri), produceRequest); + produceClient.connect(produceSocket, produceRequest); assertTrue(producerFuture.get().isOpen()); assertEquals(consumeSocket1.getReceivedMessagesCount(), 0); @@ -868,15 +868,13 @@ public void nackRedeliveryDelayTest() throws Exception { try { consumeClient.start(); - final ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); - final Future consumerFuture = consumeClient.connect(consumeSocket, URI.create(consumerUri), - consumeRequest); + final ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(URI.create(consumerUri)); + final Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); assertTrue(consumerFuture.get().isOpen()); produceClient.start(); - final ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - final Future producerFuture = produceClient.connect(produceSocket, URI.create(producerUri), - produceRequest); + final ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(URI.create(producerUri)); + final Future producerFuture = produceClient.connect(produceSocket, produceRequest); assertTrue(producerFuture.get().isOpen()); assertEquals(consumeSocket.getReceivedMessagesCount(), 0); @@ -914,9 +912,8 @@ public void ackBatchMessageTest() throws Exception { try { consumerClient.start(); - ClientUpgradeRequest consumerRequest = new ClientUpgradeRequest(); - Future consumerFuture = - consumerClient.connect(consumeSocket, URI.create(consumerUri), consumerRequest); + ClientUpgradeRequest consumerRequest = new ClientUpgradeRequest(URI.create(consumerUri)); + Future consumerFuture = consumerClient.connect(consumeSocket, consumerRequest); assertTrue(consumerFuture.get().isOpen()); assertEquals(consumeSocket.getReceivedMessagesCount(), 0); @@ -995,9 +992,8 @@ public void consumeEncryptedMessages() throws Exception { try { consumerClient.start(); - ClientUpgradeRequest consumerRequest = new ClientUpgradeRequest(); - Future consumerFuture = - consumerClient.connect(consumeSocket, URI.create(consumerUri), consumerRequest); + ClientUpgradeRequest consumerRequest = new ClientUpgradeRequest(URI.create(consumerUri)); + Future consumerFuture = consumerClient.connect(consumeSocket, consumerRequest); assertTrue(consumerFuture.get().isOpen()); assertEquals(consumeSocket.getReceivedMessagesCount(), 0); @@ -1136,12 +1132,10 @@ public void testMultiTopics() throws Exception { try { consumerClient1.start(); consumerClient2.start(); - ClientUpgradeRequest consumerRequest1 = new ClientUpgradeRequest(); - ClientUpgradeRequest consumerRequest2 = new ClientUpgradeRequest(); - Future consumerFuture1 = - consumerClient1.connect(consumeSocket1, URI.create(consumerUri1), consumerRequest1); - Future consumerFuture2 = - consumerClient2.connect(consumeSocket2, URI.create(consumerUri2), consumerRequest2); + ClientUpgradeRequest consumerRequest1 = new ClientUpgradeRequest(URI.create(consumerUri1)); + ClientUpgradeRequest consumerRequest2 = new ClientUpgradeRequest(URI.create(consumerUri2)); + Future consumerFuture1 = consumerClient1.connect(consumeSocket1, consumerRequest1); + Future consumerFuture2 = consumerClient2.connect(consumeSocket2, consumerRequest2); assertTrue(consumerFuture1.get().isOpen()); assertTrue(consumerFuture2.get().isOpen()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTlsTest.java index 8ffa09de64e7c..1c5439d4b9e96 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeTlsTest.java @@ -24,10 +24,10 @@ import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doReturn; import java.net.URI; -import java.security.GeneralSecurityException; import java.util.Optional; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import lombok.Cleanup; import org.apache.pulsar.client.api.TlsProducerConsumerBase; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.util.SecurityUtility; @@ -37,6 +37,7 @@ import org.apache.pulsar.websocket.service.WebSocketProxyConfiguration; import org.apache.pulsar.websocket.service.WebSocketServiceStarter; import org.awaitility.Awaitility; +import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.eclipse.jetty.websocket.api.Session; import org.eclipse.jetty.websocket.client.ClientUpgradeRequest; @@ -95,7 +96,7 @@ protected void cleanup() throws Exception { } @Test(timeOut = 30000) - public void socketTest() throws GeneralSecurityException { + public void socketTest() throws Exception { String consumerUri = "wss://localhost:" + proxyServer.getListenPortHTTPS().get() + "/ws/consumer/persistent/my-property/use/my-ns/my-topic/my-sub"; @@ -104,25 +105,30 @@ public void socketTest() throws GeneralSecurityException { URI consumeUri = URI.create(consumerUri); URI produceUri = URI.create(producerUri); - SslContextFactory sslContextFactory = new SslContextFactory(); + SslContextFactory.Client sslContextFactory = new SslContextFactory.Client(); sslContextFactory.setSslContext(SecurityUtility .createSslContext(false, SecurityUtility.loadCertificatesFromPemFile(CA_CERT_FILE_PATH), null)); + @Cleanup("stop") + HttpClient httpClient = new HttpClient(); + httpClient.setSslContextFactory(sslContextFactory); - WebSocketClient consumeClient = new WebSocketClient(sslContextFactory); + @Cleanup("stop") + WebSocketClient consumeClient = new WebSocketClient(httpClient); SimpleConsumerSocket consumeSocket = new SimpleConsumerSocket(); - WebSocketClient produceClient = new WebSocketClient(sslContextFactory); + @Cleanup("stop") + WebSocketClient produceClient = new WebSocketClient(httpClient); try { consumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); - Future consumerFuture = consumeClient.connect(consumeSocket, consumeUri, consumeRequest); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); + Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); log.info("Connecting to : {}", consumeUri); Assert.assertTrue(consumerFuture.get().isOpen()); SimpleProducerSocket produceSocket = new SimpleProducerSocket(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); produceClient.start(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); Assert.assertTrue(producerFuture.get().isOpen()); @@ -132,9 +138,6 @@ public void socketTest() throws GeneralSecurityException { }); consumeSocket.awaitClose(1, TimeUnit.SECONDS); produceSocket.awaitClose(1, TimeUnit.SECONDS); - } catch (Throwable t) { - log.error(t.getMessage()); - Assert.fail(t.getMessage()); } finally { try { consumeClient.stop(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeWithoutZKTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeWithoutZKTest.java index e923fd85613ec..983aa1bff77db 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeWithoutZKTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyPublishConsumeWithoutZKTest.java @@ -96,13 +96,13 @@ public void socketTest() throws Exception { try { consumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); - Future consumerFuture = consumeClient.connect(consumeSocket, consumeUri, consumeRequest); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); + Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); log.info("Connecting to : {}", consumeUri); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); produceClient.start(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); // let it connect Assert.assertTrue(consumerFuture.get().isOpen()); Assert.assertTrue(producerFuture.get().isOpen()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyRoleAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyRoleAuthTest.java index 0eb90b9fe363d..c97dbb3812b98 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyRoleAuthTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyRoleAuthTest.java @@ -231,18 +231,18 @@ public void testWebSocketProxyProduceConsumeWithAuthorization() throws Exception try { // Connect consumer with CLIENT_TOKEN in Authorization header consumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); // Add JWT token authentication for WebSocket client consumeRequest.setHeader("Authorization", "Bearer " + CLIENT_TOKEN); - Future consumerFuture = consumeClient.connect(consumeSocket, consumeUri, consumeRequest); + Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); log.info("Connecting consumer to: {} with CLIENT_TOKEN", consumeUri); // Connect producer with CLIENT_TOKEN in Authorization header produceClient.start(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); // Add JWT token authentication for WebSocket client produceRequest.setHeader("Authorization", "Bearer " + CLIENT_TOKEN); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); log.info("Connecting producer to: {} with CLIENT_TOKEN", produceUri); // Verify connections are established @@ -316,11 +316,10 @@ public void testWebSocketProxyWithUnauthorizedToken() throws Exception { SimpleConsumerSocket consumeSocket = new SimpleConsumerSocket(); unauthorizedConsumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); // Use UNAUTHORIZED_TOKEN which doesn't have permissions consumeRequest.setHeader("Authorization", "Bearer " + UNAUTHORIZED_TOKEN); - Future consumerFuture = unauthorizedConsumeClient.connect(consumeSocket, consumeUri, - consumeRequest); + Future consumerFuture = unauthorizedConsumeClient.connect(consumeSocket, consumeRequest); log.info("Attempting to connect consumer with unauthorized token to: {}", consumeUri); @@ -340,11 +339,10 @@ public void testWebSocketProxyWithUnauthorizedToken() throws Exception { SimpleProducerSocket produceSocket = new SimpleProducerSocket(); unauthorizedProduceClient.start(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); // Use UNAUTHORIZED_TOKEN which doesn't have permissions produceRequest.setHeader("Authorization", "Bearer " + UNAUTHORIZED_TOKEN); - Future producerFuture = unauthorizedProduceClient.connect(produceSocket, produceUri, - produceRequest); + Future producerFuture = unauthorizedProduceClient.connect(produceSocket, produceRequest); log.info("Attempting to connect producer with unauthorized token to: {}", produceUri); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/SimpleConsumerSocket.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/SimpleConsumerSocket.java index 6510c69d1ae5a..6ea3fa5e19525 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/SimpleConsumerSocket.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/SimpleConsumerSocket.java @@ -29,16 +29,16 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import org.eclipse.jetty.websocket.api.RemoteEndpoint; +import org.eclipse.jetty.websocket.api.Callback; import org.eclipse.jetty.websocket.api.Session; import org.eclipse.jetty.websocket.api.annotations.OnWebSocketClose; -import org.eclipse.jetty.websocket.api.annotations.OnWebSocketConnect; import org.eclipse.jetty.websocket.api.annotations.OnWebSocketMessage; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketOpen; import org.eclipse.jetty.websocket.api.annotations.WebSocket; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@WebSocket(maxTextMessageSize = 64 * 1024) +@WebSocket public class SimpleConsumerSocket { private static final String X_PULSAR_MESSAGE_ID = "messageId"; private final CountDownLatch closeLatch; @@ -70,7 +70,7 @@ public void onClose(int statusCode, String reason) { this.closeLatch.countDown(); } - @OnWebSocketConnect + @OnWebSocketOpen public void onConnect(Session session) throws InterruptedException { log.info("Got connect: {}", session); this.session = session; @@ -86,12 +86,12 @@ public synchronized void onMessage(String msg) throws JsonParseException, IOExce String messageId = message.get(X_PULSAR_MESSAGE_ID).getAsString(); consumerBuffer.add(messageId); if (customMessageHandler != null) { - this.getRemote().sendString(customMessageHandler.handle(messageId, message)); + this.getSession().sendText(customMessageHandler.handle(messageId, message), Callback.NOOP); } else { JsonObject ack = new JsonObject(); ack.add("messageId", new JsonPrimitive(messageId)); // Acking the proxy - this.getRemote().sendString(ack.toString()); + this.getSession().sendText(ack.toString(), Callback.NOOP); } } else { consumerBuffer.add(message.toString()); @@ -102,23 +102,19 @@ public void sendPermits(int nbPermits) throws IOException { JsonObject permitMessage = new JsonObject(); permitMessage.add("type", new JsonPrimitive("permit")); permitMessage.add("permitMessages", new JsonPrimitive(nbPermits)); - this.getRemote().sendString(permitMessage.toString()); + this.getSession().sendText(permitMessage.toString(), Callback.NOOP); } public void unsubscribe() throws IOException { JsonObject message = new JsonObject(); message.add("type", new JsonPrimitive("unsubscribe")); - this.getRemote().sendString(message.toString()); + this.getSession().sendText(message.toString(), Callback.NOOP); } public void isEndOfTopic() throws IOException { JsonObject message = new JsonObject(); message.add("type", new JsonPrimitive("isEndOfTopic")); - this.getRemote().sendString(message.toString()); - } - - public RemoteEndpoint getRemote() { - return this.session.getRemote(); + this.getSession().sendText(message.toString(), Callback.NOOP); } public Session getSession() { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/SimpleProducerSocket.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/SimpleProducerSocket.java index efd02dc0e5ee0..b3c86c919319c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/SimpleProducerSocket.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/SimpleProducerSocket.java @@ -30,16 +30,16 @@ import java.util.concurrent.TimeUnit; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.websocket.data.ProducerMessage; -import org.eclipse.jetty.websocket.api.RemoteEndpoint; +import org.eclipse.jetty.websocket.api.Callback; import org.eclipse.jetty.websocket.api.Session; import org.eclipse.jetty.websocket.api.annotations.OnWebSocketClose; -import org.eclipse.jetty.websocket.api.annotations.OnWebSocketConnect; import org.eclipse.jetty.websocket.api.annotations.OnWebSocketMessage; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketOpen; import org.eclipse.jetty.websocket.api.annotations.WebSocket; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@WebSocket(maxTextMessageSize = 64 * 1024) +@WebSocket public class SimpleProducerSocket { private final CountDownLatch closeLatch; @@ -75,7 +75,7 @@ public void onClose(int statusCode, String reason) { this.closeLatch.countDown(); } - @OnWebSocketConnect + @OnWebSocketOpen public void onConnect(Session session) throws Exception { log.info("Got connect: {}", session); this.session = session; @@ -84,7 +84,7 @@ public void onConnect(Session session) throws Exception { public void sendMessage(int totalMsgs) throws Exception { for (int i = 0; i < totalMsgs; i++) { - this.session.getRemote().sendString(getTestJsonPayload(i)); + this.session.sendText(getTestJsonPayload(i), Callback.NOOP); } } @@ -94,10 +94,6 @@ public synchronized void onMessage(String msg) throws JsonParseException { producerBuffer.add(ack.get("messageId").getAsString()); } - public RemoteEndpoint getRemote() { - return this.session.getRemote(); - } - public Session getSession() { return this.session; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/v1/V1ProxyAuthenticationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/v1/V1ProxyAuthenticationTest.java index e27c6a882f361..ccc09bd0b048f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/v1/V1ProxyAuthenticationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/v1/V1ProxyAuthenticationTest.java @@ -129,13 +129,13 @@ private void socketTest() throws Exception { SimpleProducerSocket produceSocket = new SimpleProducerSocket(); consumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); - Future consumerFuture = consumeClient.connect(consumeSocket, consumeUri, consumeRequest); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); + Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); log.info("Connecting to : {}", consumeUri); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); produceClient.start(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); Assert.assertTrue(consumerFuture.get().isOpen()); Assert.assertTrue(producerFuture.get().isOpen()); Awaitility.await().untilAsserted(() -> { @@ -186,13 +186,13 @@ public void statsTest() throws Exception { try { consumeClient.start(); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); - Future consumerFuture = consumeClient.connect(consumeSocket, consumeUri, consumeRequest); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumeUri); + Future consumerFuture = consumeClient.connect(consumeSocket, consumeRequest); Assert.assertTrue(consumerFuture.get().isOpen()); produceClient.start(); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - Future producerFuture = produceClient.connect(produceSocket, produceUri, produceRequest); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); + Future producerFuture = produceClient.connect(produceSocket, produceRequest); Assert.assertTrue(producerFuture.get().isOpen()); Awaitility.await().untilAsserted(() -> Assert.assertTrue(consumeSocket.getReceivedMessagesCount() >= 3)); diff --git a/pulsar-client-admin/pom.xml b/pulsar-client-admin/pom.xml index ccd409f34be7e..6749bdc8ae046 100644 --- a/pulsar-client-admin/pom.xml +++ b/pulsar-client-admin/pom.xml @@ -115,7 +115,7 @@ com.github.tomakehurst - wiremock-jre8 + wiremock-jre8-standalone ${wiremock.version} test diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java index e4ca7724ca1d7..b8338297ec13a 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java @@ -106,6 +106,8 @@ public class PulsarAdminImpl implements PulsarAdmin { private final Transactions transactions; protected final WebTarget root; protected final Authentication auth; + @Getter + private AsyncHttpConnectorProvider asyncConnectorProvider; public PulsarAdminImpl(String serviceUrl, ClientConfigurationData clientConfigData, ClassLoader clientBuilderClassLoader) throws PulsarClientException { @@ -128,7 +130,7 @@ public PulsarAdminImpl(String serviceUrl, ClientConfigurationData clientConfigDa clientConfigData.setServiceUrl(serviceUrl); } - AsyncHttpConnectorProvider asyncConnectorProvider = new AsyncHttpConnectorProvider(clientConfigData, + asyncConnectorProvider = new AsyncHttpConnectorProvider(clientConfigData, clientConfigData.getAutoCertRefreshSeconds(), acceptGzipCompression); ClientConfig httpConfig = new ClientConfig(); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java index ed17df8bd73ec..4bd361d803f59 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java @@ -58,6 +58,7 @@ import javax.ws.rs.core.Response.Status; import lombok.Data; import lombok.Getter; +import lombok.Setter; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.Validate; @@ -117,6 +118,9 @@ public class AsyncHttpConnector implements Connector, AsyncHttpRequestExecutor { private final boolean createdEventLoopGroup; private final Map> concurrencyReducers = new ConcurrentHashMap<>(); private PulsarSslFactory sslFactory; + @Getter + @Setter + private boolean followRedirects = true; public AsyncHttpConnector(Client client, ClientConfigurationData conf, int autoCertRefreshTimeSeconds, boolean acceptGzipCompression) { @@ -453,7 +457,7 @@ private CompletableFuture executeRequest(Request request, responseFuture = doExecuteRequest(request, handlerSupplier); } CompletableFuture futureWithRedirect = responseFuture.thenCompose(response -> { - if (isRedirectStatusCode(response.getStatusCode())) { + if (followRedirects && isRedirectStatusCode(response.getStatusCode())) { return executeRedirect(request, response, handlerSupplier, redirectCount); } return CompletableFuture.completedFuture(response); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorProvider.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorProvider.java index caaa356f7c709..2f1e8f798a6c6 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorProvider.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorProvider.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.admin.internal.http; +import com.google.common.annotations.VisibleForTesting; import javax.ws.rs.client.Client; import javax.ws.rs.core.Configuration; import org.apache.pulsar.client.impl.PulsarClientSharedResourcesImpl; @@ -31,9 +32,10 @@ public class AsyncHttpConnectorProvider implements ConnectorProvider { private final ClientConfigurationData conf; - private Connector connector; + private AsyncHttpConnector connector; private final int autoCertRefreshTimeSeconds; private final boolean acceptGzipCompression; + private boolean followRedirects = true; public AsyncHttpConnectorProvider(ClientConfigurationData conf, int autoCertRefreshTimeSeconds, boolean acceptGzipCompression) { @@ -46,6 +48,7 @@ public AsyncHttpConnectorProvider(ClientConfigurationData conf, int autoCertRefr public Connector getConnector(Client client, Configuration runtimeConfig) { if (connector == null) { connector = new AsyncHttpConnector(client, conf, autoCertRefreshTimeSeconds, acceptGzipCompression); + connector.setFollowRedirects(followRedirects); } return connector; } @@ -56,4 +59,16 @@ public AsyncHttpConnector getConnector(int connectTimeoutMs, int readTimeoutMs, return new AsyncHttpConnector(connectTimeoutMs, readTimeoutMs, requestTimeoutMs, autoCertRefreshTimeSeconds, conf, acceptGzipCompression, sharedResources); } + + @VisibleForTesting + public AsyncHttpConnector getAsyncHttpConnector() { + return connector; + } + + public void setFollowRedirects(boolean followRedirects) { + this.followRedirects = followRedirects; + if (connector != null) { + connector.setFollowRedirects(followRedirects); + } + } } diff --git a/pulsar-client-tools-test/pom.xml b/pulsar-client-tools-test/pom.xml index a0a3849b0f3cf..276dd4bcecb2d 100644 --- a/pulsar-client-tools-test/pom.xml +++ b/pulsar-client-tools-test/pom.xml @@ -61,6 +61,11 @@ ${project.version} test + + org.eclipse.jetty.ee8.websocket + jetty-ee8-websocket-jetty-server + test + ${project.groupId} diff --git a/pulsar-client-tools/pom.xml b/pulsar-client-tools/pom.xml index 5e50a97739b26..abda1746c03b8 100644 --- a/pulsar-client-tools/pom.xml +++ b/pulsar-client-tools/pom.xml @@ -110,8 +110,11 @@ org.eclipse.jetty.websocket - javax-websocket-client-impl - ${jetty.version} + jetty-websocket-jetty-api + + + org.eclipse.jetty.websocket + jetty-websocket-jetty-client io.swagger diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/AbstractCmdConsume.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/AbstractCmdConsume.java index 55b189ee0071d..1827bbce18d1c 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/AbstractCmdConsume.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/AbstractCmdConsume.java @@ -44,11 +44,11 @@ import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.collections.GrowableArrayBlockingQueue; -import org.eclipse.jetty.websocket.api.RemoteEndpoint; +import org.eclipse.jetty.websocket.api.Callback; import org.eclipse.jetty.websocket.api.Session; import org.eclipse.jetty.websocket.api.annotations.OnWebSocketClose; -import org.eclipse.jetty.websocket.api.annotations.OnWebSocketConnect; import org.eclipse.jetty.websocket.api.annotations.OnWebSocketMessage; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketOpen; import org.eclipse.jetty.websocket.api.annotations.WebSocket; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -230,7 +230,7 @@ protected static Map genericRecordToMap(GenericRecord value, boo return res; } - @WebSocket(maxTextMessageSize = 64 * 1024) + @WebSocket public static class ConsumerSocket { private static final String X_PULSAR_MESSAGE_ID = "messageId"; private final CountDownLatch closeLatch; @@ -255,7 +255,7 @@ public void onClose(int statusCode, String reason) { this.closeLatch.countDown(); } - @OnWebSocketConnect + @OnWebSocketOpen public void onConnect(Session session) throws InterruptedException { log.info("Got connect: {}", session); this.session = session; @@ -269,7 +269,7 @@ public synchronized void onMessage(String msg) throws Exception { String messageId = message.get(X_PULSAR_MESSAGE_ID).getAsString(); ack.add("messageId", new JsonPrimitive(messageId)); // Acking the proxy - this.getRemote().sendString(ack.toString()); + this.getSession().sendText(ack.toString(), Callback.NOOP); this.incomingMessages.put(msg); } @@ -277,10 +277,6 @@ public String receive(long timeout, TimeUnit unit) throws Exception { return incomingMessages.poll(timeout, unit); } - public RemoteEndpoint getRemote() { - return this.session.getRemote(); - } - public Session getSession() { return this.session; } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java index 7f9288ee8e8e5..06a15eb181c37 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java @@ -39,6 +39,7 @@ import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.TopicName; +import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.eclipse.jetty.websocket.client.ClientUpgradeRequest; import org.eclipse.jetty.websocket.client.WebSocketClient; @@ -278,8 +279,11 @@ private int consumeFromWebSocket(String topic) { URI consumerUri = URI.create(getWebSocketConsumeUri(topic)); - WebSocketClient consumeClient = new WebSocketClient(new SslContextFactory(true)); - ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(); + HttpClient httpClient = new HttpClient(); + httpClient.setSslContextFactory(new SslContextFactory.Client(true)); + WebSocketClient consumeClient = new WebSocketClient(httpClient); + consumeClient.setMaxTextMessageSize(64 * 1024); + ClientUpgradeRequest consumeRequest = new ClientUpgradeRequest(consumerUri); try { if (authentication != null) { authentication.start(); @@ -305,7 +309,7 @@ private int consumeFromWebSocket(String topic) { try { LOG.info("Trying to create websocket session..{}", consumerUri); - consumeClient.connect(consumerSocket, consumerUri, consumeRequest); + consumeClient.connect(consumerSocket, consumeRequest); connected.get(); } catch (Exception e) { LOG.error("Failed to create web-socket session", e); @@ -340,6 +344,17 @@ private int consumeFromWebSocket(String topic) { LOG.info("{} messages successfully consumed", numMessagesConsumed); } + + try { + consumeClient.stop(); + } catch (Exception e) { + LOG.error("Failed to stop websocket-client", e); + } + try { + httpClient.stop(); + } catch (Exception e) { + LOG.error("Failed to stop http-client", e); + } return returnCode; } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdProduce.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdProduce.java index 01ac1df333dcf..b166cd9372874 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdProduce.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdProduce.java @@ -63,12 +63,13 @@ import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.websocket.data.ProducerMessage; +import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.util.ssl.SslContextFactory; -import org.eclipse.jetty.websocket.api.RemoteEndpoint; +import org.eclipse.jetty.websocket.api.Callback; import org.eclipse.jetty.websocket.api.Session; import org.eclipse.jetty.websocket.api.annotations.OnWebSocketClose; -import org.eclipse.jetty.websocket.api.annotations.OnWebSocketConnect; import org.eclipse.jetty.websocket.api.annotations.OnWebSocketMessage; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketOpen; import org.eclipse.jetty.websocket.api.annotations.WebSocket; import org.eclipse.jetty.websocket.client.ClientUpgradeRequest; import org.eclipse.jetty.websocket.client.WebSocketClient; @@ -464,8 +465,12 @@ private int publishToWebSocket(String topic) { URI produceUri = URI.create(getWebSocketProduceUri(topic)); - WebSocketClient produceClient = new WebSocketClient(new SslContextFactory(true)); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + HttpClient httpClient = new HttpClient(); + httpClient.setSslContextFactory(new SslContextFactory.Client(true)); + WebSocketClient produceClient = new WebSocketClient(httpClient); + produceClient.setMaxTextMessageSize(64 * 1024); + + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); try { if (authentication != null) { authentication.start(); @@ -492,7 +497,7 @@ private int publishToWebSocket(String topic) { try { LOG.info("Trying to create websocket session.. on {},{}", produceUri, produceRequest); - produceClient.connect(produceSocket, produceUri, produceRequest); + produceClient.connect(produceSocket, produceRequest); connected.get(); } catch (Exception e) { LOG.error("Failed to create web-socket session", e); @@ -521,10 +526,21 @@ private int publishToWebSocket(String topic) { LOG.info("{} messages successfully produced", numMessagesSent); } + try { + produceClient.stop(); + } catch (Exception e) { + LOG.error("Failed to stop websocket-client", e); + } + try { + httpClient.stop(); + } catch (Exception e) { + LOG.error("Failed to stop http-client", e); + } + return returnCode; } - @WebSocket(maxTextMessageSize = 64 * 1024) + @WebSocket public static class ProducerSocket { private final CountDownLatch closeLatch; @@ -538,7 +554,7 @@ public ProducerSocket(CompletableFuture connected) { } public CompletableFuture send(int index, byte[] content) throws Exception { - this.session.getRemote().sendString(getTestJsonPayload(index, content)); + this.session.sendText(getTestJsonPayload(index, content), Callback.NOOP); this.result = new CompletableFuture<>(); return result; } @@ -561,7 +577,7 @@ public void onClose(int statusCode, String reason) { this.closeLatch.countDown(); } - @OnWebSocketConnect + @OnWebSocketOpen public void onConnect(Session session) { LOG.info("Got connect: {}", session); this.session = session; @@ -576,10 +592,6 @@ public synchronized void onMessage(String msg) throws JsonParseException { } } - public RemoteEndpoint getRemote() { - return this.session.getRemote(); - } - public Session getSession() { return this.session; } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java index d0e20dfa57083..72f6bacbb88d5 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java @@ -40,6 +40,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicName; +import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.eclipse.jetty.websocket.client.ClientUpgradeRequest; import org.eclipse.jetty.websocket.client.WebSocketClient; @@ -239,8 +240,10 @@ private int readFromWebSocket(String topic) { URI readerUri = URI.create(getWebSocketReadUri(topic)); - WebSocketClient readClient = new WebSocketClient(new SslContextFactory(true)); - ClientUpgradeRequest readRequest = new ClientUpgradeRequest(); + HttpClient httpClient = new HttpClient(); + httpClient.setSslContextFactory(new SslContextFactory.Client(true)); + WebSocketClient readClient = new WebSocketClient(httpClient); + ClientUpgradeRequest readRequest = new ClientUpgradeRequest(readerUri); try { if (authentication != null) { authentication.start(); @@ -266,7 +269,7 @@ private int readFromWebSocket(String topic) { try { LOG.info("Trying to create websocket session..{}", readerUri); - readClient.connect(readerSocket, readerUri, readRequest); + readClient.connect(readerSocket, readRequest); connected.get(); } catch (Exception e) { LOG.error("Failed to create web-socket session", e); @@ -301,6 +304,17 @@ private int readFromWebSocket(String topic) { LOG.info("{} messages successfully read", numMessagesRead); } + try { + readClient.stop(); + } catch (Exception e) { + LOG.error("Failed to stop websocket-client", e); + } + try { + httpClient.stop(); + } catch (Exception e) { + LOG.error("Failed to stop http-client", e); + } + return returnCode; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java index 72fcf82b85958..51b267699c2c6 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.impl; +import com.fasterxml.jackson.databind.JsonNode; import io.netty.channel.EventLoopGroup; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpResponse; @@ -228,12 +229,31 @@ public CompletableFuture get(String path, Class clazz) { // request not success if (response2.getStatusCode() != HttpURLConnection.HTTP_OK) { - log.warn("[{}] HTTP get request failed: {}", requestUrl, response2.getStatusText()); + String errorReason = response2.getStatusText(); + if ("application/json".equals(response2.getContentType()) || "text/json".equals( + response2.getContentType())) { + String responseBody = response2.getResponseBody(); + try { + JsonNode jsonNode = + ObjectMapperFactory.getMapper().getObjectMapper().readTree(responseBody); + if (jsonNode.has("reason") && jsonNode.get("reason").isTextual()) { + errorReason = jsonNode.get("reason").asText(); + } else if (jsonNode.has("message") && jsonNode.get("message").isTextual()) { + errorReason = jsonNode.get("message").asText(); + } + } catch (IOException e) { + // ignore + if (log.isDebugEnabled()) { + log.debug("[{}] Failed to parse error response: {}", requestUrl, responseBody); + } + } + } + log.warn("[{}] HTTP get request failed: {}", requestUrl, errorReason); Exception e; if (response2.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { - e = new NotFoundException("Not found: " + response2.getStatusText()); + e = new NotFoundException("Not found: " + errorReason); } else { - e = new PulsarClientException("HTTP get request failed: " + response2.getStatusText()); + e = new PulsarClientException("HTTP get request failed: " + errorReason); } future.completeExceptionally(e); return; diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index a06612629562e..bce1e4c7b34f5 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -97,6 +97,11 @@ netty-handler + + io.netty + netty-buffer + + io.netty netty-resolver-dns diff --git a/pulsar-functions/utils/pom.xml b/pulsar-functions/utils/pom.xml index 868ce39fc287f..7c2c1aa00688b 100644 --- a/pulsar-functions/utils/pom.xml +++ b/pulsar-functions/utils/pom.xml @@ -118,7 +118,7 @@ com.github.tomakehurst - wiremock-jre8 + wiremock-jre8-standalone ${wiremock.version} test diff --git a/pulsar-functions/worker/pom.xml b/pulsar-functions/worker/pom.xml index b27b6d7357482..dc48eef74bd38 100644 --- a/pulsar-functions/worker/pom.xml +++ b/pulsar-functions/worker/pom.xml @@ -104,13 +104,13 @@ - org.eclipse.jetty - jetty-servlet + org.eclipse.jetty.ee8 + jetty-ee8-servlet - org.eclipse.jetty - jetty-servlets + org.eclipse.jetty.ee8 + jetty-ee8-servlets @@ -128,6 +128,11 @@ org.apache.bookkeeper bookkeeper-server + + org.apache.pulsar + zookeeper-with-patched-admin + ${project.version} + commons-io @@ -162,11 +167,6 @@ - - io.prometheus - simpleclient_jetty - - 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 4f01f17174e31..e65760caf891c 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 @@ -19,7 +19,6 @@ package org.apache.pulsar.functions.worker.rest; import io.opentelemetry.api.OpenTelemetry; -import io.prometheus.client.jetty.JettyStatisticsCollector; import java.util.ArrayList; import java.util.EnumSet; import java.util.List; @@ -43,13 +42,18 @@ import org.apache.pulsar.functions.worker.WorkerService; 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.metrics.JettyStatisticsCollector; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.ee8.servlet.FilterHolder; +import org.eclipse.jetty.ee8.servlet.ServletContextHandler; +import org.eclipse.jetty.ee8.servlet.ServletHolder; +import org.eclipse.jetty.http.UriCompliance; 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.NetworkConnectionLimit; import org.eclipse.jetty.server.ProxyConnectionFactory; import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; @@ -57,13 +61,8 @@ 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; -import org.eclipse.jetty.server.handler.RequestLogHandler; +import org.eclipse.jetty.server.handler.QoSHandler; import org.eclipse.jetty.server.handler.StatisticsHandler; -import org.eclipse.jetty.servlet.FilterHolder; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.QoSFilter; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.glassfish.jersey.server.ResourceConfig; import org.glassfish.jersey.servlet.ServletContainer; @@ -103,10 +102,11 @@ public void start() throws Exception { private void init() { server = new Server(webServerExecutor); if (workerConfig.getMaxHttpServerConnections() > 0) { - server.addBean(new ConnectionLimit(workerConfig.getMaxHttpServerConnections(), server)); + server.addBean(new NetworkConnectionLimit(workerConfig.getMaxHttpServerConnections(), server)); } HttpConfiguration httpConfig = new HttpConfiguration(); + httpConfig.setUriCompliance(UriCompliance.LEGACY); if (workerConfig.isWebServiceTrustXForwardedFor()) { httpConfig.addCustomizer(new ForwardedRequestCustomizer()); } @@ -127,28 +127,26 @@ private void init() { List handlers = new ArrayList<>(4); handlers.add(newServletContextHandler("/admin", - new ResourceConfig(Resources.getApiV2Resources()), workerService, filterInitializer)); + new ResourceConfig(Resources.getApiV2Resources()), workerService, filterInitializer).get()); handlers.add(newServletContextHandler("/admin/v2", - new ResourceConfig(Resources.getApiV2Resources()), workerService, filterInitializer)); + new ResourceConfig(Resources.getApiV2Resources()), workerService, filterInitializer).get()); handlers.add(newServletContextHandler("/admin/v3", - new ResourceConfig(Resources.getApiV3Resources()), workerService, filterInitializer)); + new ResourceConfig(Resources.getApiV3Resources()), workerService, filterInitializer).get()); // don't require auth for metrics or config routes handlers.add(newServletContextHandler("/", new ResourceConfig(Resources.getRootResources()), workerService, - workerConfig.isAuthenticateMetricsEndpoint(), filterInitializer)); + workerConfig.isAuthenticateMetricsEndpoint(), filterInitializer).get()); - RequestLogHandler requestLogHandler = new RequestLogHandler(); 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); + server.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); + ContextHandlerCollection contexts = new ContextHandlerCollection(); - contexts.setHandlers(handlers.toArray(new Handler[handlers.size()])); - HandlerCollection handlerCollection = new HandlerCollection(); - handlerCollection.setHandlers(new Handler[]{contexts, new DefaultHandler(), requestLogHandler}); + contexts.setHandlers(handlers); + Handler.Collection handlerCollection = new Handler.Sequence(); + handlerCollection.setHandlers(contexts, new DefaultHandler()); // Metrics handler StatisticsHandler stats = new StatisticsHandler(); @@ -158,8 +156,14 @@ private void init() { } catch (IllegalArgumentException e) { // Already registered. Eg: in unit tests } - handlers.add(stats); - server.setHandler(stats); + + Handler serverHandler = stats; + if (workerConfig.getMaxConcurrentHttpRequests() > 0) { + QoSHandler qoSHandler = new QoSHandler(serverHandler); + qoSHandler.setMaxRequestCount(workerConfig.getMaxConcurrentHttpRequests()); + serverHandler = qoSHandler; + } + server.setHandler(serverHandler); if (this.workerConfig.getTlsEnabled()) { log.info("Configuring https server on port={}", this.workerConfig.getWorkerPortTls()); @@ -175,7 +179,7 @@ private void init() { workerConfig.getTlsCertRefreshCheckDurationSec(), workerConfig.getTlsCertRefreshCheckDurationSec(), TimeUnit.SECONDS); - SslContextFactory sslCtxFactory = + SslContextFactory.Server sslCtxFactory = JettySslContextFactory.createSslContextFactory(this.workerConfig.getTlsProvider(), this.sslFactory, this.workerConfig.isTlsRequireTrustedClientCertOnConnect(), this.workerConfig.getWebServiceTlsCiphers(), @@ -189,7 +193,9 @@ private void init() { // 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()); + // disable SNI host check for backwards compatibility with Jetty 9.x + boolean sniHostCheck = false; + httpConfig.addCustomizer(new SecureRequestCustomizer(sniHostCheck)); } httpsConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpsConnector.setPort(this.workerConfig.getWorkerPortTls()); @@ -209,12 +215,6 @@ private static class FilterInitializer { private final FilterHolder authenticationFilterHolder; FilterInitializer(WorkerConfig config, AuthenticationService authenticationService) { - if (config.getMaxConcurrentHttpRequests() > 0) { - FilterHolder filterHolder = new FilterHolder(QoSFilter.class); - filterHolder.setInitParameter("maxRequests", String.valueOf(config.getMaxConcurrentHttpRequests())); - filterHolders.add(filterHolder); - } - if (config.isHttpRequestsLimitEnabled()) { filterHolders.add(new FilterHolder( new RateLimitingFilter(config.getHttpRequestsMaxPerSecond(), @@ -240,9 +240,9 @@ public void addFilters(ServletContextHandler context, boolean requiresAuthentica } static ServletContextHandler newServletContextHandler(String contextPath, - ResourceConfig config, - WorkerService workerService, - FilterInitializer filterInitializer) { + ResourceConfig config, + WorkerService workerService, + FilterInitializer filterInitializer) { return newServletContextHandler(contextPath, config, workerService, true, filterInitializer); } diff --git a/pulsar-io/alluxio/pom.xml b/pulsar-io/alluxio/pom.xml index 5f174bab69640..bba0fb92696ac 100644 --- a/pulsar-io/alluxio/pom.xml +++ b/pulsar-io/alluxio/pom.xml @@ -114,6 +114,13 @@ metrics-jvm ${metrics.version} + + org.eclipse.jetty + jetty-bom + ${jetty9.version} + pom + import + diff --git a/pulsar-io/debezium/core/pom.xml b/pulsar-io/debezium/core/pom.xml index ecf2c3cfd19ef..1825346ab4693 100644 --- a/pulsar-io/debezium/core/pom.xml +++ b/pulsar-io/debezium/core/pom.xml @@ -81,6 +81,10 @@ jose4j org.bitbucket.b_c + + org.eclipse.jetty + * + diff --git a/pulsar-io/hbase/pom.xml b/pulsar-io/hbase/pom.xml index 39462bd8d6dff..f49475c61f25b 100644 --- a/pulsar-io/hbase/pom.xml +++ b/pulsar-io/hbase/pom.xml @@ -73,9 +73,13 @@ log4j log4j + + ch.qos.reload4j + reload4j + org.slf4j - slf4j-log4j12 + * org.apache.avro diff --git a/pulsar-io/http/pom.xml b/pulsar-io/http/pom.xml index 8111b80d4e3f3..50c57c4ea9659 100644 --- a/pulsar-io/http/pom.xml +++ b/pulsar-io/http/pom.xml @@ -68,7 +68,7 @@ com.github.tomakehurst - wiremock-jre8 + wiremock-jre8-standalone ${wiremock.version} test diff --git a/pulsar-io/jdbc/openmldb/pom.xml b/pulsar-io/jdbc/openmldb/pom.xml index 9f548860d0f7c..7b13473f32e69 100644 --- a/pulsar-io/jdbc/openmldb/pom.xml +++ b/pulsar-io/jdbc/openmldb/pom.xml @@ -47,9 +47,13 @@ log4j log4j + + ch.qos.reload4j + reload4j + org.slf4j - slf4j-log4j12 + * diff --git a/pulsar-io/kafka/pom.xml b/pulsar-io/kafka/pom.xml index b78f7e47efb8b..e671ca3380455 100644 --- a/pulsar-io/kafka/pom.xml +++ b/pulsar-io/kafka/pom.xml @@ -32,11 +32,6 @@ - - org.glassfish.jersey.ext - jersey-bean-validation - ${jersey.version} - org.glassfish jakarta.el diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index 088c8018b55a4..293e4f5fc3b5b 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -30,6 +30,7 @@ 9.8.0 + 10.0.24 pulsar-io-solr @@ -40,15 +41,10 @@ org.eclipse.jetty jetty-bom - 10.0.24 + ${jetty10.version} pom import - - org.eclipse.jetty - jetty-server - 10.0.24 - diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index a4f59a66a5b7e..62a2cc32822ef 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -43,8 +43,9 @@ - org.apache.zookeeper - zookeeper + org.apache.pulsar + zookeeper-with-patched-admin + ${project.version} ch.qos.logback diff --git a/pulsar-package-management/bookkeeper-storage/pom.xml b/pulsar-package-management/bookkeeper-storage/pom.xml index 26af35c936417..a0121c139a637 100644 --- a/pulsar-package-management/bookkeeper-storage/pom.xml +++ b/pulsar-package-management/bookkeeper-storage/pom.xml @@ -50,6 +50,12 @@ + + org.apache.pulsar + zookeeper-with-patched-admin + ${project.version} + + org.apache.zookeeper zookeeper @@ -102,6 +108,7 @@ ${project.groupId} testmocks ${project.version} + test diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 42d16e22d487b..d132cc4ce2c38 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -89,18 +89,18 @@ - org.eclipse.jetty - jetty-servlet + org.eclipse.jetty.ee8 + jetty-ee8-servlet - org.eclipse.jetty - jetty-servlets + org.eclipse.jetty.ee8 + jetty-ee8-servlets - org.eclipse.jetty - jetty-proxy + org.eclipse.jetty.ee8 + jetty-ee8-proxy @@ -158,11 +158,6 @@ simpleclient_servlet - - io.prometheus - simpleclient_jetty - - ${project.groupId} pulsar-broker @@ -205,7 +200,7 @@ com.github.tomakehurst - wiremock-jre8 + wiremock-jre8-standalone ${wiremock.version} test @@ -225,6 +220,16 @@ pulsar-broker-common ${project.version} test-jar + + + org.eclipse.jetty.websocket + jetty-websocket-jetty-client + test + + + + io.opentelemetry + opentelemetry-sdk-testing test diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java index 8d0c2c3ae397a..23ebed3420caf 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java @@ -19,15 +19,12 @@ package org.apache.pulsar.proxy.server; import static org.apache.commons.lang3.StringUtils.isBlank; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.net.URI; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; -import java.util.Iterator; import java.util.Set; import java.util.concurrent.Executor; import java.util.concurrent.Executors; @@ -47,16 +44,18 @@ import org.apache.pulsar.common.util.PulsarSslConfiguration; import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData; +import org.eclipse.jetty.client.ContinueProtocolHandler; import org.eclipse.jetty.client.HttpClient; -import org.eclipse.jetty.client.HttpRequest; import org.eclipse.jetty.client.ProtocolHandlers; import org.eclipse.jetty.client.RedirectProtocolHandler; -import org.eclipse.jetty.client.api.ContentProvider; -import org.eclipse.jetty.client.api.Request; -import org.eclipse.jetty.client.http.HttpClientTransportOverHTTP; +import org.eclipse.jetty.client.Request; +import org.eclipse.jetty.client.transport.HttpClientTransportOverHTTP; +import org.eclipse.jetty.ee8.proxy.ProxyServlet; +import org.eclipse.jetty.http.HttpCookieStore; +import org.eclipse.jetty.http.HttpField; import org.eclipse.jetty.http.HttpHeader; -import org.eclipse.jetty.proxy.ProxyServlet; -import org.eclipse.jetty.util.HttpCookieStore; +import org.eclipse.jetty.io.Content; +import org.eclipse.jetty.util.BufferUtil; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.eclipse.jetty.util.thread.QueuedThreadPool; import org.slf4j.Logger; @@ -125,7 +124,7 @@ protected HttpClient createHttpClient() throws ServletException { client.setFollowRedirects(true); // Must not store cookies, otherwise cookies of different clients will mix. - client.setCookieStore(new HttpCookieStore.Empty()); + client.setHttpCookieStore(new HttpCookieStore.Empty()); Executor executor; String value = config.getInitParameter("maxThreads"); @@ -180,6 +179,7 @@ protected HttpClient createHttpClient() throws ServletException { ProtocolHandlers protocolHandlers = client.getProtocolHandlers(); protocolHandlers.clear(); protocolHandlers.put(new RedirectProtocolHandler(client)); + protocolHandlers.put(new ProxyContinueProtocolHandler()); return client; } catch (Exception x) { @@ -187,48 +187,65 @@ protected HttpClient createHttpClient() throws ServletException { } } + class ProxyContinueProtocolHandler extends ContinueProtocolHandler { + + @Override + protected Runnable onContinue(Request request) { + HttpServletRequest clientRequest = + (HttpServletRequest) request.getAttributes().get(CLIENT_REQUEST_ATTRIBUTE); + return AdminProxyHandler.this.onContinue(clientRequest, request); + } + } // This class allows the request body to be replayed, the default implementation // does not - protected class ReplayableProxyContentProvider extends ProxyInputStreamContentProvider { + protected class ReplayableProxyContentProvider extends ProxyInputStreamRequestContent { static final int MIN_REPLAY_BODY_BUFFER_SIZE = 64; - private boolean bodyBufferAvailable = false; + private boolean eofReached = false; private boolean bodyBufferMaxSizeReached = false; - private final ByteArrayOutputStream bodyBuffer; - private final long httpInputMaxReplayBufferSize; + private final ByteBuffer bodyBuffer; protected ReplayableProxyContentProvider(HttpServletRequest request, HttpServletResponse response, Request proxyRequest, InputStream input, int httpInputMaxReplayBufferSize) { super(request, response, proxyRequest, input); - bodyBuffer = new ByteArrayOutputStream( + bodyBuffer = ByteBuffer.allocate( Math.min(Math.max(request.getContentLength(), MIN_REPLAY_BODY_BUFFER_SIZE), httpInputMaxReplayBufferSize)); - this.httpInputMaxReplayBufferSize = httpInputMaxReplayBufferSize; } @Override - public Iterator iterator() { - if (bodyBufferAvailable) { - return Collections.singleton(ByteBuffer.wrap(bodyBuffer.toByteArray())).iterator(); + public Content.Chunk read() { + Content.Chunk chunk; + if (!eofReached) { + chunk = super.read(); + ByteBuffer srcBuffer = chunk.getByteBuffer(); + if (chunk.isLast() && BufferUtil.isTheEmptyBuffer(srcBuffer)) { + eofReached = true; + bodyBuffer.flip(); + } + if (srcBuffer != null && !bodyBufferMaxSizeReached) { + if (bodyBuffer.remaining() >= srcBuffer.remaining()) { + srcBuffer.mark(); + bodyBuffer.put(srcBuffer); + srcBuffer.reset(); + } else { + bodyBufferMaxSizeReached = true; + bodyBuffer.clear(); + } + } } else { - bodyBufferAvailable = true; - return super.iterator(); - } - } - - @Override - protected ByteBuffer onRead(byte[] buffer, int offset, int length) { - if (!bodyBufferMaxSizeReached) { - if (bodyBuffer.size() + length < httpInputMaxReplayBufferSize) { - bodyBuffer.write(buffer, offset, length); + if (!bodyBufferMaxSizeReached) { + chunk = Content.Chunk.from(bodyBuffer.slice(), true); } else { - bodyBufferMaxSizeReached = true; - bodyBufferAvailable = false; - bodyBuffer.reset(); + chunk = super.read(); } } - return super.onRead(buffer, offset, length); + return chunk; + } + + public boolean rewind() { + return true; } } @@ -236,11 +253,12 @@ private static class JettyHttpClient extends HttpClient { private static final int NUMBER_OF_SELECTOR_THREADS = 1; public JettyHttpClient() { - super(new HttpClientTransportOverHTTP(NUMBER_OF_SELECTOR_THREADS), null); + super(new HttpClientTransportOverHTTP(NUMBER_OF_SELECTOR_THREADS)); } - public JettyHttpClient(SslContextFactory sslContextFactory) { - super(new HttpClientTransportOverHTTP(NUMBER_OF_SELECTOR_THREADS), sslContextFactory); + public JettyHttpClient(SslContextFactory.Client sslContextFactory) { + super(new HttpClientTransportOverHTTP(NUMBER_OF_SELECTOR_THREADS)); + setSslContextFactory(sslContextFactory); } /** @@ -248,20 +266,20 @@ public JettyHttpClient(SslContextFactory sslContextFactory) { * from brokers. */ @Override - protected Request copyRequest(HttpRequest oldRequest, URI newURI) { + protected Request copyRequest(Request oldRequest, URI newURI) { String authorization = oldRequest.getHeaders().get(HttpHeader.AUTHORIZATION); Request newRequest = super.copyRequest(oldRequest, newURI); if (authorization != null) { - newRequest.header(HttpHeader.AUTHORIZATION, authorization); + newRequest.headers( + mutable -> mutable.ensureField(new HttpField(HttpHeader.AUTHORIZATION, authorization))); } - return newRequest; } } @Override - protected ContentProvider proxyRequestContent(HttpServletRequest request, + protected Request.Content proxyRequestContent(HttpServletRequest request, HttpServletResponse response, Request proxyRequest) throws IOException { return new ReplayableProxyContentProvider(request, response, proxyRequest, request.getInputStream(), @@ -273,7 +291,7 @@ protected HttpClient newHttpClient() { try { if (config.isTlsEnabledWithBroker()) { try { - SslContextFactory contextFactory = new Client(this.pulsarSslFactory); + SslContextFactory.Client contextFactory = new Client(this.pulsarSslFactory); if (!config.isTlsHostnameVerificationEnabled()) { contextFactory.setEndpointIdentificationAlgorithm(null); } @@ -357,7 +375,7 @@ protected void addProxyHeaders(HttpServletRequest clientRequest, Request proxyRe super.addProxyHeaders(clientRequest, proxyRequest); String user = (String) clientRequest.getAttribute(AuthenticationFilter.AuthenticatedRoleAttributeName); if (user != null) { - proxyRequest.header(ORIGINAL_PRINCIPAL_HEADER, user); + proxyRequest.headers(mutable -> mutable.ensureField(new HttpField(ORIGINAL_PRINCIPAL_HEADER, user))); } } 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 a5504cac100a4..7b66831cec7cd 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 @@ -38,6 +38,7 @@ import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import javax.servlet.Servlet; import lombok.Getter; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.core.util.datetime.FixedDateFormat; @@ -62,9 +63,10 @@ import org.apache.pulsar.websocket.WebSocketProducerServlet; import org.apache.pulsar.websocket.WebSocketReaderServlet; import org.apache.pulsar.websocket.WebSocketService; -import org.eclipse.jetty.proxy.ProxyServlet; -import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.websocket.servlet.WebSocketServlet; +import org.eclipse.jetty.ee8.proxy.ProxyServlet; +import org.eclipse.jetty.ee8.servlet.ServletHolder; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServlet; +import org.eclipse.jetty.ee8.websocket.server.config.JettyWebSocketServletContainerInitializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import picocli.CommandLine; @@ -402,9 +404,37 @@ public static void addWebServerHandlers(WebServer server, service.getProxyAdditionalServlets().getServlets().values(); for (AdditionalServletWithClassLoader servletWithClassLoader : additionalServletCollection) { servletWithClassLoader.loadConfig(config); - server.addServlet(servletWithClassLoader.getBasePath(), servletWithClassLoader.getServletHolder(), - Collections.emptyList(), config.isAuthenticationEnabled()); - log.info("proxy add additional servlet basePath {} ", servletWithClassLoader.getBasePath()); + switch (servletWithClassLoader.getServletType()) { + case JAVAX_SERVLET -> { + Object servletInstance = servletWithClassLoader.getServletInstance(); + if (!(servletInstance instanceof javax.servlet.Servlet)) { + log.error("AdditionalServletWithClassLoader {} has invalid servlet instance type {} which " + + "doesn't match {}. Skipping.", servletWithClassLoader, + servletInstance.getClass().getName(), servletWithClassLoader.getServletType()); + try { + servletWithClassLoader.close(); + } catch (Exception e) { + log.error("Failed to close servlet {}.", servletWithClassLoader, e); + } + continue; + } + ServletHolder additionalServletHolder = + new ServletHolder((Servlet) servletInstance); + server.addServlet(servletWithClassLoader.getBasePath(), additionalServletHolder, + Collections.emptyList(), config.isAuthenticationEnabled()); + log.info("proxy add additional servlet basePath {} ", servletWithClassLoader.getBasePath()); + } + default -> { + log.error("AdditionalServletWithClassLoader {} has unsupported servlet type {}. Skipping.", + servletWithClassLoader, servletWithClassLoader.getServletType()); + try { + servletWithClassLoader.close(); + } catch (Exception e) { + log.error("Failed to close servlet {}.", servletWithClassLoader, e); + } + continue; + } + } } } @@ -418,31 +448,31 @@ public static void addWebServerHandlers(WebServer server, if (webSocketServiceRef != null) { webSocketServiceRef.set(webSocketService); } - final WebSocketServlet producerWebSocketServlet = new WebSocketProducerServlet(webSocketService); - server.addServlet(WebSocketProducerServlet.SERVLET_PATH, - new ServletHolder(producerWebSocketServlet)); - server.addServlet(WebSocketProducerServlet.SERVLET_PATH_V2, - new ServletHolder(producerWebSocketServlet)); - - final WebSocketServlet consumerWebSocketServlet = new WebSocketConsumerServlet(webSocketService); - server.addServlet(WebSocketConsumerServlet.SERVLET_PATH, - new ServletHolder(consumerWebSocketServlet)); - server.addServlet(WebSocketConsumerServlet.SERVLET_PATH_V2, - new ServletHolder(consumerWebSocketServlet)); - - final WebSocketServlet readerWebSocketServlet = new WebSocketReaderServlet(webSocketService); - server.addServlet(WebSocketReaderServlet.SERVLET_PATH, - new ServletHolder(readerWebSocketServlet)); - server.addServlet(WebSocketReaderServlet.SERVLET_PATH_V2, - new ServletHolder(readerWebSocketServlet)); + final JettyWebSocketServlet producerWebSocketServlet = new WebSocketProducerServlet(webSocketService); + addWebSocketServlet(server, WebSocketProducerServlet.SERVLET_PATH, producerWebSocketServlet); + addWebSocketServlet(server, WebSocketProducerServlet.SERVLET_PATH_V2, producerWebSocketServlet); + + final JettyWebSocketServlet consumerWebSocketServlet = new WebSocketConsumerServlet(webSocketService); + addWebSocketServlet(server, WebSocketConsumerServlet.SERVLET_PATH, consumerWebSocketServlet); + addWebSocketServlet(server, WebSocketConsumerServlet.SERVLET_PATH_V2, consumerWebSocketServlet); + + final JettyWebSocketServlet readerWebSocketServlet = new WebSocketReaderServlet(webSocketService); + addWebSocketServlet(server, WebSocketReaderServlet.SERVLET_PATH, readerWebSocketServlet); + addWebSocketServlet(server, WebSocketReaderServlet.SERVLET_PATH_V2, readerWebSocketServlet); final WebSocketMultiTopicConsumerServlet multiTopicConsumerWebSocketServlet = new WebSocketMultiTopicConsumerServlet(webSocketService); - server.addServlet(WebSocketMultiTopicConsumerServlet.SERVLET_PATH, - new ServletHolder(multiTopicConsumerWebSocketServlet)); + addWebSocketServlet(server, WebSocketMultiTopicConsumerServlet.SERVLET_PATH, + multiTopicConsumerWebSocketServlet); } } + private static void addWebSocketServlet(WebServer server, String servletPath, + JettyWebSocketServlet producerWebSocketServlet) { + JettyWebSocketServletContainerInitializer.configure(server.addServlet(servletPath, + new ServletHolder(producerWebSocketServlet)), null); + } + private static ClusterData createClusterData(ProxyConfiguration config) { if (isNotBlank(config.getBrokerServiceURL()) || isNotBlank(config.getBrokerServiceURLTLS())) { return ClusterData.builder() 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 7591b8b54db49..dbc7163ae6546 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 @@ -21,7 +21,6 @@ import static org.apache.pulsar.proxy.server.AdminProxyHandler.INIT_PARAM_REQUEST_BUFFER_SIZE; import com.fasterxml.jackson.core.JsonProcessingException; import io.opentelemetry.api.OpenTelemetry; -import io.prometheus.client.jetty.JettyStatisticsCollector; import java.io.IOException; import java.net.URI; import java.util.ArrayList; @@ -55,15 +54,20 @@ import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.common.util.PulsarSslConfiguration; import org.apache.pulsar.common.util.PulsarSslFactory; +import org.apache.pulsar.jetty.metrics.JettyStatisticsCollector; import org.apache.pulsar.jetty.tls.JettySslContextFactory; import org.apache.pulsar.proxy.stats.PulsarProxyOpenTelemetry; +import org.eclipse.jetty.ee8.servlet.FilterHolder; +import org.eclipse.jetty.ee8.servlet.ServletContextHandler; +import org.eclipse.jetty.ee8.servlet.ServletHolder; +import org.eclipse.jetty.http.UriCompliance; 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.NetworkConnectionLimit; import org.eclipse.jetty.server.ProxyConnectionFactory; import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; @@ -71,13 +75,8 @@ 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; -import org.eclipse.jetty.server.handler.RequestLogHandler; +import org.eclipse.jetty.server.handler.QoSHandler; import org.eclipse.jetty.server.handler.StatisticsHandler; -import org.eclipse.jetty.servlet.FilterHolder; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.QoSFilter; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.glassfish.jersey.server.ResourceConfig; import org.glassfish.jersey.servlet.ServletContainer; @@ -113,7 +112,7 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication config.getHttpServerThreadPoolQueueSize()); this.server = new Server(webServiceExecutor); if (config.getMaxHttpServerConnections() > 0) { - server.addBean(new ConnectionLimit(config.getMaxHttpServerConnections(), server)); + server.addBean(new NetworkConnectionLimit(config.getMaxHttpServerConnections(), server)); } this.authenticationService = authenticationService; this.config = config; @@ -121,6 +120,7 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication List connectors = new ArrayList<>(); HttpConfiguration httpConfig = new HttpConfiguration(); + httpConfig.setUriCompliance(UriCompliance.LEGACY); if (config.isWebServiceTrustXForwardedFor()) { httpConfig.addCustomizer(new ForwardedRequestCustomizer()); } @@ -154,7 +154,7 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication config.getTlsCertRefreshCheckDurationSec(), config.getTlsCertRefreshCheckDurationSec(), TimeUnit.SECONDS); } - SslContextFactory sslCtxFactory = + SslContextFactory.Server sslCtxFactory = JettySslContextFactory.createSslContextFactory(config.getTlsProvider(), sslFactory, config.isTlsRequireTrustedClientCertOnConnect(), config.getWebServiceTlsCiphers(), config.getWebServiceTlsProtocols()); @@ -167,7 +167,9 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication // 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()); + // disable SNI host check for backwards compatibility with Jetty 9.x + boolean sniHostCheck = false; + httpConfig.addCustomizer(new SecureRequestCustomizer(sniHostCheck)); } connectorTls = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connectorTls.setPort(config.getWebServicePortTls().get()); @@ -194,12 +196,6 @@ private static class FilterInitializer { private final FilterHolder authenticationFilterHolder; FilterInitializer(ProxyConfiguration config, AuthenticationService authenticationService) { - if (config.getMaxConcurrentHttpRequests() > 0) { - FilterHolder filterHolder = new FilterHolder(QoSFilter.class); - filterHolder.setInitParameter("maxRequests", String.valueOf(config.getMaxConcurrentHttpRequests())); - filterHolders.add(filterHolder); - } - if (config.isHttpRequestsLimitEnabled()) { filterHolders.add(new FilterHolder( new RateLimitingFilter(config.getHttpRequestsMaxPerSecond(), @@ -224,21 +220,23 @@ public void addFilters(ServletContextHandler context, boolean requiresAuthentica } } - public void addServlet(String basePath, ServletHolder servletHolder) { - addServlet(basePath, servletHolder, Collections.emptyList()); + public ServletContextHandler addServlet(String basePath, ServletHolder servletHolder) { + return addServlet(basePath, servletHolder, Collections.emptyList()); } - public void addServlet(String basePath, ServletHolder servletHolder, List> attributes) { - addServlet(basePath, servletHolder, attributes, true); + public ServletContextHandler addServlet(String basePath, ServletHolder servletHolder, + List> attributes) { + return addServlet(basePath, servletHolder, attributes, true); } - public void addServlet(String basePath, ServletHolder servletHolder, + public ServletContextHandler addServlet(String basePath, ServletHolder servletHolder, List> attributes, boolean requireAuthentication) { - addServlet(basePath, servletHolder, attributes, requireAuthentication, true); + return addServlet(basePath, servletHolder, attributes, requireAuthentication, true); } - private void addServlet(String basePath, ServletHolder servletHolder, - List> attributes, boolean requireAuthentication, boolean checkForExistingPaths) { + private ServletContextHandler addServlet(String basePath, ServletHolder servletHolder, + List> attributes, boolean requireAuthentication, + boolean checkForExistingPaths) { popularServletParams(servletHolder, config); if (checkForExistingPaths) { @@ -261,7 +259,9 @@ private void addServlet(String basePath, ServletHolder servletHolder, filterInitializer.addFilters(context, requireAuthentication); - handlers.add(context); + handlers.add(context.get()); + + return context; } private static void popularServletParams(ServletHolder servletHolder, ProxyConfiguration config) { @@ -318,19 +318,16 @@ public int getExternalServicePort() { } public void start() throws Exception { - RequestLogHandler requestLogHandler = new RequestLogHandler(); 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); + server.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); ContextHandlerCollection contexts = new ContextHandlerCollection(); - contexts.setHandlers(handlers.toArray(new Handler[handlers.size()])); + contexts.setHandlers(handlers); - HandlerCollection handlerCollection = new HandlerCollection(); - handlerCollection.setHandlers(new Handler[] { contexts, new DefaultHandler(), requestLogHandler }); + Handler.Collection handlerCollection = new Handler.Sequence(); + handlerCollection.setHandlers(contexts, new DefaultHandler()); // Metrics handler StatisticsHandler stats = new StatisticsHandler(); @@ -341,7 +338,13 @@ public void start() throws Exception { // Already registered. Eg: in unit tests } - server.setHandler(stats); + Handler serverHandler = stats; + if (config.getMaxConcurrentHttpRequests() > 0) { + QoSHandler qoSHandler = new QoSHandler(serverHandler); + qoSHandler.setMaxRequestCount(config.getMaxConcurrentHttpRequests()); + serverHandler = qoSHandler; + } + server.setHandler(serverHandler); try { server.start(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java index c30cd0a4afbd4..f53dd0fea276b 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java @@ -37,7 +37,7 @@ import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; import org.apache.pulsar.policies.data.loadbalancer.LoadReport; -import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.ee8.servlet.ServletHolder; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java index fdf9242c9f3d8..cb3536968a9aa 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java @@ -22,11 +22,9 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.InputStream; import java.lang.reflect.Field; import java.nio.ByteBuffer; -import java.util.Iterator; import javax.servlet.ServletConfig; import javax.servlet.ServletContext; import javax.servlet.ServletException; @@ -34,7 +32,8 @@ import javax.servlet.http.HttpServletResponse; import org.apache.pulsar.client.api.Authentication; import org.eclipse.jetty.client.HttpClient; -import org.eclipse.jetty.client.api.Request; +import org.eclipse.jetty.client.Request; +import org.eclipse.jetty.io.Content; import org.testng.Assert; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -71,7 +70,7 @@ request, mock(HttpServletResponse.class), mock(Request.class), mock(InputStream. 1024); Field field = replayableProxyContentProvider.getClass().getDeclaredField("bodyBuffer"); field.setAccessible(true); - Assert.assertEquals(((ByteArrayOutputStream) field.get(replayableProxyContentProvider)).size(), 0); + Assert.assertEquals(((ByteBuffer) field.get(replayableProxyContentProvider)).position(), 0); } catch (IllegalArgumentException e) { Assert.fail("IllegalArgumentException should not be thrown"); } @@ -92,13 +91,14 @@ adminProxyHandler.new ReplayableProxyContentProvider(request, mock(HttpServletRe maxRequestBodySize); // when - // content is consumed - Iterator byteBufferIterator = replayableProxyContentProvider.iterator(); int consumedBytes = 0; - while (byteBufferIterator.hasNext()) { - ByteBuffer byteBuffer = byteBufferIterator.next(); - consumedBytes += byteBuffer.limit(); + while (true) { + Content.Chunk chunk = replayableProxyContentProvider.read(); + consumedBytes += chunk.getByteBuffer().remaining(); + if (chunk.isLast()) { + break; + } } // then @@ -129,16 +129,18 @@ adminProxyHandler.new ReplayableProxyContentProvider(request, mock(HttpServletRe for (int i = 0; i < 3; i++) { // when consumeBuffer.clear(); - Iterator byteBufferIterator = replayableProxyContentProvider.iterator(); - while (byteBufferIterator.hasNext()) { - ByteBuffer byteBuffer = byteBufferIterator.next(); - consumeBuffer.put(byteBuffer); + while (true) { + Content.Chunk chunk = replayableProxyContentProvider.read(); + consumeBuffer.put(chunk.getByteBuffer()); + if (chunk.isLast()) { + break; + } } consumeBuffer.flip(); - byte[] consumedBytes = new byte[consumeBuffer.limit()]; + byte[] consumedBytes = new byte[consumeBuffer.remaining()]; consumeBuffer.get(consumedBytes); // then - Assert.assertEquals(consumedBytes, inputBuffer); + Assert.assertEquals(consumedBytes, inputBuffer, "i=" + i); } } } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java index 05f091623a62c..ce1d04f614602 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java @@ -38,7 +38,7 @@ import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; import org.apache.pulsar.policies.data.loadbalancer.LoadReport; -import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.ee8.servlet.ServletHolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java index 89a56794f66ab..3bde6dd431aa8 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java @@ -49,8 +49,7 @@ import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.metadata.impl.ZKMetadataStore; -import org.eclipse.jetty.server.Request; -import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.ee8.nested.Request; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -177,7 +176,9 @@ public void destroy() { AdditionalServlet proxyAdditionalServlet = Mockito.mock(AdditionalServlet.class); Mockito.when(proxyAdditionalServlet.getBasePath()).thenReturn(BASE_PATH); - Mockito.when(proxyAdditionalServlet.getServletHolder()).thenReturn(new ServletHolder(servlet)); + Mockito.when(proxyAdditionalServlet.getServletInstance()).thenReturn(servlet); + Mockito.when(proxyAdditionalServlet.getServletType()).thenReturn( + AdditionalServlet.AdditionalServletType.JAVAX_SERVLET); AdditionalServlets proxyAdditionalServlets = Mockito.mock(AdditionalServlets.class); Map map = new HashMap<>(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyIsAHttpProxyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyIsAHttpProxyTest.java index 5db12000d57b5..c869e15cef2a3 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyIsAHttpProxyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyIsAHttpProxyTest.java @@ -42,16 +42,19 @@ import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.eclipse.jetty.client.HttpClient; -import org.eclipse.jetty.client.api.Result; +import org.eclipse.jetty.client.Result; +import org.eclipse.jetty.ee8.nested.AbstractHandler; +import org.eclipse.jetty.ee8.nested.ContextHandler; +import org.eclipse.jetty.ee8.nested.Request; +import org.eclipse.jetty.ee8.servlet.ServletContextHandler; +import org.eclipse.jetty.ee8.servlet.ServletHolder; +import org.eclipse.jetty.http.UriCompliance; import org.eclipse.jetty.server.Connector; +import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; -import org.eclipse.jetty.server.Request; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; -import org.eclipse.jetty.server.handler.AbstractHandler; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.util.ProcessorUtils; import org.glassfish.jersey.client.ClientConfig; import org.glassfish.jersey.logging.LoggingFeature; @@ -91,6 +94,7 @@ protected void setup() throws Exception { backingServer3 = new Server(); HttpConfiguration httpConfig = new HttpConfiguration(); + httpConfig.setUriCompliance(UriCompliance.LEGACY); httpConfig.setRequestHeaderSize(20000); ServerConnector connector = new ServerConnector(backingServer3, new HttpConnectionFactory(httpConfig)); connector.setPort(0); @@ -99,11 +103,11 @@ protected void setup() throws Exception { backingServer3.start(); } - private static AbstractHandler newHandler(String text) { - return new AbstractHandler() { + private static Handler newHandler(String text) { + AbstractHandler handler = new AbstractHandler() { @Override - public void handle(String target, Request baseRequest, - HttpServletRequest request, HttpServletResponse response) + public void handle(String target, Request baseRequest, HttpServletRequest request, + HttpServletResponse response) throws IOException, ServletException { response.setContentType("text/plain;charset=utf-8"); response.setStatus(HttpServletResponse.SC_OK); @@ -113,6 +117,7 @@ public void handle(String target, Request baseRequest, uri.substring(0, uri.length() > 1024 ? 1024 : uri.length()))); } }; + return new ContextHandler("/", handler).get(); } private static ServletContextHandler newStreamingHandler(LinkedBlockingQueue dataQueue) { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMultiBrokerBaseTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMultiBrokerBaseTest.java new file mode 100644 index 0000000000000..6b9676008442e --- /dev/null +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMultiBrokerBaseTest.java @@ -0,0 +1,128 @@ +/* + * 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.apache.pulsar.proxy.server.ProxyServiceStarter.addWebServerHandlers; +import static org.mockito.Mockito.doReturn; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.MultiBrokerBaseTest; +import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; +import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.apache.pulsar.websocket.WebSocketService; + +@Slf4j +public abstract class ProxyMultiBrokerBaseTest extends MultiBrokerBaseTest { + private Authentication proxyClientAuthentication; + private ProxyService proxyService; + private AuthenticationService authenticationService; + private WebServer server; + private WebSocketService webSocketService; + protected PulsarAdmin proxiedAdmin; + + @Override + public int numberOfAdditionalBrokers() { + return 2; + } + + private ProxyConfiguration initializeProxyConfig() { + var proxyConfig = new ProxyConfiguration(); + proxyConfig.setNumIOThreads(8); + proxyConfig.setServicePort(Optional.of(0)); + proxyConfig.setWebServicePort(Optional.of(0)); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); + proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); + return proxyConfig; + } + + @Override + protected void additionalSetup() throws Exception { + var proxyConfig = initializeProxyConfig(); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + authenticationService = new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)); + proxyService = BrokerTestUtil.spyWithoutRecordingInvocations(new ProxyService(proxyConfig, + authenticationService, + proxyClientAuthentication)); + doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); + doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) + .createConfigurationMetadataStore(); + proxyService.start(); + server = new WebServer(proxyConfig, authenticationService); + + AtomicReference webSocketServiceRef = new AtomicReference<>(); + addWebServerHandlers(server, proxyConfig, proxyService, proxyService.getDiscoveryProvider(), + webSocketServiceRef, + proxyClientAuthentication); + webSocketService = webSocketServiceRef.get(); + + // start web-service + server.start(); + + proxiedAdmin = PulsarAdmin.builder().serviceHttpUrl(getProxyHttpServiceUrl()).build(); + proxiedAdmin = customizeProxiedAdmin(proxiedAdmin); + } + + protected PulsarAdmin customizeProxiedAdmin(PulsarAdmin proxiedAdmin) { + // disable redirects by default so that AdminProxyHandler behavior can be verified + ((PulsarAdminImpl) proxiedAdmin).getAsyncHttpConnector().setFollowRedirects(false); + ((PulsarAdminImpl) proxiedAdmin).getAsyncConnectorProvider().setFollowRedirects(false); + return proxiedAdmin; + } + + @Override + protected void additionalCleanup() throws Exception { + if (proxiedAdmin != null) { + proxiedAdmin.close(); + } + if (proxyService != null) { + proxyService.close(); + } + if (server != null) { + server.stop(); + } + if (webSocketService != null) { + webSocketService.close(); + } + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } + if (authenticationService != null) { + authenticationService.close(); + } + } + + public String getProxyHttpServiceUrl() { + return server.getServiceUri().toString(); + } + + public String getProxyServiceUrl() { + return proxyService.getServiceUrl(); + } +} 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 index 50ae6e627e820..4cae182924c5f 100644 --- 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 @@ -27,9 +27,10 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.assertj.core.api.ThrowingConsumer; import org.awaitility.Awaitility; +import org.eclipse.jetty.client.ContentResponse; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.client.ProxyProtocolClientConnectionFactory.V2; -import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.http.HttpField; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -66,7 +67,8 @@ protected void setup() throws Exception { serviceStarter.start(); webServiceUrl = "http://localhost:" + serviceStarter.getServer().getListenPortHTTP().get(); webServiceUrlTls = "https://localhost:" + serviceStarter.getServer().getListenPortHTTPS().get(); - httpClient = new HttpClient(new SslContextFactory(true)); + httpClient = new HttpClient(); + httpClient.setSslContextFactory(new SslContextFactory.Client(true)); httpClient.start(); } @@ -99,7 +101,7 @@ public void testClientIPIsPickedFromXForwardedForHeaderAndLogged(boolean tlsEnab performLoggingTest(consoleCaptor -> { // Send a GET request to the metrics URL ContentResponse response = httpClient.newRequest(url) - .header("X-Forwarded-For", "11.22.33.44") + .headers(hdrs -> hdrs.ensureField(new HttpField("X-Forwarded-For", "11.22.33.44"))) .send(); // Validate the response diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRedirectTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRedirectTest.java new file mode 100644 index 0000000000000..0b57e407580f5 --- /dev/null +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRedirectTest.java @@ -0,0 +1,84 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; +import java.util.Map; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; +import org.eclipse.jetty.client.ContentResponse; +import org.eclipse.jetty.client.HttpClient; +import org.testng.annotations.Test; + +@Slf4j +public class ProxyRedirectTest extends ProxyMultiBrokerBaseTest { + @Test + public void testProxyHandlesRedirects() throws Exception { + var namespaceName = NamespaceName.get("public", BrokerTestUtil.newUniqueName("redirecttest")); + admin.namespaces().createNamespace(namespaceName.toString(), 16); + + @Cleanup("stop") + HttpClient client = new HttpClient(); + client.setFollowRedirects(false); + client.start(); + + for (int i = 0; i < 100; i++) { + var topicName = TopicName.get(TopicDomain.persistent.toString(), namespaceName, "topic-" + i); + ContentResponse response = + client.newRequest(getProxyHttpServiceUrl() + "/lookup/v2/topic/" + topicName.getLookupName()) + .followRedirects(false).send(); + assertThat(response.getStatus()).isEqualTo(200); + } + } + + @Test + public void testProxyHandlesReplayingContent() throws Exception { + var namespaceName = NamespaceName.get("public", BrokerTestUtil.newUniqueName("replaytest")); + admin.namespaces().createNamespace(namespaceName.toString(), 16); + + @Cleanup("stop") + HttpClient client = new HttpClient(); + client.setFollowRedirects(false); + client.start(); + + for (int i = 0; i < 100; i++) { + var topicName = TopicName.get(TopicDomain.persistent.toString(), namespaceName, "topic-" + i); + proxiedAdmin.topics().createNonPartitionedTopic(topicName.toString(), Map.of("index", String.valueOf(i))); + DelayedDeliveryPolicies delayedDeliveryPolicies = DelayedDeliveryPolicies.builder() + .active(true) + .tickTime(1234) + .maxDeliveryDelayInMillis(600000) + .build(); + proxiedAdmin.topicPolicies().setDelayedDeliveryPolicy(topicName.toString(), delayedDeliveryPolicies); + + DelayedDeliveryPolicies updatedPolicy = + proxiedAdmin.topicPolicies().getDelayedDeliveryPolicy(topicName.toString()); + + assertThat(updatedPolicy).isEqualTo(delayedDeliveryPolicies); + + Map topicProperties = proxiedAdmin.topics().getProperties(topicName.toString()); + assertThat(topicProperties).containsEntry("index", String.valueOf(i)); + } + } +} 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 7703d08053d05..a6ca8ee5bffc9 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 @@ -34,7 +34,7 @@ import java.util.Optional; import java.util.Properties; import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.Future; +import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; import lombok.Cleanup; import lombok.SneakyThrows; @@ -47,9 +47,9 @@ import org.apache.pulsar.websocket.data.ProducerMessage; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.util.BufferUtil; -import org.eclipse.jetty.websocket.api.Session; -import org.eclipse.jetty.websocket.api.WebSocketAdapter; -import org.eclipse.jetty.websocket.api.WebSocketPingPongListener; +import org.eclipse.jetty.websocket.api.Callback; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketMessage; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketPong; import org.eclipse.jetty.websocket.api.annotations.WebSocket; import org.eclipse.jetty.websocket.client.WebSocketClient; import org.testng.annotations.AfterClass; @@ -142,7 +142,8 @@ public void testProduceAndConsumeMessageWithWebsocket() throws Exception { producerWebSocketClient.start(); MyWebSocket producerSocket = new MyWebSocket(); String produceUri = computeWsBasePath() + "/producer/persistent/sample/test/local/websocket-topic"; - Future producerSession = producerWebSocketClient.connect(producerSocket, URI.create(produceUri)); + CompletableFuture + producerSession = producerWebSocketClient.connect(producerSocket, URI.create(produceUri)); ProducerMessage produceRequest = new ProducerMessage(); produceRequest.setContext("context"); @@ -155,43 +156,27 @@ public void testProduceAndConsumeMessageWithWebsocket() throws Exception { consumerWebSocketClient.start(); MyWebSocket consumerSocket = new MyWebSocket(); String consumeUri = computeWsBasePath() + "/consumer/persistent/sample/test/local/websocket-topic/my-sub"; - Future consumerSession = consumerWebSocketClient.connect(consumerSocket, URI.create(consumeUri)); - consumerSession.get().getRemote().sendPing(ByteBuffer.wrap("ping".getBytes())); - producerSession.get().getRemote().sendString(ObjectMapperFactory.getMapper().writer() - .writeValueAsString(produceRequest)); + CompletableFuture + consumerSession = consumerWebSocketClient.connect(consumerSocket, URI.create(consumeUri)); + consumerSession.get().sendPing(ByteBuffer.wrap("ping".getBytes()), Callback.NOOP); + producerSession.get() + .sendText(ObjectMapperFactory.getMapper().writer().writeValueAsString(produceRequest), Callback.NOOP); assertTrue(consumerSocket.getResponse().contains("ping")); - ProducerMessage message = ObjectMapperFactory.getMapper().reader().readValue(consumerSocket.getResponse(), - ProducerMessage.class); + ProducerMessage message = + ObjectMapperFactory.getMapper().reader().readValue(consumerSocket.getResponse(), ProducerMessage.class); assertEquals(new String(Base64.getDecoder().decode(message.getPayload())), "my payload"); } @WebSocket - public static class MyWebSocket extends WebSocketAdapter implements WebSocketPingPongListener { - + public static class MyWebSocket { ArrayBlockingQueue incomingMessages = new ArrayBlockingQueue<>(10); - @Override + @OnWebSocketMessage public void onWebSocketText(String message) { incomingMessages.add(message); } - @Override - public void onWebSocketClose(int i, String s) { - } - - @Override - public void onWebSocketConnect(Session session) { - } - - @Override - public void onWebSocketError(Throwable throwable) { - } - - @Override - public void onWebSocketPing(ByteBuffer payload) { - } - - @Override + @OnWebSocketPong public void onWebSocketPong(ByteBuffer payload) { incomingMessages.add(BufferUtil.toDetailString(payload)); } 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 983be6449b5b3..d09301514fe55 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 @@ -26,7 +26,7 @@ import java.util.Base64; import java.util.Optional; import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.Future; +import java.util.concurrent.CompletableFuture; import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.Producer; @@ -35,9 +35,14 @@ import org.apache.pulsar.websocket.data.ProducerMessage; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.util.BufferUtil; +import org.eclipse.jetty.websocket.api.Callback; import org.eclipse.jetty.websocket.api.Session; -import org.eclipse.jetty.websocket.api.WebSocketAdapter; -import org.eclipse.jetty.websocket.api.WebSocketPingPongListener; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketClose; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketError; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketMessage; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketOpen; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketPing; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketPong; import org.eclipse.jetty.websocket.api.annotations.WebSocket; import org.eclipse.jetty.websocket.client.WebSocketClient; import org.testng.annotations.AfterClass; @@ -116,7 +121,8 @@ public void testProduceAndConsumeMessageWithWebsocket() throws Exception { producerWebSocketClient.start(); MyWebSocket producerSocket = new MyWebSocket(); String produceUri = "ws://localhost:" + webPort + "/ws/producer/persistent/sample/test/local/websocket-topic"; - Future producerSession = producerWebSocketClient.connect(producerSocket, URI.create(produceUri)); + CompletableFuture + producerSession = producerWebSocketClient.connect(producerSocket, URI.create(produceUri)); ProducerMessage produceRequest = new ProducerMessage(); produceRequest.setContext("context"); @@ -130,43 +136,44 @@ public void testProduceAndConsumeMessageWithWebsocket() throws Exception { MyWebSocket consumerSocket = new MyWebSocket(); String consumeUri = "ws://localhost:" + webPort + "/ws/consumer/persistent/sample/test/local/websocket-topic/my-sub"; - Future consumerSession = consumerWebSocketClient.connect(consumerSocket, URI.create(consumeUri)); - consumerSession.get().getRemote().sendPing(ByteBuffer.wrap("ping".getBytes())); - producerSession.get().getRemote().sendString(ObjectMapperFactory.getMapper().writer() - .writeValueAsString(produceRequest)); + CompletableFuture + consumerSession = consumerWebSocketClient.connect(consumerSocket, URI.create(consumeUri)); + consumerSession.get().sendPing(ByteBuffer.wrap("ping".getBytes()), Callback.NOOP); + producerSession.get() + .sendText(ObjectMapperFactory.getMapper().writer().writeValueAsString(produceRequest), Callback.NOOP); assertTrue(consumerSocket.getResponse().contains("ping")); - ProducerMessage message = ObjectMapperFactory.getMapper().reader().readValue(consumerSocket.getResponse(), - ProducerMessage.class); + ProducerMessage message = + ObjectMapperFactory.getMapper().reader().readValue(consumerSocket.getResponse(), ProducerMessage.class); assertEquals(new String(Base64.getDecoder().decode(message.getPayload())), "my payload"); } @WebSocket - public static class MyWebSocket extends WebSocketAdapter implements WebSocketPingPongListener { + public static class MyWebSocket { ArrayBlockingQueue incomingMessages = new ArrayBlockingQueue<>(10); - @Override + @OnWebSocketMessage public void onWebSocketText(String message) { incomingMessages.add(message); } - @Override + @OnWebSocketClose public void onWebSocketClose(int i, String s) { } - @Override + @OnWebSocketOpen public void onWebSocketConnect(Session session) { } - @Override + @OnWebSocketError public void onWebSocketError(Throwable throwable) { } - @Override + @OnWebSocketPing public void onWebSocketPing(ByteBuffer payload) { } - @Override + @OnWebSocketPong public void onWebSocketPong(ByteBuffer payload) { incomingMessages.add(BufferUtil.toDetailString(payload)); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java index 250e097ea187f..f7bb47cc3ad37 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java @@ -38,7 +38,7 @@ import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; import org.apache.pulsar.policies.data.loadbalancer.LoadReport; -import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.ee8.servlet.ServletHolder; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java index c41f88e616def..8003047a8eed3 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java @@ -37,7 +37,7 @@ import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.configuration.VipStatus; import org.apache.pulsar.metadata.impl.ZKMetadataStore; -import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.ee8.servlet.ServletHolder; import org.glassfish.jersey.client.ClientConfig; import org.glassfish.jersey.logging.LoggingFeature; import org.testng.Assert; diff --git a/pulsar-testclient/pom.xml b/pulsar-testclient/pom.xml index ea9a22510c8bb..a27e65e99c8b2 100644 --- a/pulsar-testclient/pom.xml +++ b/pulsar-testclient/pom.xml @@ -90,6 +90,15 @@ ${project.version} + + org.eclipse.jetty.websocket + jetty-websocket-jetty-api + + + org.eclipse.jetty.websocket + jetty-websocket-jetty-client + + info.picocli picocli @@ -131,7 +140,7 @@ com.github.tomakehurst - wiremock-jre8 + wiremock-jre8-standalone ${wiremock.version} test diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java b/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java index 4053942c05050..19b7792f8226c 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java @@ -54,6 +54,7 @@ import org.apache.pulsar.testclient.PerfClientUtils; import org.apache.pulsar.testclient.PositiveNumberParameterConvert; import org.apache.pulsar.testclient.utils.PaddingDecimalFormat; +import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.eclipse.jetty.websocket.client.ClientUpgradeRequest; import org.eclipse.jetty.websocket.client.WebSocketClient; @@ -239,12 +240,14 @@ public void runPerformanceTest() throws InterruptedException, IOException { String restPath = TopicName.get(topicName).getRestPath(); String produceBaseEndPoint = TopicName.get(topicName).isV2() ? this.proxyURL + "ws/v2/producer/" + restPath : this.proxyURL + "ws/producer/" + restPath; + HttpClient httpClient = new HttpClient(); + httpClient.setSslContextFactory(new SslContextFactory.Client(true)); for (int i = 0; i < this.numTopics; i++) { String topic = this.numTopics > 1 ? produceBaseEndPoint + i : produceBaseEndPoint; URI produceUri = URI.create(topic); - WebSocketClient produceClient = new WebSocketClient(new SslContextFactory(true)); - ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); + WebSocketClient produceClient = new WebSocketClient(httpClient); + ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(produceUri); if (StringUtils.isNotBlank(this.authPluginClassName) && StringUtils.isNotBlank(this.authParams)) { try { @@ -269,7 +272,7 @@ public void runPerformanceTest() throws InterruptedException, IOException { try { produceClient.start(); - produceClient.connect(produceSocket, produceUri, produceRequest); + produceClient.connect(produceSocket, produceRequest); } catch (IOException e1) { log.error("Fail in connecting: [{}]", e1.getMessage()); return; diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/SimpleTestProducerSocket.java b/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/SimpleTestProducerSocket.java index 0dadf33378704..499cbc951cb0c 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/SimpleTestProducerSocket.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/SimpleTestProducerSocket.java @@ -24,21 +24,22 @@ import com.google.gson.JsonObject; import com.google.gson.JsonParseException; import java.io.IOException; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import org.HdrHistogram.Recorder; -import org.eclipse.jetty.websocket.api.RemoteEndpoint; +import org.eclipse.jetty.websocket.api.Callback; import org.eclipse.jetty.websocket.api.Session; import org.eclipse.jetty.websocket.api.annotations.OnWebSocketClose; -import org.eclipse.jetty.websocket.api.annotations.OnWebSocketConnect; import org.eclipse.jetty.websocket.api.annotations.OnWebSocketMessage; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketOpen; import org.eclipse.jetty.websocket.api.annotations.WebSocket; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@WebSocket(maxTextMessageSize = 64 * 1024) +@WebSocket public class SimpleTestProducerSocket { public static Recorder recorder = new Recorder(TimeUnit.SECONDS.toMillis(120000), 5); @@ -62,7 +63,7 @@ public void onClose(int statusCode, String reason) { this.closeLatch.countDown(); } - @OnWebSocketConnect + @OnWebSocketOpen public void onConnect(Session session) throws InterruptedException, IOException, JsonParseException { log.info("Got conneceted to the proxy"); this.session = session; @@ -80,10 +81,6 @@ public void onMessage(String msg) throws JsonParseException { recorder.recordValue(NANOSECONDS.toMicros(latencyNs)); } - public RemoteEndpoint getRemote() { - return this.session.getRemote(); - } - public Session getSession() { return this.session; } @@ -93,9 +90,12 @@ public void sendMsg(String context, byte[] payloadData) String message = getEncoder().encodeToString(payloadData); String timeStamp = "{\"payload\": \"" + message + "\",\"context\": \"" + context + "\"}"; String sampleMsg = new Gson().fromJson(timeStamp, JsonObject.class).toString(); - if (this.session != null && this.session.isOpen() && this.session.getRemote() != null) { + if (this.session != null && this.session.isOpen()) { startTimeMap.put(context, System.nanoTime()); - this.session.getRemote().sendStringByFuture(sampleMsg).get(); + CompletableFuture sendFuture = new CompletableFuture<>(); + Callback callback = Callback.from(() -> sendFuture.complete(null), sendFuture::completeExceptionally); + this.session.sendText(sampleMsg, callback); + sendFuture.get(); } else { log.error("Session is already closed"); } diff --git a/pulsar-websocket/pom.xml b/pulsar-websocket/pom.xml index 1a837958b1af8..d9219a27c2101 100644 --- a/pulsar-websocket/pom.xml +++ b/pulsar-websocket/pom.xml @@ -19,7 +19,7 @@ --> + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 @@ -105,29 +105,25 @@ jackson-jaxrs-json-provider - + org.eclipse.jetty.websocket - websocket-api - ${jetty.version} + jetty-websocket-jetty-api - + - org.eclipse.jetty.websocket - websocket-server - ${jetty.version} + org.eclipse.jetty.ee8.websocket + jetty-ee8-websocket-jetty-server - + org.eclipse.jetty.websocket - javax-websocket-client-impl - ${jetty.version} + jetty-websocket-jetty-client - org.eclipse.jetty - jetty-servlets - ${jetty.version} + org.eclipse.jetty.ee8 + jetty-ee8-servlets org.hdrhistogram diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/AbstractWebSocketHandler.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/AbstractWebSocketHandler.java index b6ed27c87b6ba..ecb78d17c7033 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/AbstractWebSocketHandler.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/AbstractWebSocketHandler.java @@ -34,6 +34,7 @@ import java.util.concurrent.TimeUnit; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import lombok.Getter; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; @@ -56,9 +57,9 @@ import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.websocket.data.ConsumerCommand; -import org.eclipse.jetty.websocket.api.Session; -import org.eclipse.jetty.websocket.api.WebSocketAdapter; -import org.eclipse.jetty.websocket.servlet.ServletUpgradeResponse; +import org.eclipse.jetty.ee8.websocket.api.Session; +import org.eclipse.jetty.ee8.websocket.api.WebSocketAdapter; +import org.eclipse.jetty.ee8.websocket.server.JettyServerUpgradeResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,10 +75,12 @@ public abstract class AbstractWebSocketHandler extends WebSocketAdapter implemen ObjectMapperFactory.getMapper().reader().forType(ConsumerCommand.class); private ScheduledFuture pingFuture; + @Getter + protected boolean allowConnect = false; public AbstractWebSocketHandler(WebSocketService service, HttpServletRequest request, - ServletUpgradeResponse response) { + JettyServerUpgradeResponse response) { this.service = service; this.request = new WebSocketHttpServletRequestWrapper(request); @@ -88,7 +91,7 @@ public AbstractWebSocketHandler(WebSocketService service, extractTopicName(request); } - protected boolean checkAuth(ServletUpgradeResponse response) { + protected boolean checkAuth(JettyServerUpgradeResponse response) { String authRole = ""; String authMethodName = request.getHeader(PULSAR_AUTH_METHOD_NAME); AuthenticationState authenticationState = null; @@ -190,6 +193,10 @@ private void closePingFuture() { @Override public void onWebSocketConnect(Session session) { + if (!allowConnect) { + throw new IllegalStateException("allowConnect is false. " + + "JettyWebSocketCreator should have returned null to prevent connecting."); + } super.onWebSocketConnect(session); int webSocketPingDurationSeconds = service.getConfig().getWebSocketPingDurationSeconds(); if (webSocketPingDurationSeconds > 0) { diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ConsumerHandler.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ConsumerHandler.java index b93c4b215108e..8566ab5b5117f 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ConsumerHandler.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ConsumerHandler.java @@ -54,9 +54,9 @@ import org.apache.pulsar.websocket.data.ConsumerCommand; import org.apache.pulsar.websocket.data.ConsumerMessage; import org.apache.pulsar.websocket.data.EndOfTopicResponse; -import org.eclipse.jetty.websocket.api.Session; -import org.eclipse.jetty.websocket.api.WriteCallback; -import org.eclipse.jetty.websocket.servlet.ServletUpgradeResponse; +import org.eclipse.jetty.ee8.websocket.api.Session; +import org.eclipse.jetty.ee8.websocket.api.WriteCallback; +import org.eclipse.jetty.ee8.websocket.server.JettyServerUpgradeResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -99,7 +99,7 @@ public class ConsumerHandler extends AbstractWebSocketHandler { .expireAfterWrite(1, TimeUnit.HOURS) .build(); - public ConsumerHandler(WebSocketService service, HttpServletRequest request, ServletUpgradeResponse response) { + public ConsumerHandler(WebSocketService service, HttpServletRequest request, JettyServerUpgradeResponse response) { super(service, request, response); ConsumerBuilderImpl builder; @@ -137,6 +137,7 @@ public ConsumerHandler(WebSocketService service, HttpServletRequest request, Ser log.warn("[{}:{}] Failed to add consumer handler for topic {}", request.getRemoteAddr(), request.getRemotePort(), topic); } + allowConnect = true; } catch (Exception e) { log.warn("[{}:{}] Failed in creating subscription {} on topic {}", request.getRemoteAddr(), request.getRemotePort(), subscription, topic, e); @@ -187,7 +188,7 @@ private void receiveMessage() { public void writeFailed(Throwable th) { log.warn("[{}/{}] Failed to deliver msg to {} {}", consumer.getTopic(), subscription, - getRemote().getInetSocketAddress().toString(), th.getMessage()); + getRemote().getRemoteAddress().toString(), th.getMessage()); pendingMessages.decrementAndGet(); // schedule receive as one of the delivery failed service.getExecutor().execute(() -> receiveMessage()); @@ -198,7 +199,7 @@ public void writeSuccess() { if (log.isDebugEnabled()) { log.debug("[{}/{}] message is delivered successfully to {} ", consumer.getTopic(), - subscription, getRemote().getInetSocketAddress().toString()); + subscription, getRemote().getRemoteAddress().toString()); } updateDeliverMsgStat(msgSize); } @@ -218,7 +219,7 @@ public void writeSuccess() { subscription); } else { log.warn("[{}/{}] Error occurred while consumer handler was delivering msg to {}: {}", - consumer.getTopic(), subscription, getRemote().getInetSocketAddress().toString(), + consumer.getTopic(), subscription, getRemote().getRemoteAddress().toString(), exception.getMessage()); } return null; @@ -260,7 +261,7 @@ public void onWebSocketText(String message) { private void handleEndOfTopic() { if (log.isDebugEnabled()) { log.debug("[{}/{}] Received check reach the end of topic request from {} ", consumer.getTopic(), - subscription, getRemote().getInetSocketAddress().toString()); + subscription, getRemote().getRemoteAddress().toString()); } try { String msg = objectWriter().writeValueAsString( @@ -270,14 +271,14 @@ private void handleEndOfTopic() { @Override public void writeFailed(Throwable th) { log.warn("[{}/{}] Failed to send end of topic msg to {} due to {}", consumer.getTopic(), - subscription, getRemote().getInetSocketAddress().toString(), th.getMessage()); + subscription, getRemote().getRemoteAddress().toString(), th.getMessage()); } @Override public void writeSuccess() { if (log.isDebugEnabled()) { log.debug("[{}/{}] End of topic message is delivered successfully to {} ", - consumer.getTopic(), subscription, getRemote().getInetSocketAddress().toString()); + consumer.getTopic(), subscription, getRemote().getRemoteAddress().toString()); } } }); @@ -291,7 +292,7 @@ public void writeSuccess() { private void handleUnsubscribe(ConsumerCommand command) throws PulsarClientException { if (log.isDebugEnabled()) { log.debug("[{}/{}] Received unsubscribe request from {} ", consumer.getTopic(), - subscription, getRemote().getInetSocketAddress().toString()); + subscription, getRemote().getRemoteAddress().toString()); } consumer.unsubscribe(); } @@ -311,7 +312,7 @@ private void handleAck(ConsumerCommand command) throws IOException { MessageId msgId = MessageId.fromByteArray(Base64.getDecoder().decode(command.messageId)); if (log.isDebugEnabled()) { log.debug("[{}/{}] Received ack request of message {} from {} ", consumer.getTopic(), - subscription, msgId, getRemote().getInetSocketAddress().toString()); + subscription, msgId, getRemote().getRemoteAddress().toString()); } MessageId originalMsgId = messageIdCache.asMap().remove(command.messageId); @@ -329,7 +330,7 @@ private void handleNack(ConsumerCommand command) throws IOException { topic.toString()); if (log.isDebugEnabled()) { log.debug("[{}/{}] Received negative ack request of message {} from {} ", consumer.getTopic(), - subscription, msgId, getRemote().getInetSocketAddress().toString()); + subscription, msgId, getRemote().getRemoteAddress().toString()); } MessageId originalMsgId = messageIdCache.asMap().remove(command.messageId); @@ -344,7 +345,7 @@ private void handleNack(ConsumerCommand command) throws IOException { private void handlePermit(ConsumerCommand command) throws IOException { if (log.isDebugEnabled()) { log.debug("[{}/{}] Received {} permits request from {} ", consumer.getTopic(), - subscription, command.permitMessages, getRemote().getInetSocketAddress().toString()); + subscription, command.permitMessages, getRemote().getRemoteAddress().toString()); } if (command.permitMessages == null) { throw new IOException("Missing required permitMessages field for 'permit' command"); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/MultiTopicConsumerHandler.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/MultiTopicConsumerHandler.java index 7fbe257d2e249..f70ab554aed1e 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/MultiTopicConsumerHandler.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/MultiTopicConsumerHandler.java @@ -32,7 +32,7 @@ import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.FutureUtil; -import org.eclipse.jetty.websocket.servlet.ServletUpgradeResponse; +import org.eclipse.jetty.ee8.websocket.server.JettyServerUpgradeResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,7 +42,7 @@ public class MultiTopicConsumerHandler extends ConsumerHandler { public MultiTopicConsumerHandler(WebSocketService service, HttpServletRequest request, - ServletUpgradeResponse response) { + JettyServerUpgradeResponse response) { super(service, request, response); } diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ProducerHandler.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ProducerHandler.java index 3c0f42935e6bb..e4fb7d332f7cc 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ProducerHandler.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ProducerHandler.java @@ -62,8 +62,8 @@ import org.apache.pulsar.websocket.data.ProducerMessage; import org.apache.pulsar.websocket.service.WSSDummyMessageCryptoImpl; import org.apache.pulsar.websocket.stats.StatsBuckets; -import org.eclipse.jetty.websocket.api.WriteCallback; -import org.eclipse.jetty.websocket.servlet.ServletUpgradeResponse; +import org.eclipse.jetty.ee8.websocket.api.WriteCallback; +import org.eclipse.jetty.ee8.websocket.server.JettyServerUpgradeResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,7 +94,7 @@ public class ProducerHandler extends AbstractWebSocketHandler { private final ObjectReader producerMessageReader = ObjectMapperFactory.getMapper().reader().forType(ProducerMessage.class); - public ProducerHandler(WebSocketService service, HttpServletRequest request, ServletUpgradeResponse response) { + public ProducerHandler(WebSocketService service, HttpServletRequest request, JettyServerUpgradeResponse response) { super(service, request, response); this.numMsgsSent = new LongAdder(); this.numBytesSent = new LongAdder(); @@ -118,6 +118,7 @@ public ProducerHandler(WebSocketService service, HttpServletRequest request, Ser log.warn("[{}:{}] Failed to add producer handler for topic {}", request.getRemoteAddr(), request.getRemotePort(), topic); } + allowConnect = true; } catch (Exception e) { int errorCode = getErrorCode(e); boolean isKnownError = errorCode != HttpServletResponse.SC_INTERNAL_SERVER_ERROR; @@ -159,7 +160,7 @@ public void close() throws IOException { public void onWebSocketText(String message) { if (log.isDebugEnabled()) { log.debug("[{}] Received new message from producer {} ", producer.getTopic(), - getRemote().getInetSocketAddress().toString()); + getRemote().getRemoteAddress().toString()); } ProducerMessage sendRequest; byte[] rawPayload = null; @@ -251,7 +252,7 @@ public void onWebSocketText(String message) { builder.sendAsync().thenAccept(msgId -> { if (log.isDebugEnabled()) { log.debug("[{}] Success fully write the message to broker with returned message ID {} from producer {}", - producer.getTopic(), msgId, getRemote().getInetSocketAddress().toString()); + producer.getTopic(), msgId, getRemote().getRemoteAddress().toString()); } updateSentMsgStats(msgSize, TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - now)); if (isConnected()) { @@ -260,7 +261,7 @@ public void onWebSocketText(String message) { } }).exceptionally(exception -> { log.warn("[{}] Error occurred while producer handler was sending msg from {}", producer.getTopic(), - getRemote().getInetSocketAddress().toString(), exception); + getRemote().getRemoteAddress().toString(), exception); numMsgsFailed.increment(); sendAckResponse( new ProducerAck(UnknownError, exception.getMessage(), null, sendRequest.context)); @@ -327,7 +328,7 @@ public void writeFailed(Throwable th) { public void writeSuccess() { if (log.isDebugEnabled()) { log.debug("[{}] Ack was sent successfully to {}", producer.getTopic(), - getRemote().getInetSocketAddress().toString()); + getRemote().getRemoteAddress().toString()); } } }); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ReaderHandler.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ReaderHandler.java index 2f985b2076da2..7cee6005f05d6 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ReaderHandler.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ReaderHandler.java @@ -46,9 +46,9 @@ import org.apache.pulsar.websocket.data.ConsumerCommand; import org.apache.pulsar.websocket.data.ConsumerMessage; import org.apache.pulsar.websocket.data.EndOfTopicResponse; -import org.eclipse.jetty.websocket.api.Session; -import org.eclipse.jetty.websocket.api.WriteCallback; -import org.eclipse.jetty.websocket.servlet.ServletUpgradeResponse; +import org.eclipse.jetty.ee8.websocket.api.Session; +import org.eclipse.jetty.ee8.websocket.api.WriteCallback; +import org.eclipse.jetty.ee8.websocket.server.JettyServerUpgradeResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,7 +76,7 @@ public class ReaderHandler extends AbstractWebSocketHandler { private static final AtomicLongFieldUpdater MSG_DELIVERED_COUNTER_UPDATER = AtomicLongFieldUpdater.newUpdater(ReaderHandler.class, "msgDeliveredCounter"); - public ReaderHandler(WebSocketService service, HttpServletRequest request, ServletUpgradeResponse response) { + public ReaderHandler(WebSocketService service, HttpServletRequest request, JettyServerUpgradeResponse response) { super(service, request, response); final int receiverQueueSize = getReceiverQueueSize(); @@ -119,6 +119,7 @@ public ReaderHandler(WebSocketService service, HttpServletRequest request, Servl log.warn("[{}:{}] Failed to add reader handler for topic {}", request.getRemoteAddr(), request.getRemotePort(), topic); } + allowConnect = true; } catch (Exception e) { log.warn("[{}:{}] Failed in creating reader {} on topic {}", request.getRemoteAddr(), request.getRemotePort(), subscription, topic, e); @@ -165,7 +166,7 @@ private void receiveMessage() { @Override public void writeFailed(Throwable th) { log.warn("[{}/{}] Failed to deliver msg to {} {}", reader.getTopic(), subscription, - getRemote().getInetSocketAddress().toString(), th.getMessage()); + getRemote().getRemoteAddress().toString(), th.getMessage()); pendingMessages.decrementAndGet(); // schedule receive as one of the delivery failed service.getExecutor().execute(() -> receiveMessage()); @@ -175,7 +176,7 @@ public void writeFailed(Throwable th) { public void writeSuccess() { if (log.isDebugEnabled()) { log.debug("[{}/{}] message is delivered successfully to {} ", reader.getTopic(), - subscription, getRemote().getInetSocketAddress().toString()); + subscription, getRemote().getRemoteAddress().toString()); } updateDeliverMsgStat(msgSize); } @@ -194,7 +195,7 @@ public void writeSuccess() { log.info("[{}/{}] Reader was closed while receiving msg from broker", reader.getTopic(), subscription); } else { log.warn("[{}/{}] Error occurred while reader handler was delivering msg to {}: {}", reader.getTopic(), - subscription, getRemote().getInetSocketAddress().toString(), exception.getMessage()); + subscription, getRemote().getRemoteAddress().toString(), exception.getMessage()); } return null; }); @@ -241,14 +242,14 @@ private void handleEndOfTopic() { @Override public void writeFailed(Throwable th) { log.warn("[{}/{}] Failed to send end of topic msg to {} due to {}", reader.getTopic(), - subscription, getRemote().getInetSocketAddress().toString(), th.getMessage()); + subscription, getRemote().getRemoteAddress().toString(), th.getMessage()); } @Override public void writeSuccess() { if (log.isDebugEnabled()) { log.debug("[{}/{}] End of topic message is delivered successfully to {} ", - reader.getTopic(), subscription, getRemote().getInetSocketAddress().toString()); + reader.getTopic(), subscription, getRemote().getRemoteAddress().toString()); } } }); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketConsumerServlet.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketConsumerServlet.java index 8f718bc744d03..f97a11fd164c1 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketConsumerServlet.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketConsumerServlet.java @@ -18,10 +18,11 @@ */ package org.apache.pulsar.websocket; -import org.eclipse.jetty.websocket.servlet.WebSocketServlet; -import org.eclipse.jetty.websocket.servlet.WebSocketServletFactory; +import java.time.Duration; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServlet; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServletFactory; -public class WebSocketConsumerServlet extends WebSocketServlet { +public class WebSocketConsumerServlet extends JettyWebSocketServlet { private static final long serialVersionUID = 1L; public static final String SERVLET_PATH = "/ws/consumer"; @@ -35,12 +36,15 @@ public WebSocketConsumerServlet(WebSocketService service) { } @Override - public void configure(WebSocketServletFactory factory) { - factory.getPolicy().setMaxTextMessageSize(service.getConfig().getWebSocketMaxTextFrameSize()); + public void configure(JettyWebSocketServletFactory factory) { + factory.setMaxTextMessageSize(service.getConfig().getWebSocketMaxTextFrameSize()); if (service.getConfig().getWebSocketSessionIdleTimeoutMillis() > 0) { - factory.getPolicy().setIdleTimeout(service.getConfig().getWebSocketSessionIdleTimeoutMillis()); + factory.setIdleTimeout(Duration.ofMillis(service.getConfig().getWebSocketSessionIdleTimeoutMillis())); } - factory.setCreator( - (request, response) -> new ConsumerHandler(service, request.getHttpServletRequest(), response)); + factory.setCreator((request, response) -> { + ConsumerHandler consumerHandler = + new ConsumerHandler(service, request.getHttpServletRequest(), response); + return consumerHandler.isAllowConnect() && response.getStatusCode() < 400 ? consumerHandler : null; + }); } } diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketHttpServletRequestWrapper.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketHttpServletRequestWrapper.java index 19ce9a32363c1..6b8e6edb8a83c 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketHttpServletRequestWrapper.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketHttpServletRequestWrapper.java @@ -20,7 +20,6 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletRequestWrapper; -import org.eclipse.jetty.websocket.servlet.UpgradeHttpServletRequest; /** * WebSocket HttpServletRequest wrapper. @@ -40,7 +39,7 @@ public String getHeader(String name) { // The browser javascript WebSocket client couldn't add the auth param to the request header, use the // query param `token` to transport the auth token for the browser javascript WebSocket client. if (name.equals(HTTP_HEADER_NAME) - && !((UpgradeHttpServletRequest) this.getRequest()).getHeaders().containsKey(HTTP_HEADER_NAME)) { + && ((HttpServletRequest) getRequest()).getHeader(HTTP_HEADER_NAME) == null) { String token = getRequest().getParameter(TOKEN); if (token != null && !token.startsWith(HTTP_HEADER_VALUE_PREFIX)) { return HTTP_HEADER_VALUE_PREFIX + token; diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketMultiTopicConsumerServlet.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketMultiTopicConsumerServlet.java index 4653cea98c15d..82f9c5e1d83c4 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketMultiTopicConsumerServlet.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketMultiTopicConsumerServlet.java @@ -18,10 +18,11 @@ */ package org.apache.pulsar.websocket; -import org.eclipse.jetty.websocket.servlet.WebSocketServlet; -import org.eclipse.jetty.websocket.servlet.WebSocketServletFactory; +import java.time.Duration; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServlet; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServletFactory; -public class WebSocketMultiTopicConsumerServlet extends WebSocketServlet { +public class WebSocketMultiTopicConsumerServlet extends JettyWebSocketServlet { private static final long serialVersionUID = 1L; public static final String SERVLET_PATH = "/ws/v3/consumer"; @@ -34,12 +35,16 @@ public WebSocketMultiTopicConsumerServlet(WebSocketService service) { } @Override - public void configure(WebSocketServletFactory factory) { - factory.getPolicy().setMaxTextMessageSize(service.getConfig().getWebSocketMaxTextFrameSize()); + public void configure(JettyWebSocketServletFactory factory) { + factory.setMaxTextMessageSize(service.getConfig().getWebSocketMaxTextFrameSize()); if (service.getConfig().getWebSocketSessionIdleTimeoutMillis() > 0) { - factory.getPolicy().setIdleTimeout(service.getConfig().getWebSocketSessionIdleTimeoutMillis()); + factory.setIdleTimeout(Duration.ofMillis(service.getConfig().getWebSocketSessionIdleTimeoutMillis())); } - factory.setCreator((request, response) -> - new MultiTopicConsumerHandler(service, request.getHttpServletRequest(), response)); + factory.setCreator((request, response) -> { + MultiTopicConsumerHandler multiTopicConsumerHandler = + new MultiTopicConsumerHandler(service, request.getHttpServletRequest(), response); + return multiTopicConsumerHandler.isAllowConnect() && response.getStatusCode() < 400 + ? multiTopicConsumerHandler : null; + }); } } diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketProducerServlet.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketProducerServlet.java index c9d8861b76e7b..2dd75071e83b7 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketProducerServlet.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketProducerServlet.java @@ -18,10 +18,11 @@ */ package org.apache.pulsar.websocket; -import org.eclipse.jetty.websocket.servlet.WebSocketServlet; -import org.eclipse.jetty.websocket.servlet.WebSocketServletFactory; +import java.time.Duration; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServlet; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServletFactory; -public class WebSocketProducerServlet extends WebSocketServlet { +public class WebSocketProducerServlet extends JettyWebSocketServlet { private static final long serialVersionUID = 1L; public static final String SERVLET_PATH = "/ws/producer"; @@ -34,12 +35,14 @@ public WebSocketProducerServlet(WebSocketService service) { } @Override - public void configure(WebSocketServletFactory factory) { - factory.getPolicy().setMaxTextMessageSize(service.getConfig().getWebSocketMaxTextFrameSize()); + public void configure(JettyWebSocketServletFactory factory) { + factory.setMaxTextMessageSize(service.getConfig().getWebSocketMaxTextFrameSize()); if (service.getConfig().getWebSocketSessionIdleTimeoutMillis() > 0) { - factory.getPolicy().setIdleTimeout(service.getConfig().getWebSocketSessionIdleTimeoutMillis()); + factory.setIdleTimeout(Duration.ofMillis(service.getConfig().getWebSocketSessionIdleTimeoutMillis())); } - factory.setCreator((request, response) -> - new ProducerHandler(service, request.getHttpServletRequest(), response)); + factory.setCreator((request, response) -> { + ProducerHandler producerHandler = new ProducerHandler(service, request.getHttpServletRequest(), response); + return producerHandler.isAllowConnect() && response.getStatusCode() < 400 ? producerHandler : null; + }); } } diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketReaderServlet.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketReaderServlet.java index 9d23d10c39581..f04effc0c5259 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketReaderServlet.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketReaderServlet.java @@ -18,10 +18,11 @@ */ package org.apache.pulsar.websocket; -import org.eclipse.jetty.websocket.servlet.WebSocketServlet; -import org.eclipse.jetty.websocket.servlet.WebSocketServletFactory; +import java.time.Duration; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServlet; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServletFactory; -public class WebSocketReaderServlet extends WebSocketServlet { +public class WebSocketReaderServlet extends JettyWebSocketServlet { private static final transient long serialVersionUID = 1L; public static final String SERVLET_PATH = "/ws/reader"; @@ -35,12 +36,14 @@ public WebSocketReaderServlet(WebSocketService service) { } @Override - public void configure(WebSocketServletFactory factory) { - factory.getPolicy().setMaxTextMessageSize(service.getConfig().getWebSocketMaxTextFrameSize()); + public void configure(JettyWebSocketServletFactory factory) { + factory.setMaxTextMessageSize(service.getConfig().getWebSocketMaxTextFrameSize()); if (service.getConfig().getWebSocketSessionIdleTimeoutMillis() > 0) { - factory.getPolicy().setIdleTimeout(service.getConfig().getWebSocketSessionIdleTimeoutMillis()); + factory.setIdleTimeout(Duration.ofMillis(service.getConfig().getWebSocketSessionIdleTimeoutMillis())); } - factory.setCreator( - (request, response) -> new ReaderHandler(service, request.getHttpServletRequest(), response)); + factory.setCreator((request, response) -> { + ReaderHandler readerHandler = new ReaderHandler(service, request.getHttpServletRequest(), response); + return readerHandler.isAllowConnect() && response.getStatusCode() < 400 ? readerHandler : null; + }); } } diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java index d379d01d1be25..654fb41c3bb94 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java @@ -32,7 +32,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import javax.servlet.ServletException; -import javax.websocket.DeploymentException; import lombok.Getter; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.PulsarServerException; @@ -94,8 +93,7 @@ public WebSocketService(ClusterData localCluster, ServiceConfiguration config) { this.proxyStats = new ProxyStats(this); } - public void start() throws PulsarServerException, PulsarClientException, MalformedURLException, ServletException, - DeploymentException { + public void start() throws PulsarServerException, PulsarClientException, MalformedURLException, ServletException { if (isNotBlank(config.getConfigurationMetadataStoreUrl())) { try { 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 e7523252bd960..88ff5e8374a0f 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 @@ -21,17 +21,14 @@ import java.net.MalformedURLException; import java.util.ArrayList; import java.util.Arrays; -import java.util.EnumSet; import java.util.List; import java.util.Optional; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import javax.servlet.DispatcherType; import javax.servlet.Servlet; import javax.servlet.ServletException; -import javax.websocket.DeploymentException; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.web.JettyRequestLogFactory; import org.apache.pulsar.broker.web.JsonMapperProvider; @@ -42,25 +39,23 @@ import org.apache.pulsar.common.util.PulsarSslConfiguration; import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.ee8.servlet.ServletContextHandler; +import org.eclipse.jetty.ee8.servlet.ServletHolder; +import org.eclipse.jetty.ee8.websocket.server.config.JettyWebSocketServletContainerInitializer; +import org.eclipse.jetty.http.UriCompliance; 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.NetworkConnectionLimit; 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; -import org.eclipse.jetty.server.handler.RequestLogHandler; -import org.eclipse.jetty.servlet.FilterHolder; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.QoSFilter; +import org.eclipse.jetty.server.handler.QoSHandler; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.glassfish.jersey.server.ResourceConfig; import org.glassfish.jersey.servlet.ServletContainer; @@ -73,7 +68,6 @@ public class ProxyServer { private final List handlers = new ArrayList<>(); private final WebSocketProxyConfiguration conf; private final WebExecutorThreadPool executorService; - private final FilterHolder qualityOfServiceFilterHolder; private ServerConnector connector; private ServerConnector connectorTls; @@ -87,10 +81,11 @@ public ProxyServer(WebSocketProxyConfiguration config) config.getHttpServerThreadPoolQueueSize()); this.server = new Server(executorService); if (config.getMaxHttpServerConnections() > 0) { - server.addBean(new ConnectionLimit(config.getMaxHttpServerConnections(), server)); + server.addBean(new NetworkConnectionLimit(config.getMaxHttpServerConnections(), server)); } HttpConfiguration httpConfig = new HttpConfiguration(); + httpConfig.setUriCompliance(UriCompliance.LEGACY); if (config.isWebServiceTrustXForwardedFor()) { httpConfig.addCustomizer(new ForwardedRequestCustomizer()); } @@ -124,7 +119,7 @@ public ProxyServer(WebSocketProxyConfiguration config) config.getTlsCertRefreshCheckDurationSec(), TimeUnit.SECONDS); } - SslContextFactory sslCtxFactory = + SslContextFactory.Server sslCtxFactory = JettySslContextFactory.createSslContextFactory(config.getTlsProvider(), sslFactory, config.isTlsRequireTrustedClientCertOnConnect(), config.getWebServiceTlsCiphers(), config.getWebServiceTlsProtocols()); @@ -137,7 +132,9 @@ public ProxyServer(WebSocketProxyConfiguration config) // 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()); + // disable SNI host check for backwards compatibility with Jetty 9.x + boolean sniHostCheck = false; + httpConfig.addCustomizer(new SecureRequestCustomizer(sniHostCheck)); } connectorTls = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connectorTls.setPort(config.getWebServicePortTls().get()); @@ -151,24 +148,16 @@ public ProxyServer(WebSocketProxyConfiguration config) // file descriptors connectors.stream().forEach(c -> c.setAcceptQueueSize(config.getHttpServerAcceptQueueSize())); server.setConnectors(connectors.toArray(new ServerConnector[connectors.size()])); - - if (config.getMaxConcurrentHttpRequests() > 0) { - qualityOfServiceFilterHolder = new FilterHolder(QoSFilter.class); - qualityOfServiceFilterHolder.setInitParameter("maxRequests", - String.valueOf(config.getMaxConcurrentHttpRequests())); - } else { - qualityOfServiceFilterHolder = null; - } } public void addWebSocketServlet(String basePath, Servlet socketServlet) - throws ServletException, DeploymentException { + throws ServletException { ServletHolder servletHolder = new ServletHolder("ws-events", socketServlet); ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS); context.setContextPath(basePath); context.addServlet(servletHolder, MATCH_ALL); - addQosFilterIfNeeded(context); - handlers.add(context); + JettyWebSocketServletContainerInitializer.configure(context, null); + handlers.add(context.get()); } public void addRestResource(String basePath, String attribute, Object attributeValue, Class resourceClass) { @@ -181,35 +170,27 @@ public void addRestResource(String basePath, String attribute, Object attributeV context.setContextPath(basePath); context.addServlet(servletHolder, MATCH_ALL); context.setAttribute(attribute, attributeValue); - addQosFilterIfNeeded(context); - handlers.add(context); - } - - private void addQosFilterIfNeeded(ServletContextHandler context) { - if (qualityOfServiceFilterHolder != null) { - context.addFilter(qualityOfServiceFilterHolder, - MATCH_ALL, EnumSet.allOf(DispatcherType.class)); - } + handlers.add(context.get()); } public void start() throws PulsarServerException { log.info("Starting web socket proxy at port {}", Arrays.stream(server.getConnectors()) .map(ServerConnector.class::cast).map(ServerConnector::getPort).map(Object::toString) .collect(Collectors.joining(","))); - RequestLogHandler requestLogHandler = new RequestLogHandler(); 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); + server.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); ContextHandlerCollection contexts = new ContextHandlerCollection(); - contexts.setHandlers(handlers.toArray(new Handler[handlers.size()])); - - HandlerCollection handlerCollection = new HandlerCollection(); - handlerCollection.setHandlers(new Handler[] { contexts, new DefaultHandler(), requestLogHandler }); - server.setHandler(handlerCollection); + contexts.setHandlers(handlers); + Handler serverHandler = contexts; + if (conf.getMaxConcurrentHttpRequests() > 0) { + QoSHandler qoSHandler = new QoSHandler(serverHandler); + qoSHandler.setMaxRequestCount(conf.getMaxConcurrentHttpRequests()); + serverHandler = qoSHandler; + } + server.setHandler(serverHandler); try { server.start(); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/stats/ProxyTopicStat.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/stats/ProxyTopicStat.java index 96a1b43adde69..0c8651eb3f9e8 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/stats/ProxyTopicStat.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/stats/ProxyTopicStat.java @@ -45,7 +45,7 @@ public ProducerStats() { } public ProducerStats(ProducerHandler handler) { - this.remoteConnection = handler.getRemote().getInetSocketAddress().toString(); + this.remoteConnection = handler.getRemote().getRemoteAddress().toString(); this.numberOfMsgPublished = handler.getMsgPublishedCounter(); } @@ -62,14 +62,14 @@ public ConsumerStats(ConsumerHandler handler) { this.subscriptionName = handler.getSubscription(); this.subscriptionType = handler.getSubscriptionType(); this.subscriptionMode = handler.getSubscriptionMode(); - this.remoteConnection = handler.getRemote().getInetSocketAddress().toString(); + this.remoteConnection = handler.getRemote().getRemoteAddress().toString(); this.numberOfMsgDelivered = handler.getMsgDeliveredCounter(); } public ConsumerStats(ReaderHandler handler) { this.subscriptionName = handler.getSubscription(); this.subscriptionType = handler.getSubscriptionType(); - this.remoteConnection = handler.getRemote().getInetSocketAddress().toString(); + this.remoteConnection = handler.getRemote().getRemoteAddress().toString(); this.numberOfMsgDelivered = handler.getMsgDeliveredCounter(); } diff --git a/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/AbstractWebSocketHandlerTest.java b/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/AbstractWebSocketHandlerTest.java index e072ad5acd85b..efae1931f5760 100644 --- a/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/AbstractWebSocketHandlerTest.java +++ b/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/AbstractWebSocketHandlerTest.java @@ -38,7 +38,6 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; import lombok.Cleanup; import lombok.Getter; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; @@ -56,10 +55,11 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.websocket.service.WebSocketProxyConfiguration; +import org.eclipse.jetty.ee8.websocket.api.RemoteEndpoint; +import org.eclipse.jetty.ee8.websocket.api.Session; +import org.eclipse.jetty.ee8.websocket.server.JettyServerUpgradeResponse; import org.eclipse.jetty.http.HttpStatus; -import org.eclipse.jetty.websocket.api.RemoteEndpoint; -import org.eclipse.jetty.websocket.api.Session; -import org.eclipse.jetty.websocket.servlet.ServletUpgradeResponse; +import org.mockito.Answers; import org.mockito.Mock; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; @@ -218,8 +218,9 @@ public void parseTopicNameTest() { static class WebSocketHandlerImpl extends AbstractWebSocketHandler { public WebSocketHandlerImpl(WebSocketService service, HttpServletRequest request, - ServletUpgradeResponse response) { + JettyServerUpgradeResponse response) { super(service, request, response); + allowConnect = true; } @Override @@ -238,15 +239,15 @@ public TopicName getTopic() { } - static class MockedServletUpgradeResponse extends ServletUpgradeResponse { + abstract static class MockedServletUpgradeResponse implements JettyServerUpgradeResponse { @Getter private int statusCode; @Getter private String message; - public MockedServletUpgradeResponse(HttpServletResponse response) { - super(response); + public MockedServletUpgradeResponse() { + } public void sendError(int statusCode, String message) { @@ -267,7 +268,7 @@ PulsarClient newPulsarClient() throws PulsarClientException { class MockedProducerHandler extends ProducerHandler { public MockedProducerHandler(WebSocketService service, HttpServletRequest request, - ServletUpgradeResponse response) { + JettyServerUpgradeResponse response) { super(service, request, response); } @@ -310,7 +311,7 @@ public void producerBuilderTest() throws IOException { when(service.isAuthorizationEnabled()).thenReturn(false); when(service.getPulsarClient()).thenReturn(newPulsarClient()); - MockedServletUpgradeResponse response = new MockedServletUpgradeResponse(null); + MockedServletUpgradeResponse response = mock(MockedServletUpgradeResponse.class, Answers.CALLS_REAL_METHODS); MockedProducerHandler producerHandler = new MockedProducerHandler(service, httpServletRequest, response); assertEquals(response.getStatusCode(), 500); @@ -345,7 +346,7 @@ public void producerBuilderTest() throws IOException { class MockedConsumerHandler extends ConsumerHandler { public MockedConsumerHandler(WebSocketService service, HttpServletRequest request, - ServletUpgradeResponse response) { + JettyServerUpgradeResponse response) { super(service, request, response); } @@ -385,7 +386,7 @@ public void consumerBuilderTest() throws IOException { when(service.isAuthorizationEnabled()).thenReturn(false); when(service.getPulsarClient()).thenReturn(newPulsarClient()); - MockedServletUpgradeResponse response = new MockedServletUpgradeResponse(null); + MockedServletUpgradeResponse response = mock(MockedServletUpgradeResponse.class, Answers.CALLS_REAL_METHODS); MockedConsumerHandler consumerHandler = new MockedConsumerHandler(service, httpServletRequest, response); assertEquals(response.getStatusCode(), 500); @@ -439,7 +440,7 @@ public void testPingFuture() throws IOException { when(httpServletRequest.getRequestURI()).thenReturn(consumerV2); when(httpServletRequest.getParameterMap()).thenReturn(queryParams); - MockedServletUpgradeResponse response = new MockedServletUpgradeResponse(null); + MockedServletUpgradeResponse response = mock(MockedServletUpgradeResponse.class, Answers.CALLS_REAL_METHODS); AbstractWebSocketHandler webSocketHandler = new WebSocketHandlerImpl(webSocketService, httpServletRequest, response); diff --git a/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/PingPongSupportTest.java b/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/PingPongSupportTest.java index 0bcb8b25efd31..b40f4784e4ea3 100644 --- a/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/PingPongSupportTest.java +++ b/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/PingPongSupportTest.java @@ -27,26 +27,27 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.Future; import javax.servlet.http.HttpServletRequest; import lombok.Cleanup; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.eclipse.jetty.client.HttpClient; +import org.eclipse.jetty.ee8.servlet.ServletContextHandler; +import org.eclipse.jetty.ee8.servlet.ServletHolder; +import org.eclipse.jetty.ee8.websocket.server.JettyServerUpgradeResponse; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServlet; +import org.eclipse.jetty.ee8.websocket.server.JettyWebSocketServletFactory; +import org.eclipse.jetty.ee8.websocket.server.config.JettyWebSocketServletContainerInitializer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.util.BufferUtil; +import org.eclipse.jetty.util.component.LifeCycle; +import org.eclipse.jetty.websocket.api.Callback; import org.eclipse.jetty.websocket.api.Session; -import org.eclipse.jetty.websocket.api.WebSocketAdapter; -import org.eclipse.jetty.websocket.api.WebSocketPingPongListener; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketPong; import org.eclipse.jetty.websocket.api.annotations.WebSocket; import org.eclipse.jetty.websocket.client.WebSocketClient; -import org.eclipse.jetty.websocket.servlet.ServletUpgradeResponse; -import org.eclipse.jetty.websocket.servlet.WebSocketServlet; -import org.eclipse.jetty.websocket.servlet.WebSocketServletFactory; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; @@ -83,6 +84,8 @@ public void setup() throws Exception { ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS); context.setContextPath("/ws"); context.addServlet(servletHolder, "/*"); + JettyWebSocketServletContainerInitializer.configure(context, null); + server.setHandler(context); try { server.start(); @@ -112,20 +115,25 @@ public static Object[][] cacheEnable() { public void testPingPong(String endpoint) throws Exception { @Cleanup("stop") HttpClient httpClient = new HttpClient(); - WebSocketClient webSocketClient = new WebSocketClient(httpClient); - webSocketClient.start(); + WebSocketClient container = new WebSocketClient(httpClient); + container.start(); + @Cleanup("stop") + LifeCycle lifeCycle = (LifeCycle) container; MyWebSocket myWebSocket = new MyWebSocket(); String webSocketUri = "ws://localhost:8080/ws/v2/" + endpoint + "/persistent/my-property/my-ns/my-topic"; - Future sessionFuture = webSocketClient.connect(myWebSocket, URI.create(webSocketUri)); - sessionFuture.get().getRemote().sendPing(ByteBuffer.wrap("test".getBytes())); + Session session = container.connect(myWebSocket, URI.create(webSocketUri)).get(); + Callback.Completable callback = new Callback.Completable(); + session.sendPing(ByteBuffer.wrap("test".getBytes()), callback); + callback.get(); assertTrue(myWebSocket.getResponse().contains("test")); } public static class GenericWebSocketHandler extends AbstractWebSocketHandler { public GenericWebSocketHandler(WebSocketService service, HttpServletRequest request, - ServletUpgradeResponse response) { + JettyServerUpgradeResponse response) { super(service, request, response); + allowConnect = true; } @Override @@ -139,7 +147,7 @@ public void close() throws IOException { } } - public static class GenericWebSocketServlet extends WebSocketServlet { + public static class GenericWebSocketServlet extends JettyWebSocketServlet { private static final long serialVersionUID = 1L; private final WebSocketService service; @@ -149,34 +157,18 @@ public GenericWebSocketServlet(WebSocketService service) { } @Override - public void configure(WebSocketServletFactory factory) { + public void configure(JettyWebSocketServletFactory factory) { factory.setCreator((request, response) -> new GenericWebSocketHandler(service, request.getHttpServletRequest(), response)); } } @WebSocket - public static class MyWebSocket extends WebSocketAdapter implements WebSocketPingPongListener { + public static class MyWebSocket { ArrayBlockingQueue incomingMessages = new ArrayBlockingQueue<>(10); - @Override - public void onWebSocketClose(int i, String s) { - } - - @Override - public void onWebSocketConnect(Session session) { - } - - @Override - public void onWebSocketError(Throwable throwable) { - } - - @Override - public void onWebSocketPing(ByteBuffer payload) { - } - - @Override + @OnWebSocketPong public void onWebSocketPong(ByteBuffer payload) { incomingMessages.add(BufferUtil.toDetailString(payload)); } diff --git a/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/ProducerHandlerTest.java b/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/ProducerHandlerTest.java index bca3671b73c74..d9adcebb5a334 100644 --- a/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/ProducerHandlerTest.java +++ b/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/ProducerHandlerTest.java @@ -40,7 +40,7 @@ import org.apache.pulsar.client.impl.TypedMessageBuilderImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.websocket.data.ProducerMessage; -import org.eclipse.jetty.websocket.servlet.ServletUpgradeResponse; +import org.eclipse.jetty.ee8.websocket.server.JettyServerUpgradeResponse; import org.testng.annotations.Test; public class ProducerHandlerTest { @@ -82,7 +82,7 @@ public void testProduceMessageAttributes() throws IOException { when(producer.newMessage()).thenReturn(messageBuilder); when(messageBuilder.sendAsync()).thenReturn(CompletableFuture.completedFuture(new MessageIdImpl(1, 2, 3))); - ServletUpgradeResponse response = mock(ServletUpgradeResponse.class); + JettyServerUpgradeResponse response = mock(JettyServerUpgradeResponse.class); ProducerHandler producerHandler = new ProducerHandler(service, httpServletRequest, response); producerHandler.onWebSocketText(ObjectMapperFactory.getMapper().writer().writeValueAsString(produceRequest)); diff --git a/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/ReaderHandlerTest.java b/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/ReaderHandlerTest.java index be2fa241e1035..a79899ab6fac7 100644 --- a/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/ReaderHandlerTest.java +++ b/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/ReaderHandlerTest.java @@ -32,7 +32,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.Function; import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; @@ -44,7 +43,7 @@ import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; import org.apache.pulsar.client.impl.MultiTopicsReaderImpl; import org.apache.pulsar.client.impl.ReaderImpl; -import org.eclipse.jetty.websocket.servlet.ServletUpgradeResponse; +import org.eclipse.jetty.ee8.websocket.server.JettyServerUpgradeResponse; import org.testng.Assert; import org.testng.annotations.Test; @@ -71,8 +70,7 @@ public void testCreateReaderImp() throws IOException { HttpServletRequest request = mock(HttpServletRequest.class); when(request.getRequestURI()).thenReturn("/ws/v2/producer/persistent/my-property/my-ns/my-topic"); // create reader handler - HttpServletResponse response = spy(HttpServletResponse.class); - ServletUpgradeResponse servletUpgradeResponse = new ServletUpgradeResponse(response); + JettyServerUpgradeResponse servletUpgradeResponse = mock(JettyServerUpgradeResponse.class); ReaderHandler readerHandler = new ReaderHandler(wss, request, servletUpgradeResponse); // verify success Assert.assertEquals(readerHandler.getSubscription(), subName); @@ -101,8 +99,7 @@ public void testCreateMultipleTopicReaderImp() throws IOException { HttpServletRequest request = mock(HttpServletRequest.class); when(request.getRequestURI()).thenReturn("/ws/v2/producer/persistent/my-property/my-ns/my-topic"); // create reader handler - HttpServletResponse response = spy(HttpServletResponse.class); - ServletUpgradeResponse servletUpgradeResponse = new ServletUpgradeResponse(response); + JettyServerUpgradeResponse servletUpgradeResponse = mock(JettyServerUpgradeResponse.class); ReaderHandler readerHandler = new ReaderHandler(wss, request, servletUpgradeResponse); // verify success Assert.assertEquals(readerHandler.getSubscription(), subName); @@ -127,11 +124,10 @@ public void testCreateIllegalReaderImp() throws IOException { HttpServletRequest request = mock(HttpServletRequest.class); when(request.getRequestURI()).thenReturn("/ws/v2/producer/persistent/my-property/my-ns/my-topic"); // create reader handler - HttpServletResponse response = spy(HttpServletResponse.class); - ServletUpgradeResponse servletUpgradeResponse = new ServletUpgradeResponse(response); + JettyServerUpgradeResponse servletUpgradeResponse = spy(JettyServerUpgradeResponse.class); new ReaderHandler(wss, request, servletUpgradeResponse); // verify get error - verify(response, times(1)).sendError(anyInt(), anyString()); + verify(servletUpgradeResponse, times(1)).sendError(anyInt(), anyString()); } diff --git a/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/WebSocketHttpServletRequestWrapperTest.java b/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/WebSocketHttpServletRequestWrapperTest.java index 187e3823677fc..f7955e9362e10 100644 --- a/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/WebSocketHttpServletRequestWrapperTest.java +++ b/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/WebSocketHttpServletRequestWrapperTest.java @@ -18,10 +18,10 @@ */ package org.apache.pulsar.websocket; +import javax.servlet.http.HttpServletRequest; import lombok.Cleanup; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.websocket.service.WebSocketProxyConfiguration; -import org.eclipse.jetty.websocket.servlet.UpgradeHttpServletRequest; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.Test; @@ -40,7 +40,7 @@ public class WebSocketHttpServletRequestWrapperTest { @Test public void testTokenParamWithBearerPrefix() { - UpgradeHttpServletRequest httpServletRequest = Mockito.mock(UpgradeHttpServletRequest.class); + HttpServletRequest httpServletRequest = Mockito.mock(HttpServletRequest.class); Mockito.when(httpServletRequest.getParameter(WebSocketHttpServletRequestWrapper.TOKEN)) .thenReturn(BEARER_TOKEN); @@ -53,7 +53,7 @@ public void testTokenParamWithBearerPrefix() { @Test public void testTokenParamWithOutBearerPrefix() { - UpgradeHttpServletRequest httpServletRequest = Mockito.mock(UpgradeHttpServletRequest.class); + HttpServletRequest httpServletRequest = Mockito.mock(HttpServletRequest.class); Mockito.when(httpServletRequest.getParameter(WebSocketHttpServletRequestWrapper.TOKEN)) .thenReturn(TOKEN); @@ -75,7 +75,7 @@ public void mockRequestTest() throws Exception { WebSocketService service = new WebSocketService(config); service.start(); - UpgradeHttpServletRequest httpServletRequest = Mockito.mock(UpgradeHttpServletRequest.class); + HttpServletRequest httpServletRequest = Mockito.mock(HttpServletRequest.class); Mockito.when(httpServletRequest.getRemoteAddr()).thenReturn("localhost"); Mockito.when(httpServletRequest.getRemotePort()).thenReturn(8080); Mockito.when(httpServletRequest.getParameter(WebSocketHttpServletRequestWrapper.TOKEN)) diff --git a/testmocks/pom.xml b/testmocks/pom.xml index a9c32d65a15b5..5a49e97d42b2a 100644 --- a/testmocks/pom.xml +++ b/testmocks/pom.xml @@ -35,8 +35,9 @@ - org.apache.zookeeper - zookeeper + org.apache.pulsar + zookeeper-with-patched-admin + ${project.version} ch.qos.logback diff --git a/tests/docker-images/java-test-plugins/src/main/java/org/apache/pulsar/tests/integration/plugins/RandomAdditionalServlet.java b/tests/docker-images/java-test-plugins/src/main/java/org/apache/pulsar/tests/integration/plugins/RandomAdditionalServlet.java index 9d4f7e18a6db8..7d6e793a266c8 100644 --- a/tests/docker-images/java-test-plugins/src/main/java/org/apache/pulsar/tests/integration/plugins/RandomAdditionalServlet.java +++ b/tests/docker-images/java-test-plugins/src/main/java/org/apache/pulsar/tests/integration/plugins/RandomAdditionalServlet.java @@ -30,7 +30,6 @@ import javax.servlet.http.HttpServletResponse; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlet; import org.apache.pulsar.common.configuration.PulsarConfiguration; -import org.eclipse.jetty.servlet.ServletHolder; public class RandomAdditionalServlet extends HttpServlet implements AdditionalServlet { @@ -49,8 +48,8 @@ public String getBasePath() { } @Override - public ServletHolder getServletHolder() { - return new ServletHolder(this); + public Object getServletInstance() { + return this; } @Override diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index 3f3c07d70c0d7..ee940a36e7c1c 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -252,6 +252,11 @@ k3s test + + org.eclipse.jetty.websocket + jetty-websocket-jetty-client + test + io.kubernetes diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/websocket/WebSocketTestSuite.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/websocket/WebSocketTestSuite.java index ae0ad2113774e..3b309065ed1a6 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/websocket/WebSocketTestSuite.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/websocket/WebSocketTestSuite.java @@ -32,7 +32,10 @@ import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.tests.integration.suites.PulsarTestSuite; import org.eclipse.jetty.client.HttpClient; -import org.eclipse.jetty.websocket.api.WebSocketAdapter; +import org.eclipse.jetty.websocket.api.Callback; +import org.eclipse.jetty.websocket.api.Session; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketMessage; +import org.eclipse.jetty.websocket.api.annotations.OnWebSocketOpen; import org.eclipse.jetty.websocket.api.annotations.WebSocket; import org.eclipse.jetty.websocket.client.WebSocketClient; import org.slf4j.Logger; @@ -81,9 +84,10 @@ protected void testWebSocket(String url) throws Exception { } @WebSocket - public static class Client extends WebSocketAdapter implements AutoCloseable { + public abstract static class Client implements AutoCloseable { final BlockingQueue incomingMessages = new ArrayBlockingQueue<>(10); private final WebSocketClient client; + private Session session; Client(String webSocketUri) throws Exception { HttpClient httpClient = new HttpClient(); @@ -92,11 +96,16 @@ public static class Client extends WebSocketAdapter implements AutoCloseable { client.connect(this, URI.create(webSocketUri)).get(); } + @OnWebSocketOpen + public void onWebSocketConnect(Session session) { + this.session = session; + } + void sendText(String payload) throws IOException { - getSession().getRemote().sendString(payload); + session.sendText(payload, Callback.NOOP); } - @Override + @OnWebSocketMessage public void onWebSocketText(String s) { incomingMessages.add(s); } @@ -107,7 +116,6 @@ Map getResponse() throws Exception { Assert.fail("Did not get websocket response within timeout"); } return ObjectMapperFactory.getMapper().getObjectMapper().readValue(response, new TypeReference<>() {}); - } @Override diff --git a/tiered-storage/file-system/pom.xml b/tiered-storage/file-system/pom.xml index 3d39e0ca64ec4..2cd0f927eb924 100644 --- a/tiered-storage/file-system/pom.xml +++ b/tiered-storage/file-system/pom.xml @@ -20,239 +20,251 @@ --> - 4.0.0 + 4.0.0 - - org.apache.pulsar - tiered-storage-parent - 4.2.0-SNAPSHOT - + + org.apache.pulsar + tiered-storage-parent + 4.2.0-SNAPSHOT + - tiered-storage-file-system - Apache Pulsar :: Tiered Storage :: File System + tiered-storage-file-system + Apache Pulsar :: Tiered Storage :: File System + + - ${project.groupId} - managed-ledger - ${project.version} - provided + org.eclipse.jetty + jetty-bom + ${jetty9.version} + pom + import - - org.apache.hadoop - hadoop-common - ${hdfs-offload-version3} - - - log4j - log4j - - - org.slf4j - * - - - dnsjava - dnsjava - - - org.bouncycastle - bcprov-jdk15on - - - io.netty - * - - - - - org.bouncycastle - bcprov-jdk18on - - - org.apache.hadoop - hadoop-hdfs-client - ${hdfs-offload-version3} - - - org.apache.avro - avro - - - org.mortbay.jetty - jetty - - - com.sun.jersey - jersey-core - - - com.sun.jersey - jersey-server - - - javax.servlet - servlet-api - - - dnsjava - dnsjava - - - org.bouncycastle - bcprov-jdk15on - - - - - - org.apache.avro - avro - ${avro.version} - - - - net.minidev - json-smart - ${json-smart.version} - - - com.google.protobuf - protobuf-java - + + + + + ${project.groupId} + managed-ledger + ${project.version} + provided + + + org.apache.hadoop + hadoop-common + ${hdfs-offload-version3} + + + log4j + log4j + + + org.slf4j + * + + + dnsjava + dnsjava + + + org.bouncycastle + bcprov-jdk15on + + + io.netty + * + + + + + org.bouncycastle + bcprov-jdk18on + + + org.apache.hadoop + hadoop-hdfs-client + ${hdfs-offload-version3} + + + org.apache.avro + avro + + + org.mortbay.jetty + jetty + + + com.sun.jersey + jersey-core + + + com.sun.jersey + jersey-server + + + javax.servlet + servlet-api + + + dnsjava + dnsjava + + + org.bouncycastle + bcprov-jdk15on + + + + + + org.apache.avro + avro + ${avro.version} + + + + net.minidev + json-smart + ${json-smart.version} + + + com.google.protobuf + protobuf-java + - - ${project.groupId} - testmocks - ${project.version} - test - + + ${project.groupId} + testmocks + ${project.version} + test + - - org.apache.hadoop - hadoop-minicluster - ${hdfs-offload-version3} - test - - - io.netty - netty-all - - - org.bouncycastle - * - - - org.slf4j - * - - - dnsjava - dnsjava - - - org.bouncycastle - bcprov-jdk15on - - - + + org.apache.hadoop + hadoop-minicluster + ${hdfs-offload-version3} + test + + + io.netty + netty-all + + + org.bouncycastle + * + + + org.slf4j + * + + + dnsjava + dnsjava + + + org.bouncycastle + bcprov-jdk15on + + + - - org.bouncycastle - bcpkix-jdk18on - test - + + org.bouncycastle + bcpkix-jdk18on + test + - - io.netty - netty-codec-http - test - + + io.netty + netty-codec-http + test + - - org.eclipse.jetty - jetty-server - test - - - org.eclipse.jetty - jetty-alpn-conscrypt-server - test - - - org.eclipse.jetty - jetty-servlet - test - - - org.eclipse.jetty - jetty-util - test - - + + org.eclipse.jetty + jetty-server + test + + + org.eclipse.jetty + jetty-alpn-conscrypt-server + test + + + org.eclipse.jetty + jetty-servlet + test + + + org.eclipse.jetty + jetty-util + test + + - + + + + org.apache.nifi + nifi-nar-maven-plugin + + + com.github.spotbugs + spotbugs-maven-plugin + ${spotbugs-maven-plugin.version} + + ${basedir}/src/main/resources/findbugsExclude.xml + + + + spotbugs + verify + + check + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + checkstyle + verify + + check + + + + + + + + + + owasp-dependency-check + - - org.apache.nifi - nifi-nar-maven-plugin - - - com.github.spotbugs - spotbugs-maven-plugin - ${spotbugs-maven-plugin.version} - - ${basedir}/src/main/resources/findbugsExclude.xml - - - - spotbugs - verify - - check - - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - - - checkstyle - verify - - check - - - - + + org.owasp + dependency-check-maven + + + + aggregate + + none + + + - - - - - owasp-dependency-check - - - - org.owasp - dependency-check-maven - - - - aggregate - - none - - - - - - - + + +