PulsarConnectorConfig.java 10.7 KB
Newer Older
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20
/**
 * 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.sql.presto;

21
import com.fasterxml.jackson.databind.ObjectMapper;
22 23
import io.airlift.configuration.Config;
import org.apache.pulsar.client.admin.PulsarAdmin;
24 25
import org.apache.pulsar.client.admin.PulsarAdminBuilder;
import org.apache.pulsar.client.api.Authentication;
26
import org.apache.pulsar.client.api.PulsarClientException;
27
import org.apache.bookkeeper.stats.NullStatsProvider;
28
import org.apache.pulsar.common.naming.NamedEntity;
29
import org.apache.pulsar.common.protocol.Commands;
30 31

import javax.validation.constraints.NotNull;
32
import java.io.IOException;
33 34
import java.util.HashMap;
import java.util.Map;
35
import java.util.regex.Matcher;
36 37 38 39 40 41

public class PulsarConnectorConfig implements AutoCloseable {

    private String brokerServiceUrl = "http://localhost:8080";
    private String zookeeperUri = "localhost:2181";
    private int entryReadBatchSize = 100;
42 43 44
    private int targetNumSplits = 2;
    private int maxSplitMessageQueueSize = 10000;
    private int maxSplitEntryQueueSize = 1000;
45
    private int maxMessageSize = Commands.DEFAULT_MAX_MESSAGE_SIZE;
46
    private String statsProvider = NullStatsProvider.class.getName();
47

48
    private Map<String, String> statsProviderConfigs = new HashMap<>();
49 50 51 52 53
    private String authPluginClassName;
    private String authParams;
    private String tlsTrustCertsFilePath;
    private Boolean tlsAllowInsecureConnection;
    private Boolean tlsHostnameVerificationEnable;
54

55 56 57
    private boolean namespaceDelimiterRewriteEnable = false;
    private String rewriteNamespaceDelimiter = "/";

58 59 60 61 62 63
    /**** --- Ledger Offloading --- ****/
    private String managedLedgerOffloadDriver = null;
    private int managedLedgerOffloadMaxThreads = 2;
    private String offloadersDirectory = "./offloaders";
    private Map<String, String> offloaderProperties = new HashMap<>();

64 65 66 67 68 69 70 71 72 73 74 75 76
    private PulsarAdmin pulsarAdmin;

    @NotNull
    public String getBrokerServiceUrl() {
        return brokerServiceUrl;
    }

    @Config("pulsar.broker-service-url")
    public PulsarConnectorConfig setBrokerServiceUrl(String brokerServiceUrl) {
        this.brokerServiceUrl = brokerServiceUrl;
        return this;
    }

77 78 79 80 81 82 83 84 85 86
    @Config("pulsar.max-message-size")
    public PulsarConnectorConfig setMaxMessageSize(int maxMessageSize) {
        this.maxMessageSize = maxMessageSize;
        return this;
    }

    public int getMaxMessageSize() {
        return this.maxMessageSize;
    }

87 88 89 90 91 92 93 94 95 96 97 98
    @NotNull
    public String getZookeeperUri() {
        return this.zookeeperUri;
    }

    @Config("pulsar.zookeeper-uri")
    public PulsarConnectorConfig setZookeeperUri(String zookeeperUri) {
        this.zookeeperUri = zookeeperUri;
        return this;
    }

    @NotNull
99
    public int getMaxEntryReadBatchSize() {
100 101 102
        return this.entryReadBatchSize;
    }

103 104
    @Config("pulsar.max-entry-read-batch-size")
    public PulsarConnectorConfig setMaxEntryReadBatchSize(int batchSize) {
105 106 107 108 109 110 111 112 113 114 115 116 117 118 119
        this.entryReadBatchSize = batchSize;
        return this;
    }

    @NotNull
    public int getTargetNumSplits() {
        return this.targetNumSplits;
    }

    @Config("pulsar.target-num-splits")
    public PulsarConnectorConfig setTargetNumSplits(int targetNumSplits) {
        this.targetNumSplits = targetNumSplits;
        return this;
    }

120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141
    @NotNull
    public int getMaxSplitMessageQueueSize() {
        return this.maxSplitMessageQueueSize;
    }

    @Config("pulsar.max-split-message-queue-size")
    public PulsarConnectorConfig setMaxSplitMessageQueueSize(int maxSplitMessageQueueSize) {
        this.maxSplitMessageQueueSize = maxSplitMessageQueueSize;
        return this;
    }

    @NotNull
    public int getMaxSplitEntryQueueSize() {
        return this.maxSplitEntryQueueSize;
    }

    @Config("pulsar.max-split-entry-queue-size")
    public PulsarConnectorConfig setMaxSplitEntryQueueSize(int maxSplitEntryQueueSize) {
        this.maxSplitEntryQueueSize = maxSplitEntryQueueSize;
        return this;
    }

142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158
    @NotNull
    public String getStatsProvider() {
        return statsProvider;
    }

    @Config("pulsar.stats-provider")
    public PulsarConnectorConfig setStatsProvider(String statsProvider) {
        this.statsProvider = statsProvider;
        return this;
    }

    @NotNull
    public Map<String, String> getStatsProviderConfigs() {
        return statsProviderConfigs;
    }

    @Config("pulsar.stats-provider-configs")
159 160
    public PulsarConnectorConfig setStatsProviderConfigs(String statsProviderConfigs) throws IOException {
        this.statsProviderConfigs = new ObjectMapper().readValue(statsProviderConfigs, Map.class);
161 162 163
        return this;
    }

164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190
    public String getRewriteNamespaceDelimiter() {
        return rewriteNamespaceDelimiter;
    }

    @Config("pulsar.rewrite-namespace-delimiter")
    public PulsarConnectorConfig setRewriteNamespaceDelimiter(String rewriteNamespaceDelimiter) {
        Matcher m = NamedEntity.NAMED_ENTITY_PATTERN.matcher(rewriteNamespaceDelimiter);
        if (m.matches()) {
            throw new IllegalArgumentException(
                    "Can't use " + rewriteNamespaceDelimiter + "as delimiter, "
                            + "because delimiter must contain characters which name of namespace not allowed"
            );
        }
        this.rewriteNamespaceDelimiter = rewriteNamespaceDelimiter;
        return this;
    }

    public boolean getNamespaceDelimiterRewriteEnable() {
        return namespaceDelimiterRewriteEnable;
    }

    @Config("pulsar.namespace-delimiter-rewrite-enable")
    public PulsarConnectorConfig setNamespaceDelimiterRewriteEnable(boolean namespaceDelimiterRewriteEnable) {
        this.namespaceDelimiterRewriteEnable = namespaceDelimiterRewriteEnable;
        return this;
    }

191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233
    /**** --- Ledger Offloading --- ****/

    public int getManagedLedgerOffloadMaxThreads() {
        return this.managedLedgerOffloadMaxThreads;
    }

    @Config("pulsar.managed-ledger-offload-max-threads")
    public PulsarConnectorConfig setManagedLedgerOffloadMaxThreads(int managedLedgerOffloadMaxThreads) throws IOException {
        this.managedLedgerOffloadMaxThreads = managedLedgerOffloadMaxThreads;
        return this;
    }

    public String getManagedLedgerOffloadDriver() {
        return this.managedLedgerOffloadDriver;
    }

    @Config("pulsar.managed-ledger-offload-driver")
    public PulsarConnectorConfig setManagedLedgerOffloadDriver(String managedLedgerOffloadDriver) throws IOException {
        this.managedLedgerOffloadDriver = managedLedgerOffloadDriver;
        return this;
    }

    public String getOffloadersDirectory() {
        return this.offloadersDirectory;
    }


    @Config("pulsar.offloaders-directory")
    public PulsarConnectorConfig setOffloadersDirectory(String offloadersDirectory) throws IOException {
        this.offloadersDirectory = offloadersDirectory;
        return this;
    }

    public Map<String, String> getOffloaderProperties() {
        return this.offloaderProperties;
    }

    @Config("pulsar.offloader-properties")
    public PulsarConnectorConfig setOffloaderProperties(String offloaderProperties) throws IOException {
        this.offloaderProperties = new ObjectMapper().readValue(offloaderProperties, Map.class);
        return this;
    }

234 235 236 237
    /**** --- Authentication --- ****/

    public String getAuthPlugin() {
        return this.authPluginClassName;
238 239
    }

240 241 242
    @Config("pulsar.auth-plugin")
    public PulsarConnectorConfig setAuthPlugin(String authPluginClassName) throws IOException {
        this.authPluginClassName = authPluginClassName;
243 244 245
        return this;
    }

246 247
    public String getAuthParams() {
        return this.authParams;
248 249
    }

250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282
    @Config("pulsar.auth-params")
    public PulsarConnectorConfig setAuthParams(String authParams) throws IOException {
        this.authParams = authParams;
        return this;
    }

    public Boolean isTlsAllowInsecureConnection() {
        return tlsAllowInsecureConnection;
    }

    @Config("pulsar.tls-allow-insecure-connection")
    public PulsarConnectorConfig setTlsAllowInsecureConnection(boolean tlsAllowInsecureConnection) {
        this.tlsAllowInsecureConnection = tlsAllowInsecureConnection;
        return this;
    }

    public Boolean isTlsHostnameVerificationEnable() {
        return tlsHostnameVerificationEnable;
    }

    @Config("pulsar.tls-hostname-verification-enable")
    public PulsarConnectorConfig setTlsHostnameVerificationEnable(boolean tlsHostnameVerificationEnable) {
        this.tlsHostnameVerificationEnable = tlsHostnameVerificationEnable;
        return this;
    }

    public String getTlsTrustCertsFilePath() {
        return tlsTrustCertsFilePath;
    }

    @Config("pulsar.tls-trust-cert-file-path")
    public PulsarConnectorConfig setTlsTrustCertsFilePath(String tlsTrustCertsFilePath) {
        this.tlsTrustCertsFilePath = tlsTrustCertsFilePath;
283 284 285
        return this;
    }

286 287 288
    @NotNull
    public PulsarAdmin getPulsarAdmin() throws PulsarClientException {
        if (this.pulsarAdmin == null) {
289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307
            PulsarAdminBuilder builder = PulsarAdmin.builder();

            if (getAuthPlugin() != null) {
                builder.authentication(getAuthPlugin(), getAuthParams());
            }

            if (isTlsAllowInsecureConnection() != null) {
                builder.allowTlsInsecureConnection(isTlsAllowInsecureConnection());
            }

            if (isTlsHostnameVerificationEnable() != null) {
                builder.enableTlsHostnameVerification(isTlsHostnameVerificationEnable());
            }

            if (getTlsTrustCertsFilePath() != null) {
                builder.tlsTrustCertsFilePath(getTlsTrustCertsFilePath());
            }

            this.pulsarAdmin = builder.serviceHttpUrl(getBrokerServiceUrl()).build();
308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323
        }
        return this.pulsarAdmin;
    }

    @Override
    public void close() throws Exception {
        this.pulsarAdmin.close();
    }

    @Override
    public String toString() {
        return "PulsarConnectorConfig{" +
                "brokerServiceUrl='" + brokerServiceUrl + '\'' +
                '}';
    }
}