diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/Category.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/Category.java new file mode 100644 index 0000000000000..7a2c81e48f53c --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/Category.java @@ -0,0 +1,33 @@ +/** + * 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.common.configuration; + +/** + * Setting Category. + */ +public @interface Category { + + /** + * Description of the category. + * + * @return description of the category + */ + String description() default ""; + +} diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/FieldContext.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/FieldContext.java index 81c1121d15a92..17a3c49fd510d 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/FieldContext.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/FieldContext.java @@ -37,33 +37,55 @@ * * @return true if attribute is required else returns false */ - public boolean required() default false; + boolean required() default false; /** * binds numeric value's lower bound * * @return minimum value of the field */ - public long minValue() default Long.MIN_VALUE; + long minValue() default Long.MIN_VALUE; /** * binds numeric value's upper bound * * @return maximum value of the field */ - public long maxValue() default Long.MAX_VALUE; + long maxValue() default Long.MAX_VALUE; /** * binds character length of text * * @return character length of field */ - public int maxCharLength() default Integer.MAX_VALUE; + int maxCharLength() default Integer.MAX_VALUE; /** * allow field to be updated dynamically * * @return */ - public boolean dynamic() default false; + boolean dynamic() default false; + + /** + * Category to group settings. + * + * @return category name + */ + String category() default ""; + + /** + * Documentation of the settings. + * + * @return the documentation of the settings. + */ + String doc() default ""; + + /** + * Whether the setting is deprecated or not. + * + * @return true if the setting is deprecated, otherwise false. + */ + boolean deprecated() default false; + } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/PropertiesContext.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/PropertiesContext.java new file mode 100644 index 0000000000000..27f42775451ad --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/PropertiesContext.java @@ -0,0 +1,33 @@ +/** + * 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.common.configuration; + +/** + * Documentation Annotation for Properties. + */ +public @interface PropertiesContext { + + /** + * Return list of properties. + * + * @return list of fields + */ + PropertyContext[] properties(); + +} diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/PropertyContext.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/PropertyContext.java new file mode 100644 index 0000000000000..59657e728b335 --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/common/configuration/PropertyContext.java @@ -0,0 +1,40 @@ +/** + * 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.common.configuration; + +/** + * Annotation for a given property. + */ +public @interface PropertyContext { + + /** + * Key of the property. + * + * @return key of the property. + */ + String key(); + + /** + * Documentation of the property. + * + * @return documentation of the property. + */ + FieldContext doc(); + +} diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java index f694df1579ee6..45eba27c9d1ac 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java @@ -32,7 +32,10 @@ import lombok.Getter; import lombok.Setter; import org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider; +import org.apache.pulsar.common.configuration.Category; import org.apache.pulsar.common.configuration.FieldContext; +import org.apache.pulsar.common.configuration.PropertiesContext; +import org.apache.pulsar.common.configuration.PropertyContext; import org.apache.pulsar.common.configuration.PulsarConfiguration; import org.slf4j.Logger; @@ -43,109 +46,286 @@ public class ProxyConfiguration implements PulsarConfiguration { private final static Logger log = LoggerFactory.getLogger(ProxyConfiguration.class); - // Local-Zookeeper quorum connection string + @Category + private static final String CATEGORY_SERVER = "Server"; + @Category + private static final String CATEGORY_BROKER_DISCOVERY = "Broker Discovery"; + @Category + private static final String CATEGORY_AUTHENTICATION = "Proxy Authentication"; + @Category + private static final String CATEGORY_AUTHORIZATION = "Proxy Authorization"; + @Category( + description = "the settings are for configuring how proxies authenticates with Pulsar brokers" + ) + private static final String CATEGORY_CLIENT_AUTHENTICATION = "Broker Client Authorization"; + @Category + private static final String CATEGORY_RATE_LIMITING = "RateLimiting"; + @Category + private static final String CATEGORY_TLS = "TLS"; + @Category + private static final String CATEGORY_TOKEN_AUTH = "Token Authentication Provider"; + @Category + private static final String CATEGORY_HTTP = "HTTP"; + + @FieldContext( + category = CATEGORY_BROKER_DISCOVERY, + doc = "The ZooKeeper quorum connection string (as a comma-separated list)" + ) private String zookeeperServers; + @FieldContext( + category = CATEGORY_BROKER_DISCOVERY, + doc = "Configuration store connection string (as a comma-separated list)" + ) + private String configurationStoreServers; + @FieldContext( + category = CATEGORY_BROKER_DISCOVERY, + doc = "Global ZooKeeper quorum connection string (as a comma-separated list)" + ) @Deprecated - // Global-Zookeeper quorum connection string private String globalZookeeperServers; - // Configuration Store connection string - private String configurationStoreServers; - - // ZooKeeper session timeout + @FieldContext( + category = CATEGORY_BROKER_DISCOVERY, + doc = "ZooKeeper session timeout (in milliseconds)" + ) private int zookeeperSessionTimeoutMs = 30_000; - // if Service Discovery is Disabled this url should point to the discovery service provider. + @FieldContext( + category = CATEGORY_BROKER_DISCOVERY, + doc = "The service url points to the broker cluster" + ) private String brokerServiceURL; + @FieldContext( + category = CATEGORY_BROKER_DISCOVERY, + doc = "The tls service url points to the broker cluster" + ) private String brokerServiceURLTLS; - // These settings are unnecessary if `zookeeperServers` is specified + @FieldContext( + category = CATEGORY_BROKER_DISCOVERY, + doc = "The web service url points to the broker cluster" + ) private String brokerWebServiceURL; + @FieldContext( + category = CATEGORY_BROKER_DISCOVERY, + doc = "The tls web service url points to the broker cluster" + ) private String brokerWebServiceURLTLS; - // function worker web services + @FieldContext( + category = CATEGORY_BROKER_DISCOVERY, + doc = "The web service url points to the function worker cluster." + + " Only configure it when you setup function workers in a separate cluster" + ) private String functionWorkerWebServiceURL; + @FieldContext( + category = CATEGORY_BROKER_DISCOVERY, + doc = "The tls web service url points to the function worker cluster." + + " Only configure it when you setup function workers in a separate cluster" + ) private String functionWorkerWebServiceURLTLS; - // Port to use to server binary-proto request + @FieldContext( + category = CATEGORY_SERVER, + doc = "The port for serving binary protobuf request" + ) private int servicePort = 6650; - // Port to use to server binary-proto-tls request + @FieldContext( + category = CATEGORY_SERVER, + doc = "The port for serving tls secured binary protobuf request" + ) private int servicePortTls = 6651; - // Port to use to server HTTP request + @FieldContext( + category = CATEGORY_SERVER, + doc = "The port for serving http requests" + ) private int webServicePort = 8080; - // Port to use to server HTTPS request + @FieldContext( + category = CATEGORY_SERVER, + doc = "The port for serving https requests" + ) private int webServicePortTls = 8443; - // Path for the file used to determine the rotation status for the broker - // when responding to service discovery health checks + @FieldContext( + category = CATEGORY_SERVER, + doc = "Path for the file used to determine the rotation status for the proxy instance" + + " when responding to service discovery health checks" + ) private String statusFilePath; - // Role names that are treated as "super-user", meaning they will be able to - // do all admin operations and publish/consume from all topics + @FieldContext( + category = CATEGORY_AUTHORIZATION, + doc = "A list of role names (a comma-separated list of strings) that are treated as" + + " `super-user`, meaning they will be able to do all admin operations and publish" + + " & consume from all topics" + ) private Set superUserRoles = Sets.newTreeSet(); - // Enable authentication + @FieldContext( + category = CATEGORY_AUTHENTICATION, + doc = "Whether authentication is enabled for the Pulsar proxy" + ) private boolean authenticationEnabled = false; - // Authentication provider name list, which is a list of class names + @FieldContext( + category = CATEGORY_AUTHENTICATION, + doc = "Authentication provider name list (a comma-separated list of class names" + ) private Set authenticationProviders = Sets.newTreeSet(); - // Enforce authorization + @FieldContext( + category = CATEGORY_AUTHORIZATION, + doc = "Whether authorization is enforced by the Pulsar proxy" + ) private boolean authorizationEnabled = false; - // Authorization provider fully qualified class-name + @FieldContext( + category = CATEGORY_AUTHORIZATION, + doc = "Authorization provider as a fully qualified class name" + ) private String authorizationProvider = PulsarAuthorizationProvider.class.getName(); - // Forward client authData to Broker for re authorization - // make sure authentication is enabled for this to take effect + @FieldContext( + category = CATEGORY_AUTHORIZATION, + doc = "Whether client authorization credentials are forwarded to the broker for re-authorization." + + "Authentication must be enabled via configuring `authenticationEnabled` to be true for this" + + "to take effect" + ) private boolean forwardAuthorizationCredentials = false; - // Max concurrent inbound Connections + @FieldContext( + category = CATEGORY_RATE_LIMITING, + doc = "Max concurrent inbound connections. The proxy will reject requests beyond that" + ) private int maxConcurrentInboundConnections = 10000; - // Max concurrent outbound Connections + @FieldContext( + category = CATEGORY_RATE_LIMITING, + doc = "Max concurrent lookup requests. The proxy will reject requests beyond that" + ) private int maxConcurrentLookupRequests = 50000; - // Authentication settings of the proxy itself. Used to connect to brokers + @FieldContext( + category = CATEGORY_CLIENT_AUTHENTICATION, + doc = "The authentication plugin used by the Pulsar proxy to authenticate with Pulsar brokers" + ) private String brokerClientAuthenticationPlugin; + @FieldContext( + category = CATEGORY_CLIENT_AUTHENTICATION, + doc = "The authentication parameters used by the Pulsar proxy to authenticate with Pulsar brokers" + ) private String brokerClientAuthenticationParameters; + @FieldContext( + category = CATEGORY_CLIENT_AUTHENTICATION, + doc = "The path to trusted certificates used by the Pulsar proxy to authenticate with Pulsar brokers" + ) private String brokerClientTrustCertsFilePath; + @FieldContext( + category = CATEGORY_CLIENT_AUTHENTICATION, + doc = "Whether TLS is enabled when communicating with Pulsar brokers" + ) + private boolean tlsEnabledWithBroker = false; + /***** --- TLS --- ****/ - // Enable TLS for the proxy handler + @FieldContext( + category = CATEGORY_TLS, + doc = "Whether TLS is enabled for the proxy" + ) private boolean tlsEnabledInProxy = false; - // Enable TLS when talking with the brokers - private boolean tlsEnabledWithBroker = false; - - // Path for the TLS certificate file + @FieldContext( + category = CATEGORY_TLS, + doc = "Path for the TLS certificate file" + ) private String tlsCertificateFilePath; - // Path for the TLS private key file + @FieldContext( + category = CATEGORY_TLS, + doc = "Path for the TLS private key file" + ) private String tlsKeyFilePath; - // Path for the trusted TLS certificate file + @FieldContext( + category = CATEGORY_TLS, + doc = "Path for the trusted TLS certificate file.\n\n" + + "This cert is used to verify that any certs presented by connecting clients" + + " are signed by a certificate authority. If this verification fails, then the" + + " certs are untrusted and the connections are dropped" + ) private String tlsTrustCertsFilePath; - // Accept untrusted TLS certificate from client + @FieldContext( + category = CATEGORY_TLS, + doc = "Accept untrusted TLS certificate from client.\n\n" + + "If true, a client with a cert which cannot be verified with the `tlsTrustCertsFilePath`" + + " cert will be allowed to connect to the server, though the cert will not be used for" + + " client authentication" + ) private boolean tlsAllowInsecureConnection = false; - // Validates hostname when proxy creates tls connection with broker + @FieldContext( + category = CATEGORY_TLS, + doc = "Whether the hostname is validated when the proxy creates a TLS connection with brokers" + ) private boolean tlsHostnameVerificationEnabled = false; - // Specify the tls protocols the broker will use to negotiate during TLS Handshake. - // Example:- [TLSv1.2, TLSv1.1, TLSv1] + @FieldContext( + category = CATEGORY_TLS, + doc = "Specify the tls protocols the broker will use to negotiate during TLS handshake" + + " (a comma-separated list of protocol names).\n\n" + + "Examples:- [TLSv1.2, TLSv1.1, TLSv1]" + ) private Set tlsProtocols = Sets.newTreeSet(); - // Specify the tls cipher the broker will use to negotiate during TLS Handshake. - // Example:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256] + @FieldContext( + category = CATEGORY_TLS, + doc = "Specify the tls cipher the broker will use to negotiate during TLS Handshake" + + " (a comma-separated list of ciphers).\n\n" + + "Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256]" + ) private Set tlsCiphers = Sets.newTreeSet(); - // Specify whether Client certificates are required for TLS - // Reject the Connection if the Client Certificate is not trusted. + @FieldContext( + category = CATEGORY_TLS, + doc = "Whether client certificates are required for TLS.\n\n" + + " Connections are rejected if the client certificate isn't trusted" + ) private boolean tlsRequireTrustedClientCertOnConnect = false; - // Http redirects to redirect to non-pulsar services + @FieldContext( + category = CATEGORY_HTTP, + doc = "Http directs to redirect to non-pulsar services" + ) private Set httpReverseProxyConfigs = Sets.newHashSet(); - // Http output buffer size. The amount of data that will be buffered for http requests - // before it is flushed to the channel. A larger buffer size may result in higher http throughput - // though it may take longer for the client to see data. - // If using HTTP streaming via the reverse proxy, this should be set to the minimum value, 1, - // so that clients see the data as soon as possible. - @FieldContext(minValue = 1) + @FieldContext( + minValue = 1, + category = CATEGORY_HTTP, + doc = "Http output buffer size.\n\n" + + "The amount of data that will be buffered for http requests " + + "before it is flushed to the channel. A larger buffer size may " + + "result in higher http throughput though it may take longer for " + + "the client to see data. If using HTTP streaming via the reverse " + + "proxy, this should be set to the minimum value, 1, so that clients " + + "see the data as soon as possible." + ) private int httpOutputBufferSize = 32*1024; + @PropertiesContext( + properties = { + @PropertyContext( + key = "tokenPublicKey", + doc = @FieldContext( + category = CATEGORY_TOKEN_AUTH, + doc = "Asymmetric public/private key pair.\n\n" + + "Configure the public key to be used to validate auth tokens" + + " The key can be specified like:\n\n" + + "tokenPublicKey=data:base64,xxxxxxxxx\n" + + "tokenPublicKey=file:///my/public.key") + ), + @PropertyContext( + key = "tokenSecretKey", + doc = @FieldContext( + category = CATEGORY_TOKEN_AUTH, + doc = "Symmetric key.\n\n" + + "Configure the secret key to be used to validate auth tokens" + + "The key can be specified like:\n\n" + + "tokenSecretKey=data:base64,xxxxxxxxx\n" + + "tokenSecretKey=file:///my/secret.key") + ) + } + ) private Properties properties = new Properties(); public Properties getProperties() {