You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by zh...@apache.org on 2020/05/08 12:33:46 UTC

[pulsar] branch branch-2.5 updated (c36170c -> 9e72dfb)

This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a change to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git.


    from c36170c  [python client]Support generate pulsar-client for python3.8 (#6741)
     new d39113a  [hotfix]Python function protobuf missing field and broker test failed (#6641)
     new 7f0cd1a  Retention policy should be respected when there is no traffic (#6676)
     new 8035ae0  ISSUE-6612 FIX: parse long field in GenricJsonRecord (#6612) (#6622)
     new 7e951e3  Fix typo in io-debezium-source.md (#6729)
     new 40b6c91  Fixed double delete on a namespace (#6713)
     new 0fd02bd  Fix deadlock by consumer and reader (#6728)
     new e75e0a0  fix_admin_getIndividualMsgsFromBatch_bug (#6715)
     new 14a1a2b  [issue 6694][AVRO ENCODE] Reset cursor if message encode fails. (#6695)
     new 0d5d0c1  [pulsar-broker] avoid backpressure by skipping dispatching if consumer channel is not writable (#6740)
     new 8c2d1cf  fix_msgMetadata_not_recycle (#6745)
     new 219bec9  Bump netty version to 4.1.48.Final (#6746)
     new 3cfa938  Ensure that all dangling consumers are cleaned up during failures (#6778)
     new d1fe636  Fix localrunner netty dependency issue and add default log4j2 config file (#6779)
     new 48b9613  Extract common method in tiered storage to managed-ledger module (#6533)
     new c3fa923e [Issue 6283][tiered-storage] Offload policies per namespace (#6422)
     new 74c668a  Avoid prefetch too much data when offloading data to HDFS (#6717)
     new 0ce7a66  Change nodurable cursor to active (#6769)
     new 4cb536b  [broker] register loadbalance znode should attempt to wait until session expired (#6788)
     new 5b74f81  Fix check backlogged cursors without consumer (#6766)
     new b0ca8e8  Fix some empty message related problems in the compacted topic. (#6795)
     new 8783907  Make messageReceiveTimeoutMs in the PulsarConsumerSource configurable (#6783)
     new 0f1636a  [C++] Fix message id is always the default value in send callback (#6812)
     new e135afb  [PY] Fix serialization of enums with json/avro schemas (#6808)
     new 1c02945  [Issue #5395][broker] Implement AutoTopicCreation by namespace override (#6471)
     new a2d5d5b  Avoid creating partitioned topic for partition name (#6846)
     new 561328f  Add null check for offload policy(#6775)
     new d8b6e3f  Make SchemaStorage accessible in Offloader (#6567)
     new cadc852  Fix Get schema by version can get the deleted schema info #6754 (#6764)
     new 53f74c7  Fixing JDBC sink to handle null fields. Also added new unit tests (#6848)
     new 1856f2f  [Broker] Handle all exceptions from `topic.addProducer` (#6881)
     new 4063f79  change log level to debug for bouncy castle load (#6860)
     new 121cf08  [broker] Increase timeout for loading topics (#6750)
     new 54329f2  Fix validation of function's update (#6888)
     new 2aaf4d2  Fixed dashboard start failed (#6857)
     new 026a610  Add note for bookkeeper explicit LAC (#6908)
     new 55d5430  Support function with format: Function<I, CompletableFuture<O>> (#6684)
     new 77c7f1c  [pulsar-client] Add support to load tls certs/key dynamically from inputstream (#6760)
     new 9e72dfb  Add Tls with keystore type config support (#6853)

The 38 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .github/workflows/ci-integration-process.yaml      |   8 +
 .github/workflows/ci-integration-thread.yaml       |   8 +
 conf/broker.conf                                   |  63 ++++
 conf/client.conf                                   |  11 +
 conf/standalone.conf                               | 100 +++++-
 dashboard/Dockerfile                               |   5 +-
 dashboard/init-postgres.sh                         |   2 +-
 distribution/server/src/assemble/LICENSE.bin.txt   |  34 +-
 .../bookkeeper/mledger/LedgerOffloaderFactory.java |  18 ++
 .../apache/bookkeeper/mledger/ManagedLedger.java   |  13 +-
 .../bookkeeper/mledger/ManagedLedgerConfig.java    |  44 ---
 .../mledger/ManagedLedgerFactoryConfig.java        |   5 -
 .../bookkeeper/mledger/impl/ManagedLedgerImpl.java | 107 +++----
 .../mledger/impl/NonDurableCursorImpl.java         |  15 -
 .../bookkeeper/mledger/offload/OffloadUtils.java   |  92 ++++++
 .../bookkeeper/mledger/impl/ManagedCursorTest.java |  11 +
 .../bookkeeper/mledger/impl/ManagedLedgerTest.java |  80 -----
 .../mledger/impl/OffloadLedgerDeleteTest.java      |   6 +-
 .../bookkeeper/mledger/impl/OffloadPrefixTest.java |  22 +-
 pom.xml                                            |   4 +-
 .../apache/pulsar/broker/ServiceConfiguration.java | 131 +++++++-
 .../authentication/AuthenticationDataHttps.java    |   1 -
 .../OneStageAuthenticationState.java               |   2 +-
 .../java/org/apache/pulsar/PulsarStandalone.java   |  25 +-
 .../pulsar/broker/ManagedLedgerClientFactory.java  |   1 -
 .../org/apache/pulsar/broker/PulsarService.java    |  53 ++-
 .../pulsar/broker/admin/impl/NamespacesBase.java   | 154 ++++++++-
 .../apache/pulsar/broker/admin/v2/Namespaces.java  |  38 +++
 .../loadbalance/impl/ModularLoadManagerImpl.java   |  37 +--
 .../loadbalance/impl/SimpleLoadManagerImpl.java    |  30 +-
 .../pulsar/broker/service/BrokerService.java       | 131 ++++++--
 .../org/apache/pulsar/broker/service/Producer.java |   5 +-
 .../broker/service/PulsarChannelInitializer.java   |  62 ++--
 .../apache/pulsar/broker/service/PulsarStats.java  |   2 +-
 .../apache/pulsar/broker/service/ServerCnx.java    |   6 +-
 .../org/apache/pulsar/broker/service/Topic.java    |   6 +
 .../service/nonpersistent/NonPersistentTopic.java  |   5 +
 .../PersistentDispatcherMultipleConsumers.java     |  27 +-
 ...istentStickyKeyDispatcherMultipleConsumers.java |   7 +-
 .../broker/service/persistent/PersistentTopic.java |  96 +++---
 .../service/schema/BookkeeperSchemaStorage.java    |  15 +-
 .../schema/BookkeeperSchemaStorageFactory.java     |   1 +
 .../service/schema/SchemaRegistryService.java      |  32 +-
 .../service/schema/SchemaRegistryServiceImpl.java  |  38 ++-
 .../service/schema/SchemaStorageFactory.java       |   1 +
 .../org/apache/pulsar/broker/web/WebService.java   |  35 +-
 .../pulsar/client/impl/RawBatchConverter.java      |  10 +-
 .../pulsar/compaction/CompactedTopicImpl.java      |  46 +--
 .../pulsar/compaction/TwoPhaseCompactor.java       |  19 +-
 .../pulsar/broker/admin/AdminApiOffloadTest.java   |   2 +-
 .../apache/pulsar/broker/admin/AdminApiTest.java   |  19 ++
 .../pulsar/broker/admin/AdminTopicApiTest.java     | 103 ++++++
 .../apache/pulsar/broker/admin/NamespacesTest.java | 112 ++++++-
 .../pulsar/broker/admin/TopicAutoCreationTest.java |  88 +++++
 .../broker/auth/MockedPulsarServiceBaseTest.java   |   2 +-
 .../BrokerServiceAutoTopicCreationTest.java        | 199 ++++++++++--
 .../broker/service/ConsumedLedgersTrimTest.java    |  93 ++++++
 .../broker/service/PersistentTopicE2ETest.java     |  42 +++
 .../pulsar/broker/service/PersistentTopicTest.java | 142 ++++++++-
 .../broker/service/schema/SchemaServiceTest.java   |   4 +-
 .../client/api/SimpleProducerConsumerTest.java     |   4 +-
 .../pulsar/client/api/TlsProducerConsumerTest.java | 111 +++++++
 .../client/impl/AdminApiKeyStoreTlsAuthTest.java   | 229 +++++++++++++
 .../KeyStoreTlsProducerConsumerTestWithAuth.java   | 267 ++++++++++++++++
 ...KeyStoreTlsProducerConsumerTestWithoutAuth.java | 255 +++++++++++++++
 .../apache/pulsar/client/impl/KeyStoreTlsTest.java |  80 +++++
 .../apache/pulsar/client/impl/RawReaderTest.java   |  19 +-
 .../apache/pulsar/compaction/CompactionTest.java   | 340 ++++++++++++++++++++
 .../authentication/keystoretls/broker.keystore.jks | Bin 0 -> 2767 bytes
 .../keystoretls/broker.truststore.jks              | Bin 0 -> 731 bytes
 .../keystoretls/brokerKeyStorePW.txt               |   1 +
 .../keystoretls/brokerTrustStorePW.txt             |   1 +
 .../authentication/keystoretls/client.keystore.jks | Bin 0 -> 2767 bytes
 .../keystoretls/client.truststore.jks              | Bin 0 -> 731 bytes
 .../keystoretls/clientKeyStorePW.txt               |   1 +
 .../keystoretls/clientTrustStorePW.txt             |   1 +
 .../org/apache/pulsar/client/admin/Namespaces.java |  59 +++-
 .../pulsar/client/admin/PulsarAdminBuilder.java    |  60 +++-
 .../client/admin/internal/NamespacesImpl.java      |  29 +-
 .../admin/internal/PulsarAdminBuilderImpl.java     |  49 ++-
 .../pulsar/client/admin/internal/TopicsImpl.java   |   4 +-
 .../admin/internal/http/AsyncHttpConnector.java    |  46 ++-
 .../client/api/AuthenticationDataProvider.java     |   9 +
 .../apache/pulsar/client/api/ClientBuilder.java    |  64 ++++
 .../apache/pulsar/client/api/KeyStoreParams.java   |  20 +-
 pulsar-client-cpp/lib/BatchMessageContainer.cc     |  12 +-
 pulsar-client-cpp/lib/BatchMessageContainer.h      |  10 +-
 pulsar-client-cpp/lib/ProducerImpl.cc              |   2 +-
 pulsar-client-cpp/python/pulsar/schema/__init__.py |   7 +-
 pulsar-client-cpp/python/pulsar/schema/schema.py   |  18 +-
 pulsar-client-cpp/python/schema_test.py            |  57 ++++
 pulsar-client-cpp/tests/BasicEndToEndTest.cc       |  37 +++
 pulsar-client-cpp/tests/BatchMessageTest.cc        |  50 ++-
 .../pulsar/admin/cli/PulsarAdminToolTest.java      |  14 +-
 .../org/apache/pulsar/admin/cli/CmdNamespaces.java |  92 +++++-
 .../apache/pulsar/admin/cli/PulsarAdminTool.java   |  22 +-
 .../apache/pulsar/client/cli/PulsarClientTool.java |  18 ++
 .../pulsar/client/impl/ClientBuilderImpl.java      |  47 ++-
 .../org/apache/pulsar/client/impl/HttpClient.java  |  68 ++--
 .../pulsar/client/impl/HttpLookupService.java      |   3 +-
 .../org/apache/pulsar/client/impl/MessageImpl.java |   2 +-
 .../client/impl/PulsarChannelInitializer.java      |  39 ++-
 .../impl/auth/AuthenticationDataKeyStoreTls.java   |  32 +-
 .../client/impl/auth/AuthenticationDataTls.java    |  47 ++-
 .../impl/auth/AuthenticationKeyStoreTls.java       | 136 ++++++++
 .../pulsar/client/impl/auth/AuthenticationTls.java |  20 +-
 .../client/impl/conf/ClientConfigurationData.java  |  15 +
 .../impl/schema/generic/GenericJsonRecord.java     |  12 +-
 .../client/impl/schema/writer/AvroWriter.java      |  10 +-
 .../pulsar/client/impl/schema/AvroSchemaTest.java  |  23 ++
 .../impl/schema/generic/GenericJsonRecordTest.java |  57 ++++
 pulsar-common/pom.xml                              |   4 +
 .../pulsar/common/api/raw/MessageParser.java       |   2 +-
 .../pulsar/common/api/raw/RawMessageIdImpl.java    |   3 +
 .../pulsar/common/functions/FunctionConfig.java    |   3 +
 .../policies/data/AutoTopicCreationOverride.java   |  89 ++++++
 .../common/policies/data/OffloadPolicies.java      |  23 +-
 .../pulsar/common/policies/data/Policies.java      |   6 +-
 .../data/TopicType.java}                           |  36 ++-
 .../common/protocol}/schema/SchemaStorage.java     |   6 +-
 .../common/protocol}/schema/StoredSchema.java      |  12 +-
 .../common/util/ClientSslContextRefresher.java     |  67 ----
 .../common/util/DefaultSslContextBuilder.java      |  18 +-
 .../util/NettyClientSslContextRefresher.java       |  74 +++++
 ...lder.java => NettyServerSslContextBuilder.java} |  33 +-
 .../pulsar/common/util/SearchBcNarUtils.java       |   6 +-
 .../apache/pulsar/common/util/SecurityUtility.java |  54 +++-
 .../common/util/SslContextAutoRefreshBuilder.java  |  46 +--
 .../util/keystoretls/KeyStoreSSLContext.java       | 355 +++++++++++++++++++++
 .../util/keystoretls/NetSslContextBuilder.java     |  92 ++++++
 .../NettySSLContextAutoRefreshBuilder.java         | 144 +++++++++
 .../keystoretls/SSLContextValidatorEngine.java     | 176 ++++++++++
 .../SslContextFactoryWithAutoRefresh.java          |  63 ++++
 .../common/util/keystoretls/package-info.java      |  13 +-
 .../data/AutoTopicCreationOverrideTest.java        |  63 ++++
 .../src/test/resources/broker.keystore.jks         | Bin 0 -> 2767 bytes
 .../src/test/resources/broker.truststore.jks       | Bin 0 -> 731 bytes
 .../src/test/resources/brokerKeyStorePW.txt        |   1 +
 .../src/test/resources/brokerTrustStorePW.txt      |   1 +
 pulsar-common/src/test/resources/ca-cert           |  16 +
 pulsar-common/src/test/resources/ca-cert.srl       |   1 +
 pulsar-common/src/test/resources/ca-key            |  30 ++
 pulsar-common/src/test/resources/cert-file         |  17 +
 pulsar-common/src/test/resources/cert-signed       |  22 ++
 .../src/test/resources/client.keystore.jks         | Bin 0 -> 2767 bytes
 .../src/test/resources/client.truststore.jks       | Bin 0 -> 731 bytes
 .../src/test/resources/clientKeyStorePW.txt        |   1 +
 .../src/test/resources/clientTrustStorePW.txt      |   1 +
 .../src/test/resources/old/broker.keystore.jks     | Bin 0 -> 2928 bytes
 .../src/test/resources/old/broker.truststore.jks   | Bin 0 -> 797 bytes
 .../src/test/resources/old/brokerKeyStorePW.txt    |   1 +
 .../src/test/resources/old/brokerTrustStorePW.txt  |   1 +
 .../src/test/resources/old/client.keystore.jks     | Bin 0 -> 2926 bytes
 .../src/test/resources/old/client.truststore.jks   | Bin 0 -> 797 bytes
 .../src/test/resources/old/clientKeyStorePW.txt    |   1 +
 .../src/test/resources/old/clientTrustStorePW.txt  |   1 +
 .../service/ServiceChannelInitializer.java         |  47 ++-
 .../discovery/service/server/ServerManager.java    |  35 +-
 .../discovery/service/server/ServiceConfig.java    | 222 ++-----------
 .../connectors/pulsar/PulsarConsumerSource.java    |   3 +-
 .../connectors/pulsar/PulsarSourceBuilder.java     |  16 +
 .../pulsar/functions/instance/InstanceConfig.java  |   3 +
 .../pulsar/functions/instance/JavaInstance.java    |  64 +++-
 .../functions/instance/JavaInstanceRunnable.java   |  41 +--
 .../pulsar/functions/source/PulsarSource.java      |  25 +-
 .../instance/src/main/python/Function_pb2.py       | 320 +++++++++++--------
 .../functions/instance/JavaInstanceTest.java       |  96 +++++-
 .../pulsar/functions/source/PulsarSourceTest.java  | 113 +++++--
 .../api/examples/AsyncContextFunction.java         |  59 ++++
 .../JavaNativeAsyncExclamationFunction.java        |  40 +--
 pulsar-functions/localrun-shaded/pom.xml           |  14 -
 .../org/apache/pulsar/functions/LocalRunner.java   |   2 +
 .../localrun/src/main/resources/log4j2.xml         |  33 ++
 .../functions/runtime/JavaInstanceStarter.java     |   4 +
 .../pulsar/functions/worker/WorkerConfig.java      |   6 +
 .../functions/utils/FunctionConfigUtils.java       |   2 +-
 .../functions/utils/FunctionConfigUtilsTest.java   |  15 +
 .../pulsar/functions/worker/FunctionActioner.java  |   1 +
 .../apache/pulsar/io/jdbc/JdbcAutoSchemaSink.java  |  37 ++-
 .../org/apache/pulsar/io/jdbc/JdbcSinkTest.java    | 150 ++++++++-
 .../pulsar/proxy/server/DirectProxyHandler.java    |  14 +-
 .../pulsar/proxy/server/ProxyConfiguration.java    | 104 +++++-
 .../pulsar/proxy/server/ProxyConnection.java       |  21 +-
 .../proxy/server/ServiceChannelInitializer.java    |  90 +++++-
 .../org/apache/pulsar/proxy/server/WebServer.java  |  37 ++-
 .../proxy/server/ProxyKeyStoreTlsTestWithAuth.java | 202 ++++++++++++
 .../server/ProxyKeyStoreTlsTestWithoutAuth.java    | 186 +++++++++++
 .../authentication/keystoretls/broker.keystore.jks | Bin 0 -> 2767 bytes
 .../keystoretls/broker.truststore.jks              | Bin 0 -> 731 bytes
 .../keystoretls/brokerKeyStorePW.txt               |   1 +
 .../keystoretls/brokerTrustStorePW.txt             |   1 +
 .../authentication/keystoretls/client.keystore.jks | Bin 0 -> 2767 bytes
 .../keystoretls/client.truststore.jks              | Bin 0 -> 731 bytes
 .../keystoretls/clientKeyStorePW.txt               |   1 +
 .../keystoretls/clientTrustStorePW.txt             |   1 +
 pulsar-sql/presto-distribution/LICENSE             |  32 +-
 .../java/org/apache/pulsar/zookeeper/ZkUtils.java  |  78 +++++
 site2/docs/io-debezium-source.md                   |   4 +-
 site2/docs/reference-configuration.md              |  12 +
 site2/docs/reference-pulsar-admin.md               |   2 +
 site2/docs/sql-deployment-configurations.md        |   5 +-
 .../impl/FileStoreBackedReadHandleImpl.java        |  60 +---
 .../impl/FileSystemManagedLedgerOffloader.java     | 109 ++++---
 .../offload/jcloud/impl/OffloadIndexBlockImpl.java | 181 +----------
 204 files changed, 7366 insertions(+), 1574 deletions(-)
 create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTopicApiTest.java
 create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java
 create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumedLedgersTrimTest.java
 create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AdminApiKeyStoreTlsAuthTest.java
 create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithAuth.java
 create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithoutAuth.java
 create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsTest.java
 create mode 100644 pulsar-broker/src/test/resources/authentication/keystoretls/broker.keystore.jks
 create mode 100644 pulsar-broker/src/test/resources/authentication/keystoretls/broker.truststore.jks
 create mode 100644 pulsar-broker/src/test/resources/authentication/keystoretls/brokerKeyStorePW.txt
 create mode 100644 pulsar-broker/src/test/resources/authentication/keystoretls/brokerTrustStorePW.txt
 create mode 100644 pulsar-broker/src/test/resources/authentication/keystoretls/client.keystore.jks
 create mode 100644 pulsar-broker/src/test/resources/authentication/keystoretls/client.truststore.jks
 create mode 100644 pulsar-broker/src/test/resources/authentication/keystoretls/clientKeyStorePW.txt
 create mode 100644 pulsar-broker/src/test/resources/authentication/keystoretls/clientTrustStorePW.txt
 copy pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaStorageFactory.java => pulsar-client-api/src/main/java/org/apache/pulsar/client/api/KeyStoreParams.java (70%)
 copy pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java => pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationDataKeyStoreTls.java (56%)
 create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationKeyStoreTls.java
 create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/generic/GenericJsonRecordTest.java
 create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverride.java
 copy pulsar-common/src/main/java/org/apache/pulsar/common/{api/raw/RawMessageIdImpl.java => policies/data/TopicType.java} (59%)
 copy {pulsar-broker/src/main/java/org/apache/pulsar/broker/service => pulsar-common/src/main/java/org/apache/pulsar/common/protocol}/schema/SchemaStorage.java (92%)
 rename {pulsar-broker/src/main/java/org/apache/pulsar/broker/service => pulsar-common/src/main/java/org/apache/pulsar/common/protocol}/schema/StoredSchema.java (86%)
 delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/ClientSslContextRefresher.java
 create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyClientSslContextRefresher.java
 rename pulsar-common/src/main/java/org/apache/pulsar/common/util/{NettySslContextBuilder.java => NettyServerSslContextBuilder.java} (52%)
 create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/KeyStoreSSLContext.java
 create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NetSslContextBuilder.java
 create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NettySSLContextAutoRefreshBuilder.java
 create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/SSLContextValidatorEngine.java
 create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/SslContextFactoryWithAutoRefresh.java
 copy pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaStorageFactory.java => pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/package-info.java (75%)
 create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverrideTest.java
 create mode 100644 pulsar-common/src/test/resources/broker.keystore.jks
 create mode 100644 pulsar-common/src/test/resources/broker.truststore.jks
 create mode 100644 pulsar-common/src/test/resources/brokerKeyStorePW.txt
 create mode 100644 pulsar-common/src/test/resources/brokerTrustStorePW.txt
 create mode 100644 pulsar-common/src/test/resources/ca-cert
 create mode 100644 pulsar-common/src/test/resources/ca-cert.srl
 create mode 100644 pulsar-common/src/test/resources/ca-key
 create mode 100644 pulsar-common/src/test/resources/cert-file
 create mode 100644 pulsar-common/src/test/resources/cert-signed
 create mode 100644 pulsar-common/src/test/resources/client.keystore.jks
 create mode 100644 pulsar-common/src/test/resources/client.truststore.jks
 create mode 100644 pulsar-common/src/test/resources/clientKeyStorePW.txt
 create mode 100644 pulsar-common/src/test/resources/clientTrustStorePW.txt
 create mode 100644 pulsar-common/src/test/resources/old/broker.keystore.jks
 create mode 100644 pulsar-common/src/test/resources/old/broker.truststore.jks
 create mode 100644 pulsar-common/src/test/resources/old/brokerKeyStorePW.txt
 create mode 100644 pulsar-common/src/test/resources/old/brokerTrustStorePW.txt
 create mode 100644 pulsar-common/src/test/resources/old/client.keystore.jks
 create mode 100644 pulsar-common/src/test/resources/old/client.truststore.jks
 create mode 100644 pulsar-common/src/test/resources/old/clientKeyStorePW.txt
 create mode 100644 pulsar-common/src/test/resources/old/clientTrustStorePW.txt
 create mode 100644 pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/AsyncContextFunction.java
 rename pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaStorage.java => pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/JavaNativeAsyncExclamationFunction.java (55%)
 create mode 100644 pulsar-functions/localrun/src/main/resources/log4j2.xml
 create mode 100644 pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithAuth.java
 create mode 100644 pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithoutAuth.java
 create mode 100644 pulsar-proxy/src/test/resources/authentication/keystoretls/broker.keystore.jks
 create mode 100644 pulsar-proxy/src/test/resources/authentication/keystoretls/broker.truststore.jks
 create mode 100644 pulsar-proxy/src/test/resources/authentication/keystoretls/brokerKeyStorePW.txt
 create mode 100644 pulsar-proxy/src/test/resources/authentication/keystoretls/brokerTrustStorePW.txt
 create mode 100644 pulsar-proxy/src/test/resources/authentication/keystoretls/client.keystore.jks
 create mode 100644 pulsar-proxy/src/test/resources/authentication/keystoretls/client.truststore.jks
 create mode 100644 pulsar-proxy/src/test/resources/authentication/keystoretls/clientKeyStorePW.txt
 create mode 100644 pulsar-proxy/src/test/resources/authentication/keystoretls/clientTrustStorePW.txt
 create mode 100644 pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZkUtils.java


[pulsar] 22/38: [C++] Fix message id is always the default value in send callback (#6812)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 0f1636a1aaebed4ee4865d6271c5f29ab5a45c5d
Author: BewareMyPower <xy...@163.com>
AuthorDate: Sat Apr 25 14:57:26 2020 +0800

    [C++] Fix message id is always the default value in send callback (#6812)
    
    * Fix bug: sendCallback's 2nd argument was always the default MessageId
    
    * Set batch index for each message's callback of batch
    
    * Add test for message id in send callback
    
    * Ensure all send callbacks completed before ASSERT_EQ
    (cherry picked from commit fc69a628abb92e3b5ecd8e98b8b00cc3738f4603)
---
 pulsar-client-cpp/lib/BatchMessageContainer.cc | 12 ++++---
 pulsar-client-cpp/lib/BatchMessageContainer.h  | 10 +++---
 pulsar-client-cpp/lib/ProducerImpl.cc          |  2 +-
 pulsar-client-cpp/tests/BasicEndToEndTest.cc   | 37 +++++++++++++++++++
 pulsar-client-cpp/tests/BatchMessageTest.cc    | 50 +++++++++++++++++++++++++-
 5 files changed, 99 insertions(+), 12 deletions(-)

diff --git a/pulsar-client-cpp/lib/BatchMessageContainer.cc b/pulsar-client-cpp/lib/BatchMessageContainer.cc
index 9f904a2..7413d57 100644
--- a/pulsar-client-cpp/lib/BatchMessageContainer.cc
+++ b/pulsar-client-cpp/lib/BatchMessageContainer.cc
@@ -67,7 +67,7 @@ void BatchMessageContainer::add(const Message& msg, SendCallback sendCallback, b
                                                        maxAllowedMessageBatchSizeInBytes_);
     LOG_DEBUG(*this << " After serialization payload size in bytes = " << impl_->payload.readableBytes());
 
-    messagesContainerListPtr_->push_back(MessageContainer(msg, sendCallback, msg.getMessageId()));
+    messagesContainerListPtr_->emplace_back(msg, sendCallback);
 
     LOG_DEBUG(*this << " Number of messages in Batch = " << messagesContainerListPtr_->size());
     LOG_DEBUG(*this << " Batch Payload Size In Bytes = " << batchSizeInBytes_);
@@ -105,7 +105,7 @@ void BatchMessageContainer::sendMessage(FlushCallback flushCallback) {
     if (impl_->payload.readableBytes() > producer_.keepMaxMessageSize_) {
         // At this point the compressed batch is above the overall MaxMessageSize. There
         // can only 1 single message in the batch at this point.
-        batchMessageCallBack(ResultMessageTooBig, messagesContainerListPtr_, nullptr);
+        batchMessageCallBack(ResultMessageTooBig, MessageId{}, messagesContainerListPtr_, nullptr);
         clear();
         return;
     }
@@ -115,7 +115,7 @@ void BatchMessageContainer::sendMessage(FlushCallback flushCallback) {
 
     // bind keeps a copy of the parameters
     SendCallback callback = std::bind(&BatchMessageContainer::batchMessageCallBack, std::placeholders::_1,
-                                      messagesContainerListPtr_, flushCallback);
+                                      std::placeholders::_2, messagesContainerListPtr_, flushCallback);
 
     producer_.sendMessage(msg, callback);
     clear();
@@ -144,7 +144,8 @@ void BatchMessageContainer::clear() {
     batchSizeInBytes_ = 0;
 }
 
-void BatchMessageContainer::batchMessageCallBack(Result r, MessageContainerListPtr messagesContainerListPtr,
+void BatchMessageContainer::batchMessageCallBack(Result r, const MessageId& messageId,
+                                                 MessageContainerListPtr messagesContainerListPtr,
                                                  FlushCallback flushCallback) {
     if (!messagesContainerListPtr) {
         if (flushCallback) {
@@ -156,7 +157,8 @@ void BatchMessageContainer::batchMessageCallBack(Result r, MessageContainerListP
               << r << "] [numOfMessages = " << messagesContainerListPtr->size() << "]");
     size_t batch_size = messagesContainerListPtr->size();
     for (size_t i = 0; i < batch_size; i++) {
-        messagesContainerListPtr->operator[](i).callBack(r);
+        MessageId messageIdInBatch(messageId.partition(), messageId.ledgerId(), messageId.entryId(), i);
+        messagesContainerListPtr->operator[](i).callBack(r, messageIdInBatch);
     }
     if (flushCallback) {
         flushCallback(ResultOk);
diff --git a/pulsar-client-cpp/lib/BatchMessageContainer.h b/pulsar-client-cpp/lib/BatchMessageContainer.h
index 424c7f4..93b4d81 100644
--- a/pulsar-client-cpp/lib/BatchMessageContainer.h
+++ b/pulsar-client-cpp/lib/BatchMessageContainer.h
@@ -45,12 +45,11 @@ namespace pulsar {
 class BatchMessageContainer {
    public:
     struct MessageContainer {
-        MessageContainer(Message message, SendCallback sendCallback, MessageId messageId)
-            : message_(message), sendCallback_(sendCallback), messageId_(messageId) {}
+        MessageContainer(Message message, SendCallback sendCallback)
+            : message_(message), sendCallback_(sendCallback) {}
         Message message_;
         SendCallback sendCallback_;
-        MessageId messageId_;
-        void callBack(const pulsar::Result& r) { sendCallback_(r, messageId_); }
+        void callBack(Result r, const MessageId& messageId) { sendCallback_(r, messageId); }
     };
     typedef std::vector<MessageContainer> MessageContainerList;
     typedef std::shared_ptr<MessageContainerList> MessageContainerListPtr;
@@ -65,7 +64,8 @@ class BatchMessageContainer {
 
     void clear();
 
-    static void batchMessageCallBack(Result r, MessageContainerListPtr messages, FlushCallback callback);
+    static void batchMessageCallBack(Result r, const MessageId& messageId, MessageContainerListPtr messages,
+                                     FlushCallback callback);
 
     friend inline std::ostream& operator<<(std::ostream& os,
                                            const BatchMessageContainer& batchMessageContainer);
diff --git a/pulsar-client-cpp/lib/ProducerImpl.cc b/pulsar-client-cpp/lib/ProducerImpl.cc
index f38ed2a..0095dc8 100644
--- a/pulsar-client-cpp/lib/ProducerImpl.cc
+++ b/pulsar-client-cpp/lib/ProducerImpl.cc
@@ -259,7 +259,7 @@ void ProducerImpl::failPendingMessages(Result result) {
     }
 
     // this function can handle null pointer
-    BatchMessageContainer::batchMessageCallBack(result, messageContainerListPtr, NULL);
+    BatchMessageContainer::batchMessageCallBack(result, MessageId{}, messageContainerListPtr, NULL);
 }
 
 void ProducerImpl::resendMessages(ClientConnectionPtr cnx) {
diff --git a/pulsar-client-cpp/tests/BasicEndToEndTest.cc b/pulsar-client-cpp/tests/BasicEndToEndTest.cc
index 54077ea..99b7f87 100644
--- a/pulsar-client-cpp/tests/BasicEndToEndTest.cc
+++ b/pulsar-client-cpp/tests/BasicEndToEndTest.cc
@@ -3179,3 +3179,40 @@ TEST(BasicEndToEndTest, testCumulativeAcknowledgeNotAllowed) {
     }
     client.shutdown();
 }
+
+TEST(BasicEndToEndTest, testSendCallback) {
+    const std::string topicName = "persistent://public/default/BasicEndToEndTest-testSendCallback";
+
+    Client client(lookupUrl);
+
+    Producer producer;
+    ASSERT_EQ(ResultOk, client.createProducer(topicName, producer));
+
+    Consumer consumer;
+    ASSERT_EQ(ResultOk, client.subscribe(topicName, "SubscriptionName", consumer));
+
+    Latch latch(100);
+    std::set<MessageId> sentIdSet;
+    for (int i = 0; i < 100; i++) {
+        const auto msg = MessageBuilder().setContent("a").build();
+        producer.sendAsync(msg, [&sentIdSet, i, &latch](Result result, const MessageId &id) {
+            ASSERT_EQ(ResultOk, result);
+            sentIdSet.emplace(id);
+            latch.countdown();
+        });
+    }
+
+    std::set<MessageId> receivedIdSet;
+    for (int i = 0; i < 100; i++) {
+        Message msg;
+        ASSERT_EQ(ResultOk, consumer.receive(msg));
+        receivedIdSet.emplace(msg.getMessageId());
+    }
+
+    latch.wait();
+    ASSERT_EQ(sentIdSet, receivedIdSet);
+
+    consumer.close();
+    producer.close();
+    client.close();
+}
diff --git a/pulsar-client-cpp/tests/BatchMessageTest.cc b/pulsar-client-cpp/tests/BatchMessageTest.cc
index 3fe46ed..f9638f8 100644
--- a/pulsar-client-cpp/tests/BatchMessageTest.cc
+++ b/pulsar-client-cpp/tests/BatchMessageTest.cc
@@ -25,6 +25,7 @@
 
 #include <lib/Commands.h>
 #include <lib/Future.h>
+#include <lib/Latch.h>
 #include <lib/LogUtils.h>
 #include <lib/TopicName.h>
 #include <lib/Utils.h>
@@ -982,4 +983,51 @@ TEST(BatchMessageTest, testPraseMessageBatchEntry) {
         ASSERT_EQ(message.getDataAsString(), expected.content);
         ASSERT_EQ(message.getProperty(expected.propKey), expected.propValue);
     }
-}
\ No newline at end of file
+}
+
+TEST(BatchMessageTest, testSendCallback) {
+    const std::string topicName = "persistent://public/default/BasicMessageTest-testSendCallback";
+
+    Client client(lookupUrl);
+
+    constexpr int numMessagesOfBatch = 3;
+
+    ProducerConfiguration producerConfig;
+    producerConfig.setBatchingEnabled(5);
+    producerConfig.setBatchingMaxMessages(numMessagesOfBatch);
+    producerConfig.setBatchingMaxPublishDelayMs(1000);  // 1 s, it's long enough for 3 messages batched
+    producerConfig.setMaxPendingMessages(numMessagesOfBatch);
+
+    Producer producer;
+    ASSERT_EQ(ResultOk, client.createProducer(topicName, producerConfig, producer));
+
+    Consumer consumer;
+    ASSERT_EQ(ResultOk, client.subscribe(topicName, "SubscriptionName", consumer));
+
+    Latch latch(numMessagesOfBatch);
+    std::set<MessageId> sentIdSet;
+    for (int i = 0; i < numMessagesOfBatch; i++) {
+        const auto msg = MessageBuilder().setContent("a").build();
+        producer.sendAsync(msg, [&sentIdSet, i, &latch](Result result, const MessageId& id) {
+            ASSERT_EQ(ResultOk, result);
+            ASSERT_EQ(i, id.batchIndex());
+            sentIdSet.emplace(id);
+            LOG_INFO("id of batch " << i << ": " << id);
+            latch.countdown();
+        });
+    }
+
+    std::set<MessageId> receivedIdSet;
+    for (int i = 0; i < numMessagesOfBatch; i++) {
+        Message msg;
+        ASSERT_EQ(ResultOk, consumer.receive(msg));
+        receivedIdSet.emplace(msg.getMessageId());
+    }
+
+    latch.wait();
+    ASSERT_EQ(sentIdSet, receivedIdSet);
+
+    consumer.close();
+    producer.close();
+    client.close();
+}


[pulsar] 31/38: change log level to debug for bouncy castle load (#6860)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 4063f79819fcc0f44d9979b84208d290560d4e9f
Author: Jia Zhai <zh...@apache.org>
AuthorDate: Wed May 6 20:31:31 2020 +0800

    change log level to debug for bouncy castle load (#6860)
    
    change log level to debug to avoid annoying.
    
    (cherry picked from commit 4e6b2f09a08880faf8e8eb5753285ced0b3bf519)
---
 .../main/java/org/apache/pulsar/common/util/SearchBcNarUtils.java | 6 ++++--
 .../main/java/org/apache/pulsar/common/util/SecurityUtility.java  | 8 ++++++--
 2 files changed, 10 insertions(+), 4 deletions(-)

diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SearchBcNarUtils.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SearchBcNarUtils.java
index e3d4f88..0dfa469 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SearchBcNarUtils.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SearchBcNarUtils.java
@@ -89,8 +89,10 @@ public class SearchBcNarUtils {
                         }
 
                         Provider provider = ((BCLoader) loader).getProvider();
-                        log.info("Found Bouncy Castle loader {} from {}, provider: {}",
-                                loader.getClass().getCanonicalName(), path, provider.getName());
+                        if (log.isDebugEnabled()) {
+                            log.debug("Found Bouncy Castle loader {} from {}, provider: {}",
+                                    loader.getClass().getCanonicalName(), path, provider.getName());
+                        }
                         loadFuture.complete(provider);
                     } catch (Throwable t) {
                         log.error("Failed to load Bouncy Castle Provider ", t);
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java
index d3a2615..648e9f2 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java
@@ -87,7 +87,9 @@ public class SecurityUtility {
             Provider provider = Security.getProvider(BC) != null
                     ? Security.getProvider(BC)
                     : Security.getProvider(BC_FIPS);
-            log.info("Already instantiated Bouncy Castle provider {}", provider.getName());
+            if (log.isDebugEnabled()) {
+                log.debug("Already instantiated Bouncy Castle provider {}", provider.getName());
+            }
             return provider;
         }
 
@@ -125,7 +127,9 @@ public class SecurityUtility {
 
         Provider provider = (Provider) clazz.newInstance();
         Security.addProvider(provider);
-        log.info("Found and Instantiated Bouncy Castle provider in classpath {}", provider.getName());
+        if (log.isDebugEnabled()) {
+            log.debug("Found and Instantiated Bouncy Castle provider in classpath {}", provider.getName());
+        }
         return provider;
     }
 


[pulsar] 36/38: Support function with format: Function> (#6684)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 55d5430701d41d92ce290d838e332eb9d9154b9e
Author: Jia Zhai <zh...@apache.org>
AuthorDate: Fri May 8 11:04:09 2020 +0800

    Support function with format: Function<I, CompletableFuture<O>> (#6684)
    
    Fixes #6519
    
    ### Motivation
    
    Currently, Pulsar Functions not support Async mode, e.g. user passed in a Function in format :
    ```
    Function<I, CompletableFuture<O>>
    ```
    This kind of function is useful if the function might use RPCs to call external systems.
    
    e.g.
    ```java
    public class AsyncFunction implements Function<String, CompletableFuture<O>> {
        CompletableFuture<O> apply (String input) {
            CompletableFuture future = new CompletableFuture();
            ...function compute...
            future.whenComplete(() -> {
                ...  call external system  ...
            });
            return future;
        }
    ```
    
    ### Modifications
    - add support for Async Functions support.
    
    ### Verifying this change
    current ut passed.
    
    * support func: Function<I, CompletableFuture<O>>
    
    * add 2 examples
    
    * add limit to the max outstanding items
    (cherry picked from commit 7cd28b9d5cc862fe239f5e8fa1ca616237764607)
---
 .../pulsar/common/functions/FunctionConfig.java    |  3 +
 .../pulsar/functions/instance/InstanceConfig.java  |  3 +
 .../pulsar/functions/instance/JavaInstance.java    | 64 +++++++++++++--
 .../functions/instance/JavaInstanceRunnable.java   | 41 ++++-----
 .../functions/instance/JavaInstanceTest.java       | 96 ++++++++++++++++++++--
 .../api/examples/AsyncContextFunction.java         | 59 +++++++++++++
 .../JavaNativeAsyncExclamationFunction.java        | 41 +++++++++
 .../org/apache/pulsar/functions/LocalRunner.java   |  2 +
 .../functions/runtime/JavaInstanceStarter.java     |  4 +
 .../pulsar/functions/worker/WorkerConfig.java      |  6 ++
 .../pulsar/functions/worker/FunctionActioner.java  |  1 +
 11 files changed, 289 insertions(+), 31 deletions(-)

diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/functions/FunctionConfig.java b/pulsar-common/src/main/java/org/apache/pulsar/common/functions/FunctionConfig.java
index 5ae0621..13123bc 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/functions/FunctionConfig.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/functions/FunctionConfig.java
@@ -120,4 +120,7 @@ public class FunctionConfig {
     // to change behavior at runtime. Currently, this primarily used by the KubernetesManifestCustomizer
     // interface
     private String customRuntimeOptions;
+    // Max pending async requests per instance to avoid large number of concurrent requests.
+    // Only used in AsyncFunction. Default: 1000.
+    private Integer maxPendingAsyncRequests = 1000;
 }
diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/InstanceConfig.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/InstanceConfig.java
index f823728..86f57eda 100644
--- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/InstanceConfig.java
+++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/InstanceConfig.java
@@ -44,6 +44,9 @@ public class InstanceConfig {
     private Function.FunctionAuthenticationSpec functionAuthenticationSpec;
     private int port;
     private String clusterName;
+    // Max pending async requests per instance to avoid large number of concurrent requests.
+    // Only used in AsyncFunction. Default: 1000
+    private int maxPendingAsyncRequests = 1000;
 
     /**
      * Get the string representation of {@link #getInstanceId()}.
diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
index 8aee702..1e18a07 100644
--- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
+++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstance.java
@@ -18,6 +18,11 @@
  */
 package org.apache.pulsar.functions.instance;
 
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
 import lombok.AccessLevel;
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
@@ -40,9 +45,18 @@ public class JavaInstance implements AutoCloseable {
     private Function function;
     private java.util.function.Function javaUtilFunction;
 
-    public JavaInstance(ContextImpl contextImpl, Object userClassObject) {
+    // for Async function max out standing items
+    private final InstanceConfig instanceConfig;
+    private final Executor executor;
+    @Getter
+    private final LinkedBlockingQueue<CompletableFuture<Void>> pendingAsyncRequests;
+
+    public JavaInstance(ContextImpl contextImpl, Object userClassObject, InstanceConfig instanceConfig) {
 
         this.context = contextImpl;
+        this.instanceConfig = instanceConfig;
+        this.executor = Executors.newSingleThreadExecutor();
+        this.pendingAsyncRequests = new LinkedBlockingQueue<>(this.instanceConfig.getMaxPendingAsyncRequests());
 
         // create the functions
         if (userClassObject instanceof Function) {
@@ -52,23 +66,63 @@ public class JavaInstance implements AutoCloseable {
         }
     }
 
-    public JavaExecutionResult handleMessage(Record<?> record, Object input) {
+    public CompletableFuture<JavaExecutionResult> handleMessage(Record<?> record, Object input) {
         if (context != null) {
             context.setCurrentMessageContext(record);
         }
+
+        final CompletableFuture<JavaExecutionResult> future = new CompletableFuture<>();
         JavaExecutionResult executionResult = new JavaExecutionResult();
+
+        final Object output;
+
         try {
-            Object output;
             if (function != null) {
                 output = function.process(input, context);
             } else {
                 output = javaUtilFunction.apply(input);
             }
-            executionResult.setResult(output);
         } catch (Exception ex) {
             executionResult.setUserException(ex);
+            future.complete(executionResult);
+            return future;
+        }
+
+        if (output instanceof CompletableFuture) {
+            // Function is in format: Function<I, CompletableFuture<O>>
+            try {
+                pendingAsyncRequests.put((CompletableFuture) output);
+            } catch (InterruptedException ie) {
+                log.warn("Exception while put Async requests", ie);
+                executionResult.setUserException(ie);
+                future.complete(executionResult);
+                return future;
+            }
+
+            ((CompletableFuture) output).whenCompleteAsync((obj, throwable) -> {
+                if (log.isDebugEnabled()) {
+                    log.debug("Got result async: object: {}, throwable: {}", obj, throwable);
+                }
+
+                if (throwable != null) {
+                    executionResult.setUserException(new Exception((Throwable)throwable));
+                    pendingAsyncRequests.remove(output);
+                    future.complete(executionResult);
+                    return;
+                }
+                executionResult.setResult(obj);
+                pendingAsyncRequests.remove(output);
+                future.complete(executionResult);
+            }, executor);
+        } else {
+            if (log.isDebugEnabled()) {
+                log.debug("Got result: object: {}", output);
+            }
+            executionResult.setResult(output);
+            future.complete(executionResult);
         }
-        return executionResult;
+
+        return future;
     }
 
     @Override
diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java
index 2db60c7..b983991 100644
--- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java
+++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java
@@ -24,6 +24,7 @@ import com.google.gson.Gson;
 import com.google.gson.reflect.TypeToken;
 import io.netty.buffer.ByteBuf;
 import io.prometheus.client.CollectorRegistry;
+import java.util.concurrent.CompletableFuture;
 import lombok.AccessLevel;
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
@@ -216,7 +217,7 @@ public class JavaInstanceRunnable implements AutoCloseable, Runnable {
         // start any log topic handler
         setupLogHandler();
 
-        return new JavaInstance(contextImpl, object);
+        return new JavaInstance(contextImpl, object, instanceConfig);
     }
 
     ContextImpl setupContext() {
@@ -256,7 +257,7 @@ public class JavaInstanceRunnable implements AutoCloseable, Runnable {
                 }
 
                 addLogTopicHandler();
-                JavaExecutionResult result;
+                CompletableFuture<JavaExecutionResult> result;
 
                 // set last invocation time
                 stats.setLastInvocation(System.currentTimeMillis());
@@ -274,10 +275,6 @@ public class JavaInstanceRunnable implements AutoCloseable, Runnable {
 
                 removeLogTopicHandler();
 
-                if (log.isDebugEnabled()) {
-                    log.debug("Got result: {}", result.getResult());
-                }
-
                 try {
                     processResult(currentRecord, result);
                 } catch (Exception e) {
@@ -417,23 +414,27 @@ public class JavaInstanceRunnable implements AutoCloseable, Runnable {
     }
 
     private void processResult(Record srcRecord,
-                               JavaExecutionResult result) throws Exception {
-        if (result.getUserException() != null) {
-            log.info("Encountered user exception when processing message {}", srcRecord, result.getUserException());
-            stats.incrUserExceptions(result.getUserException());
-            srcRecord.fail();
-        } else {
-            if (result.getResult() != null) {
-                sendOutputMessage(srcRecord, result.getResult());
+                               CompletableFuture<JavaExecutionResult> result) throws Exception {
+        result.whenComplete((result1, throwable) -> {
+            if (throwable != null || result1.getUserException() != null) {
+                Throwable t = throwable != null ? throwable : result1.getUserException();
+                log.warn("Encountered exception when processing message {}",
+                        srcRecord, t);
+                stats.incrUserExceptions(t);
+                srcRecord.fail();
             } else {
-                if (instanceConfig.getFunctionDetails().getAutoAck()) {
-                    // the function doesn't produce any result or the user doesn't want the result.
-                    srcRecord.ack();
+                if (result1.getResult() != null) {
+                    sendOutputMessage(srcRecord, result1.getResult());
+                } else {
+                    if (instanceConfig.getFunctionDetails().getAutoAck()) {
+                        // the function doesn't produce any result or the user doesn't want the result.
+                        srcRecord.ack();
+                    }
                 }
+                // increment total successfully processed
+                stats.incrTotalProcessedSuccessfully();
             }
-            // increment total successfully processed
-            stats.incrTotalProcessedSuccessfully();
-        }
+        });
     }
 
     private void sendOutputMessage(Record srcRecord, Object output) {
diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceTest.java
index 0cb361d..5061d1e 100644
--- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceTest.java
+++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceTest.java
@@ -22,10 +22,14 @@ import static org.mockito.Mockito.mock;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertNotNull;
 
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import lombok.extern.slf4j.Slf4j;
 import org.apache.pulsar.functions.api.Function;
 import org.apache.pulsar.functions.api.Record;
 import org.testng.annotations.Test;
 
+@Slf4j
 public class JavaInstanceTest {
 
     /**
@@ -33,14 +37,94 @@ public class JavaInstanceTest {
      * @throws Exception
      */
     @Test
-    public void testLambda() {
+    public void testLambda() throws Exception {
         JavaInstance instance = new JavaInstance(
-            mock(ContextImpl.class),
-            (Function<String, String>) (input, context) -> input + "-lambda");
+                mock(ContextImpl.class),
+                (Function<String, String>) (input, context) -> input + "-lambda",
+                new InstanceConfig());
         String testString = "ABC123";
-        JavaExecutionResult result = instance.handleMessage(mock(Record.class), testString);
-        assertNotNull(result.getResult());
-        assertEquals(new String(testString + "-lambda"), result.getResult());
+        CompletableFuture<JavaExecutionResult> result = instance.handleMessage(mock(Record.class), testString);
+        assertNotNull(result.get().getResult());
+        assertEquals(new String(testString + "-lambda"), result.get().getResult());
+        instance.close();
+    }
+
+    @Test
+    public void testAsyncFunction() throws Exception {
+        InstanceConfig instanceConfig = new InstanceConfig();
+
+        Function<String, CompletableFuture<String>> function = (input, context) -> {
+            log.info("input string: {}", input);
+            CompletableFuture<String> result  = new CompletableFuture<>();
+            Executors.newCachedThreadPool().submit(() -> {
+                try {
+                    Thread.sleep(500);
+                    result.complete(String.format("%s-lambda", input));
+                } catch (Exception e) {
+                    result.completeExceptionally(e);
+                }
+            });
+
+            return result;
+        };
+
+        JavaInstance instance = new JavaInstance(
+                mock(ContextImpl.class),
+                function,
+                instanceConfig);
+        String testString = "ABC123";
+        CompletableFuture<JavaExecutionResult> result = instance.handleMessage(mock(Record.class), testString);
+        assertNotNull(result.get().getResult());
+        assertEquals(new String(testString + "-lambda"), result.get().getResult());
+        instance.close();
+    }
+
+    @Test
+    public void testAsyncFunctionMaxPending() throws Exception {
+        InstanceConfig instanceConfig = new InstanceConfig();
+        int pendingQueueSize = 2;
+        instanceConfig.setMaxPendingAsyncRequests(pendingQueueSize);
+
+        Function<String, CompletableFuture<String>> function = (input, context) -> {
+            log.info("input string: {}", input);
+            CompletableFuture<String> result  = new CompletableFuture<>();
+            Executors.newCachedThreadPool().submit(() -> {
+                try {
+                    Thread.sleep(500);
+                    result.complete(String.format("%s-lambda", input));
+                } catch (Exception e) {
+                    result.completeExceptionally(e);
+                }
+            });
+
+            return result;
+        };
+
+        JavaInstance instance = new JavaInstance(
+                mock(ContextImpl.class),
+                function,
+                instanceConfig);
+        String testString = "ABC123";
+
+        long startTime = System.currentTimeMillis();
+        assertEquals(pendingQueueSize, instance.getPendingAsyncRequests().remainingCapacity());
+        CompletableFuture<JavaExecutionResult> result1 = instance.handleMessage(mock(Record.class), testString);
+        assertEquals(pendingQueueSize - 1, instance.getPendingAsyncRequests().remainingCapacity());
+        CompletableFuture<JavaExecutionResult> result2 = instance.handleMessage(mock(Record.class), testString);
+        assertEquals(pendingQueueSize - 2, instance.getPendingAsyncRequests().remainingCapacity());
+        CompletableFuture<JavaExecutionResult> result3 = instance.handleMessage(mock(Record.class), testString);
+        // no space left
+        assertEquals(0, instance.getPendingAsyncRequests().remainingCapacity());
+
+        instance.getPendingAsyncRequests().remainingCapacity();
+        assertNotNull(result1.get().getResult());
+        assertNotNull(result2.get().getResult());
+        assertNotNull(result3.get().getResult());
+
+        assertEquals(new String(testString + "-lambda"), result1.get().getResult());
+        long endTime = System.currentTimeMillis();
+
+        log.info("start:{} end:{} during:{}", startTime, endTime, endTime - startTime);
         instance.close();
     }
 }
diff --git a/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/AsyncContextFunction.java b/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/AsyncContextFunction.java
new file mode 100644
index 0000000..b70bc7c
--- /dev/null
+++ b/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/AsyncContextFunction.java
@@ -0,0 +1,59 @@
+/**
+ * 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.functions.api.examples;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import org.apache.pulsar.functions.api.Context;
+import org.apache.pulsar.functions.api.Function;
+import org.slf4j.Logger;
+
+public class AsyncContextFunction implements Function<String, CompletableFuture<Void>> {
+    @Override
+    public CompletableFuture<Void> process(String input, Context context) {
+        Logger LOG = context.getLogger();
+        CompletableFuture<Void> future = new CompletableFuture();
+
+        // this method only delay a function execute.
+        Executors.newCachedThreadPool().submit(() -> {
+            try {
+                Thread.sleep(500);
+            } catch (Exception e) {
+                LOG.error("Exception when Thread.sleep", e);
+                future.completeExceptionally(e);
+            }
+
+            String inputTopics = context.getInputTopics().stream().collect(Collectors.joining(", "));
+            String funcName = context.getFunctionName();
+
+            String logMessage = String
+                    .format("A message with value of \"%s\" has arrived on one of the following topics: %s\n",
+                            input, inputTopics);
+            LOG.info(logMessage);
+
+            String metricName = String.format("function-%s-messages-received", funcName);
+            context.recordMetric(metricName, 1);
+
+            future.complete(null);
+        });
+
+        return future;
+    }
+}
diff --git a/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/JavaNativeAsyncExclamationFunction.java b/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/JavaNativeAsyncExclamationFunction.java
new file mode 100644
index 0000000..7cad46b
--- /dev/null
+++ b/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/JavaNativeAsyncExclamationFunction.java
@@ -0,0 +1,41 @@
+/**
+ * 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.functions.api.examples;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+
+public class JavaNativeAsyncExclamationFunction implements Function<String, CompletableFuture<String>> {
+    @Override
+    public CompletableFuture<String> apply(String input) {
+        CompletableFuture<String> future = new CompletableFuture();
+
+        Executors.newCachedThreadPool().submit(() -> {
+            try {
+                Thread.sleep(500);
+                future.complete(String.format("%s-!!", input));
+            } catch (Exception e) {
+                future.completeExceptionally(e);
+            }
+        });
+
+        return future;
+    }
+}
diff --git a/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java b/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java
index 244a757..a606d3b 100644
--- a/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java
+++ b/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java
@@ -358,6 +358,7 @@ public class LocalRunner {
                 instanceConfig.setMaxBufferedTuples(1024);
                 instanceConfig.setPort(FunctionCommon.findAvailablePort());
                 instanceConfig.setClusterName("local");
+                instanceConfig.setMaxPendingAsyncRequests(functionConfig.getMaxPendingAsyncRequests());
                 RuntimeSpawner runtimeSpawner = new RuntimeSpawner(
                         instanceConfig,
                         userCodeFile,
@@ -417,6 +418,7 @@ public class LocalRunner {
             instanceConfig.setMaxBufferedTuples(1024);
             instanceConfig.setPort(FunctionCommon.findAvailablePort());
             instanceConfig.setClusterName("local");
+            instanceConfig.setMaxPendingAsyncRequests(functionConfig.getMaxPendingAsyncRequests());
             RuntimeSpawner runtimeSpawner = new RuntimeSpawner(
                     instanceConfig,
                     userCodeFile,
diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java
index ec2e36a..970047f 100644
--- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java
+++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java
@@ -125,6 +125,9 @@ public class JavaInstanceStarter implements AutoCloseable {
     @Parameter(names = "--cluster_name", description = "The name of the cluster this instance is running on", required = true)
     public String clusterName;
 
+    @Parameter(names = "--pending_async_requests", description = "Max pending async requests per instance", required = false)
+    public int maxPendingAsyncRequests = 1000;
+
     private Server server;
     private RuntimeSpawner runtimeSpawner;
     private ThreadRuntimeFactory containerFactory;
@@ -147,6 +150,7 @@ public class JavaInstanceStarter implements AutoCloseable {
         instanceConfig.setInstanceId(instanceId);
         instanceConfig.setMaxBufferedTuples(maxBufferedTuples);
         instanceConfig.setClusterName(clusterName);
+        instanceConfig.setMaxPendingAsyncRequests(maxPendingAsyncRequests);
         Function.FunctionDetails.Builder functionDetailsBuilder = Function.FunctionDetails.newBuilder();
         if (functionDetailsJsonString.charAt(0) == '\'') {
             functionDetailsJsonString = functionDetailsJsonString.substring(1);
diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java
index 01dff75..6d4619f 100644
--- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java
+++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java
@@ -404,6 +404,12 @@ public class WorkerConfig implements Serializable, PulsarConfiguration {
     )
     private Map<String, Object> runtimeCustomizerConfig = Collections.emptyMap();
 
+    @FieldContext(
+            doc = "Max pending async requests per instance to avoid large number of concurrent requests."
+                  + "Only used in AsyncFunction. Default: 1000"
+    )
+    private int maxPendingAsyncRequests = 1000;
+
     public String getFunctionMetadataTopic() {
         return String.format("persistent://%s/%s", pulsarFunctionsNamespace, functionMetadataTopicName);
     }
diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java
index c1b4971..36d9f13 100644
--- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java
+++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java
@@ -185,6 +185,7 @@ public class FunctionActioner {
         instanceConfig.setPort(FunctionCommon.findAvailablePort());
         instanceConfig.setClusterName(clusterName);
         instanceConfig.setFunctionAuthenticationSpec(functionAuthSpec);
+        instanceConfig.setMaxPendingAsyncRequests(workerConfig.getMaxPendingAsyncRequests());
         return instanceConfig;
     }
 


[pulsar] 21/38: Make messageReceiveTimeoutMs in the PulsarConsumerSource configurable (#6783)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 8783907da1ec91e10fcd6b8dc7d5d928e2778498
Author: luceneReader <54...@qq.com>
AuthorDate: Sat Apr 25 14:47:56 2020 +0800

    Make messageReceiveTimeoutMs in the PulsarConsumerSource configurable (#6783)
    
    The messageReceiveTimeoutMs value in the PulsarConsumerSource class is hardcoded to 100ms and cannot be altered through configuration at present.
    (cherry picked from commit 47b4dd071a7e4f6d31f0d6f9abb1f182260820d8)
---
 .../connectors/pulsar/PulsarConsumerSource.java          |  3 ++-
 .../streaming/connectors/pulsar/PulsarSourceBuilder.java | 16 ++++++++++++++++
 2 files changed, 18 insertions(+), 1 deletion(-)

diff --git a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarConsumerSource.java b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarConsumerSource.java
index 9606dfc..e137cae 100644
--- a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarConsumerSource.java
+++ b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarConsumerSource.java
@@ -54,7 +54,7 @@ class PulsarConsumerSource<T> extends MessageAcknowledgingSourceBase<T, MessageI
 
     private static final Logger LOG = LoggerFactory.getLogger(PulsarConsumerSource.class);
 
-    private final int messageReceiveTimeoutMs = 100;
+    private int messageReceiveTimeoutMs;
 
     private ClientConfigurationData clientConfigurationData;
     private ConsumerConfigurationData<byte[]> consumerConfigurationData;
@@ -81,6 +81,7 @@ class PulsarConsumerSource<T> extends MessageAcknowledgingSourceBase<T, MessageI
         this.consumerConfigurationData = builder.consumerConfigurationData;
         this.deserializer = builder.deserializationSchema;
         this.acknowledgementBatchSize = builder.acknowledgementBatchSize;
+        this.messageReceiveTimeoutMs = builder.messageReceiveTimeoutMs;
     }
 
     @Override
diff --git a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarSourceBuilder.java b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarSourceBuilder.java
index 15dc6e4..67690be 100644
--- a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarSourceBuilder.java
+++ b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/PulsarSourceBuilder.java
@@ -44,6 +44,7 @@ public class PulsarSourceBuilder<T> {
     private static final String SERVICE_URL = "pulsar://localhost:6650";
     private static final long ACKNOWLEDGEMENT_BATCH_SIZE = 100;
     private static final long MAX_ACKNOWLEDGEMENT_BATCH_SIZE = 1000;
+    private static final int DEFAULT_MESSAGE_RECEIVE_TIMEOUT_MS = 100;
     private static final String SUBSCRIPTION_NAME = "flink-sub";
 
     final DeserializationSchema<T> deserializationSchema;
@@ -52,6 +53,8 @@ public class PulsarSourceBuilder<T> {
     ConsumerConfigurationData<byte[]> consumerConfigurationData;
 
     long acknowledgementBatchSize = ACKNOWLEDGEMENT_BATCH_SIZE;
+    //
+    int messageReceiveTimeoutMs = DEFAULT_MESSAGE_RECEIVE_TIMEOUT_MS;
 
     private PulsarSourceBuilder(DeserializationSchema<T> deserializationSchema) {
         this.deserializationSchema = deserializationSchema;
@@ -193,6 +196,19 @@ public class PulsarSourceBuilder<T> {
     }
 
     /**
+     * parameterize messageReceiveTimeoutMs for `PulsarConsumerSource`.
+     * @param timeout timeout in ms, should be gt 0
+     * @return this builder
+     */
+    public PulsarSourceBuilder<T> messageReceiveTimeoutMs(int timeout) {
+        if (timeout <= 0) {
+            throw new IllegalArgumentException("messageReceiveTimeoutMs can only take values > 0");
+        }
+        this.messageReceiveTimeoutMs = timeout;
+        return this;
+    }
+
+    /**
      * Set the authentication provider to use in the Pulsar client instance.
      *
      * @param authentication an instance of the {@link Authentication} provider already constructed


[pulsar] 29/38: Fixing JDBC sink to handle null fields. Also added new unit tests (#6848)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 53f74c7fd9eb0bf4a81f09d2dc1ab0a7895e1b97
Author: Chris Bartholomew <c_...@yahoo.com>
AuthorDate: Fri May 1 13:53:23 2020 -0400

    Fixing JDBC sink to handle null fields. Also added new unit tests (#6848)
    
    ### Motivation
    
    JDBC sink does not handle `null` fields. For example, the field `example` can potentially be null. The schema registered in Pulsar allows for it, and the table schema in MySQL has a column of the same name, is configured as double and also allows nulls. When messages are sent to the JDBC sink without that field, an exception like this is seen:
    
    ```
    21:08:38.472 [pool-5-thread-1] ERROR org.apache.pulsar.io.jdbc.JdbcAbstractSink - Got exception
    java.sql.SQLException: Data truncated for column 'example' at row 1
    	at com.mysql.cj.jdbc.exceptions.SQLError.createSQLException(SQLError.java:127) ~[mysql-connector-java-8.0.11.jar:8.0.11]
    	at com.mysql.cj.jdbc.exceptions.SQLError.createSQLException(SQLError.java:95) ~[mysql-connector-java-8.0.11.jar:8.0.11]
    	at com.mysql.cj.jdbc.exceptions.SQLExceptionsMapping.translateException(SQLExceptionsMapping.java:122) ~[mysql-connector-java-8.0.11.jar:8.0.11]
    	at com.mysql.cj.jdbc.ClientPreparedStatement.executeInternal(ClientPreparedStatement.java:960) ~[mysql-connector-java-8.0.11.jar:8.0.11]
    	at com.mysql.cj.jdbc.ClientPreparedStatement.execute(ClientPreparedStatement.java:388) ~[mysql-connector-java-8.0.11.jar:8.0.11]
    	at org.apache.pulsar.io.jdbc.JdbcAbstractSink.flush(JdbcAbstractSink.java:202) ~[pulsar-io-jdbc-2.5.0.nar-unpacked/:?]
    	at org.apache.pulsar.io.jdbc.JdbcAbstractSink.lambda$open$0(JdbcAbstractSink.java:108) ~[pulsar-io-jdbc-2.5.0.nar-unpacked/:?]
    	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_232]
    	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [?:1.8.0_232]
    	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_232]
    ```
    Looking at the code for the JDBC sink, there was no handling of the case where the field was `null`. The PR adds code to handle that case. It also adds unit tests to cover this for both binary and JSON encoding of the schema.
    
    ### Modifications
    
    When the sink encounters a `null` field value it uses the `setColumnNull` method to properly reflect this in the database row.
    (cherry picked from commit c622de5116f41cdb7174647b09ad70d9f2462bbc)
---
 .../apache/pulsar/io/jdbc/JdbcAutoSchemaSink.java  |  37 ++++-
 .../org/apache/pulsar/io/jdbc/JdbcSinkTest.java    | 150 ++++++++++++++++++++-
 2 files changed, 184 insertions(+), 3 deletions(-)

diff --git a/pulsar-io/jdbc/src/main/java/org/apache/pulsar/io/jdbc/JdbcAutoSchemaSink.java b/pulsar-io/jdbc/src/main/java/org/apache/pulsar/io/jdbc/JdbcAutoSchemaSink.java
index de146c4..a916ca3 100644
--- a/pulsar-io/jdbc/src/main/java/org/apache/pulsar/io/jdbc/JdbcAutoSchemaSink.java
+++ b/pulsar-io/jdbc/src/main/java/org/apache/pulsar/io/jdbc/JdbcAutoSchemaSink.java
@@ -20,6 +20,7 @@
 package org.apache.pulsar.io.jdbc;
 
 import java.sql.PreparedStatement;
+import java.sql.Types;
 import java.util.List;
 
 import com.google.common.collect.Lists;
@@ -60,12 +61,44 @@ public class JdbcAutoSchemaSink extends JdbcAbstractSink<GenericRecord> {
         int index = 1;
         for (ColumnId columnId : columns) {
             String colName = columnId.getName();
-            Object obj = record.getField(colName);
-            setColumnValue(statement, index++, obj);
+            int colType = columnId.getType();
+            if (log.isDebugEnabled()) {
+                log.debug("colName: {} colType: {}", colName, colType);
+            }
+            try {
+                Object obj = record.getField(colName);
+                if (obj != null) {
+                    setColumnValue(statement, index++, obj);
+                } else {
+                    if (log.isDebugEnabled()) {
+                        log.debug("Column {} is null", colName);
+                    }
+                    setColumnNull(statement, index++, colType);
+                }
+            } catch (NullPointerException e) {
+                // With JSON schema field is omitted, so get NPE
+                // In this case we want to set column to Null
+                if (log.isDebugEnabled()) {
+                    log.debug("Column {} is null", colName);
+                }
+                setColumnNull(statement, index++, colType);
+            }
+            
         }
     }
 
+    private static void setColumnNull(PreparedStatement statement, int index, int type) throws Exception {
+        if (log.isDebugEnabled()) {
+            log.debug("Setting column value to null, statement: {}, index: {}, value: {}", statement.toString(), index);
+        }
+        statement.setNull(index, type);
+
+    }
+
     private static void setColumnValue(PreparedStatement statement, int index, Object value) throws Exception {
+
+        log.debug("Setting column value, statement: {}, index: {}, value: {}", statement.toString(), index, value.toString());
+
         if (value instanceof Integer) {
             statement.setInt(index, (Integer) value);
         } else if (value instanceof Long) {
diff --git a/pulsar-io/jdbc/src/test/java/org/apache/pulsar/io/jdbc/JdbcSinkTest.java b/pulsar-io/jdbc/src/test/java/org/apache/pulsar/io/jdbc/JdbcSinkTest.java
index aa2a76a..3c33a16 100644
--- a/pulsar-io/jdbc/src/test/java/org/apache/pulsar/io/jdbc/JdbcSinkTest.java
+++ b/pulsar-io/jdbc/src/test/java/org/apache/pulsar/io/jdbc/JdbcSinkTest.java
@@ -34,7 +34,9 @@ import org.apache.pulsar.client.api.schema.GenericSchema;
 import org.apache.pulsar.client.api.schema.SchemaDefinition;
 import org.apache.pulsar.client.impl.MessageImpl;
 import org.apache.pulsar.client.impl.schema.AvroSchema;
+import org.apache.pulsar.client.impl.schema.JSONSchema;
 import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema;
+import org.apache.pulsar.client.impl.schema.generic.GenericSchemaImpl;
 import org.apache.pulsar.functions.api.Record;
 import org.apache.pulsar.functions.source.PulsarRecord;
 import org.testng.Assert;
@@ -109,13 +111,144 @@ public class JdbcSinkTest {
         jdbcSink.close();
     }
 
+    private void testOpenAndWriteSinkNullValue(Map<String, String> actionProperties) throws Exception {
+        Message<GenericRecord> insertMessage = mock(MessageImpl.class);
+        GenericSchema<GenericRecord> genericAvroSchema;
+        // prepare a foo Record
+        Foo insertObj = new Foo();
+        insertObj.setField1("ValueOfField1");
+        // Not setting field2
+        // Field1 is the key and field3 is used for selecting records 
+        insertObj.setField3(3);
+        AvroSchema<Foo> schema = AvroSchema.of(SchemaDefinition.<Foo>builder().withPojo(Foo.class).withAlwaysAllowNull(true).build());
+
+        byte[] insertBytes = schema.encode(insertObj);
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        Record<GenericRecord> insertRecord = PulsarRecord.<GenericRecord>builder()
+            .message(insertMessage)
+            .topicName("fake_topic_name")
+            .ackFunction(() -> future.complete(null))
+            .build();
+
+        genericAvroSchema = new GenericAvroSchema(schema.getSchemaInfo());
+        when(insertMessage.getValue()).thenReturn(genericAvroSchema.decode(insertBytes));
+        when(insertMessage.getProperties()).thenReturn(actionProperties);
+        log.info("foo:{}, Message.getValue: {}, record.getValue: {}",
+                insertObj.toString(),
+                insertMessage.getValue().toString(),
+                insertRecord.getValue().toString());
+
+        // write should success.
+        jdbcSink.write(insertRecord);
+        log.info("executed write");
+        // sleep to wait backend flush complete
+        future.get(1, TimeUnit.SECONDS);
+
+        // value has been written to db, read it out and verify.
+        String querySql = "SELECT * FROM " + tableName + " WHERE field3=3";
+        int count = sqliteUtils.select(querySql, (resultSet) -> {
+            Assert.assertEquals(insertObj.getField1(), resultSet.getString(1));
+            Assert.assertNull(insertObj.getField2());
+            Assert.assertEquals(insertObj.getField3(), resultSet.getInt(3));
+        });
+        Assert.assertEquals(count, 1);
+
+    }
+
+    private void testOpenAndWriteSinkJson(Map<String, String> actionProperties) throws Exception {
+        Message<GenericRecord> insertMessage = mock(MessageImpl.class);
+        GenericSchema<GenericRecord> genericAvroSchema;
+        // prepare a foo Record
+        Foo insertObj = new Foo();
+        insertObj.setField1("ValueOfField1");
+        insertObj.setField2("ValueOfField2");
+        insertObj.setField3(3);
+        JSONSchema<Foo> schema = JSONSchema.of(SchemaDefinition.<Foo>builder().withPojo(Foo.class).withAlwaysAllowNull(true).build());
+
+        byte[] insertBytes = schema.encode(insertObj);
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        Record<GenericRecord> insertRecord = PulsarRecord.<GenericRecord>builder()
+            .message(insertMessage)
+            .topicName("fake_topic_name")
+            .ackFunction(() -> future.complete(null))
+            .build();
+
+        GenericSchema<GenericRecord> decodeSchema = GenericSchemaImpl.of(schema.getSchemaInfo());
+        when(insertMessage.getValue()).thenReturn(decodeSchema.decode(insertBytes));
+        when(insertMessage.getProperties()).thenReturn(actionProperties);
+        log.info("foo:{}, Message.getValue: {}, record.getValue: {}",
+                insertObj.toString(),
+                insertMessage.getValue().toString(),
+                insertRecord.getValue().toString());
+
+        // write should success.
+        jdbcSink.write(insertRecord);
+        log.info("executed write");
+        // sleep to wait backend flush complete
+        future.get(1, TimeUnit.SECONDS);
+
+        // value has been written to db, read it out and verify.
+        String querySql = "SELECT * FROM " + tableName + " WHERE field3=3";
+        int count = sqliteUtils.select(querySql, (resultSet) -> {
+            Assert.assertEquals(insertObj.getField1(), resultSet.getString(1));
+            Assert.assertEquals(insertObj.getField2(), resultSet.getString(2));
+            Assert.assertEquals(insertObj.getField3(), resultSet.getInt(3));
+        });
+        Assert.assertEquals(count, 1);
+
+    }
+
+    private void testOpenAndWriteSinkNullValueJson(Map<String, String> actionProperties) throws Exception {
+        Message<GenericRecord> insertMessage = mock(MessageImpl.class);
+        GenericSchema<GenericRecord> genericAvroSchema;
+        // prepare a foo Record
+        Foo insertObj = new Foo();
+        insertObj.setField1("ValueOfField1");
+        // Not setting field2
+        // Field1 is the key and field3 is used for selecting records 
+        insertObj.setField3(3);
+        JSONSchema<Foo> schema = JSONSchema.of(SchemaDefinition.<Foo>builder().withPojo(Foo.class).withAlwaysAllowNull(true).build());
+
+        byte[] insertBytes = schema.encode(insertObj);
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        Record<GenericRecord> insertRecord = PulsarRecord.<GenericRecord>builder()
+            .message(insertMessage)
+            .topicName("fake_topic_name")
+            .ackFunction(() -> future.complete(null))
+            .build();
+
+        GenericSchema<GenericRecord> decodeSchema = GenericSchemaImpl.of(schema.getSchemaInfo());
+        when(insertMessage.getValue()).thenReturn(decodeSchema.decode(insertBytes));
+        when(insertMessage.getProperties()).thenReturn(actionProperties);
+        log.info("foo:{}, Message.getValue: {}, record.getValue: {}",
+                insertObj.toString(),
+                insertMessage.getValue().toString(),
+                insertRecord.getValue().toString());
+
+        // write should success.
+        jdbcSink.write(insertRecord);
+        log.info("executed write");
+        // sleep to wait backend flush complete
+        future.get(1, TimeUnit.SECONDS);
+
+        // value has been written to db, read it out and verify.
+        String querySql = "SELECT * FROM " + tableName + " WHERE field3=3";
+        int count = sqliteUtils.select(querySql, (resultSet) -> {
+            Assert.assertEquals(insertObj.getField1(), resultSet.getString(1));
+            Assert.assertNull(insertObj.getField2());
+            Assert.assertEquals(insertObj.getField3(), resultSet.getInt(3));
+        });
+        Assert.assertEquals(count, 1);
+
+    }
+
     private void testOpenAndWriteSink(Map<String, String> actionProperties) throws Exception {
         Message<GenericRecord> insertMessage = mock(MessageImpl.class);
         GenericSchema<GenericRecord> genericAvroSchema;
         // prepare a foo Record
         Foo insertObj = new Foo();
         insertObj.setField1("ValueOfField1");
-        insertObj.setField2("ValueOfField1");
+        insertObj.setField2("ValueOfField2");
         insertObj.setField3(3);
         AvroSchema<Foo> schema = AvroSchema.of(SchemaDefinition.<Foo>builder().withPojo(Foo.class).build());
 
@@ -163,6 +296,21 @@ public class JdbcSinkTest {
     }
 
     @Test
+    public void TestNoActionNullValue() throws Exception {
+        testOpenAndWriteSinkNullValue(ImmutableMap.of("ACTION", "INSERT"));
+    }
+
+    @Test
+    public void TestNoActionNullValueJson() throws Exception {
+        testOpenAndWriteSinkNullValueJson(ImmutableMap.of("ACTION", "INSERT"));
+    }
+
+    @Test
+    public void TestNoActionJson() throws Exception {
+        testOpenAndWriteSinkJson(ImmutableMap.of("ACTION", "INSERT"));
+    }
+
+    @Test
     public void TestUnknownAction() throws Exception {
         Record<GenericRecord> recordRecord = mock(Record.class);
         when(recordRecord.getProperties()).thenReturn(ImmutableMap.of("ACTION", "UNKNOWN"));


[pulsar] 23/38: [PY] Fix serialization of enums with json/avro schemas (#6808)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit e135afbeccbe2d2eab2e7e479708601ace6e1413
Author: Matteo Merli <mm...@apache.org>
AuthorDate: Mon Apr 27 00:49:05 2020 -0700

    [PY] Fix serialization of enums with json/avro schemas (#6808)
    
    ### Motivation
    
    In Python client, the serialization of enums when using the schema is currently broken, throwing error because it's not possible to directly serialize them into json.
    
    Instead, for both Avro and JSON, we need to handle the enum serialization on its own way.
    (cherry picked from commit 5ec9d7b0c030bd377b35ed2128b79c7be93bee26)
---
 pulsar-client-cpp/python/pulsar/schema/__init__.py |  7 ++-
 pulsar-client-cpp/python/pulsar/schema/schema.py   | 18 ++++++-
 pulsar-client-cpp/python/schema_test.py            | 57 ++++++++++++++++++++++
 3 files changed, 76 insertions(+), 6 deletions(-)

diff --git a/pulsar-client-cpp/python/pulsar/schema/__init__.py b/pulsar-client-cpp/python/pulsar/schema/__init__.py
index 096e64a..a38513f 100644
--- a/pulsar-client-cpp/python/pulsar/schema/__init__.py
+++ b/pulsar-client-cpp/python/pulsar/schema/__init__.py
@@ -17,8 +17,7 @@
 # under the License.
 #
 
+from .definition import Record, Field, Null, Boolean, Integer, Long, \
+            Float, Double, Bytes, String, Array, Map
 
-from .definition import *
-from .schema import *
-
-
+from .schema import Schema, BytesSchema, StringSchema, JsonSchema, AvroSchema
diff --git a/pulsar-client-cpp/python/pulsar/schema/schema.py b/pulsar-client-cpp/python/pulsar/schema/schema.py
index 260d7b0..5f69ea2 100644
--- a/pulsar-client-cpp/python/pulsar/schema/schema.py
+++ b/pulsar-client-cpp/python/pulsar/schema/schema.py
@@ -23,6 +23,7 @@ import json
 import fastavro
 import _pulsar
 import io
+import enum
 
 
 class Schema(object):
@@ -78,9 +79,15 @@ class JsonSchema(Schema):
         super(JsonSchema, self).__init__(record_cls, _pulsar.SchemaType.JSON,
                                          record_cls.schema(), 'JSON')
 
+    def _get_serialized_value(self, o):
+        if isinstance(o, enum.Enum):
+            return o.value
+        else:
+            return o.__dict__
+
     def encode(self, obj):
         self._validate_object_type(obj)
-        return json.dumps(obj.__dict__, default=lambda o: o.__dict__, indent=True).encode('utf-8')
+        return json.dumps(obj.__dict__, default=self._get_serialized_value, indent=True).encode('utf-8')
 
     def decode(self, data):
         return self._record_cls(**json.loads(data))
@@ -92,10 +99,17 @@ class AvroSchema(Schema):
                                          record_cls.schema(), 'AVRO')
         self._schema = record_cls.schema()
 
+    def _get_serialized_value(self, x):
+        if isinstance(x, enum.Enum):
+            return x.name
+        else:
+            return x
+
     def encode(self, obj):
         self._validate_object_type(obj)
         buffer = io.BytesIO()
-        fastavro.schemaless_writer(buffer, self._schema, obj.__dict__)
+        m = {k: self._get_serialized_value(v) for k, v in obj.__dict__.items()}
+        fastavro.schemaless_writer(buffer, self._schema, m)
         return buffer.getvalue()
 
     def decode(self, data):
diff --git a/pulsar-client-cpp/python/schema_test.py b/pulsar-client-cpp/python/schema_test.py
index 07c7134..9aead6d 100755
--- a/pulsar-client-cpp/python/schema_test.py
+++ b/pulsar-client-cpp/python/schema_test.py
@@ -419,5 +419,62 @@ class SchemaTest(TestCase):
         self.assertEqual(r, msg.value())
         client.close()
 
+    def test_json_enum(self):
+        class MyEnum(Enum):
+            A = 1
+            B = 2
+            C = 3
+
+        class Example(Record):
+            name = String()
+            v = MyEnum
+
+        topic = 'my-json-enum-topic'
+
+        client = pulsar.Client(self.serviceUrl)
+        producer = client.create_producer(
+                        topic=topic,
+                        schema=JsonSchema(Example))
+
+        consumer = client.subscribe(topic, 'test',
+                                    schema=JsonSchema(Example))
+
+        r = Example(name='test', v=MyEnum.C)
+        producer.send(r)
+
+        msg = consumer.receive()
+
+        self.assertEqual('test', msg.value().name)
+        self.assertEqual(MyEnum.C, MyEnum(msg.value().v))
+        client.close()
+
+    def test_avro_enum(self):
+        class MyEnum(Enum):
+            A = 1
+            B = 2
+            C = 3
+
+        class Example(Record):
+            name = String()
+            v = MyEnum
+
+        topic = 'my-avro-enum-topic'
+
+        client = pulsar.Client(self.serviceUrl)
+        producer = client.create_producer(
+                        topic=topic,
+                        schema=AvroSchema(Example))
+
+        consumer = client.subscribe(topic, 'test',
+                                    schema=AvroSchema(Example))
+
+        r = Example(name='test', v=MyEnum.C)
+        producer.send(r)
+
+        msg = consumer.receive()
+        self.assertEqual(MyEnum.C, MyEnum[msg.value().v])
+        client.close()
+
+
 if __name__ == '__main__':
     main()


[pulsar] 15/38: [Issue 6283][tiered-storage] Offload policies per namespace (#6422)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit c3fa923e46ae37c7bab5503dbd40c3a0c54fe154
Author: Alexandre DUVAL <al...@wanadoo.fr>
AuthorDate: Sat Mar 28 11:35:53 2020 +0100

    [Issue 6283][tiered-storage] Offload policies per namespace (#6422)
    
    Fixes #6283
    
    ### Modifications
    
    Define and use custom deletionLag and threshold for offloadpolicies per ns.
    All is stuff is required for https://github.com/apache/pulsar/pull/6354.
    
    (cherry picked from commit 347d3851b6e62be99b0953a21d8c1a6d502ae111)
---
 .../bookkeeper/mledger/ManagedLedgerConfig.java    |  44 --------
 .../bookkeeper/mledger/impl/ManagedLedgerImpl.java |  82 ++++++++-------
 .../mledger/impl/OffloadLedgerDeleteTest.java      |   6 +-
 .../bookkeeper/mledger/impl/OffloadPrefixTest.java |  22 ++--
 .../pulsar/broker/admin/impl/NamespacesBase.java   |  38 ++++++-
 .../pulsar/broker/service/BrokerService.java       |  23 ++---
 .../pulsar/broker/admin/AdminApiOffloadTest.java   |   2 +-
 .../apache/pulsar/broker/admin/NamespacesTest.java | 112 ++++++++++++++++++++-
 .../pulsar/admin/cli/PulsarAdminToolTest.java      |   5 +-
 .../org/apache/pulsar/admin/cli/CmdNamespaces.java |  33 +++++-
 .../common/policies/data/OffloadPolicies.java      |  18 +++-
 site2/docs/reference-pulsar-admin.md               |   2 +
 12 files changed, 276 insertions(+), 111 deletions(-)

diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java
index 4af66eb..24dbaab 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java
@@ -56,8 +56,6 @@ public class ManagedLedgerConfig {
     private long retentionTimeMs = 0;
     private long retentionSizeInMB = 0;
     private boolean autoSkipNonRecoverableData;
-    private long offloadLedgerDeletionLagMs = TimeUnit.HOURS.toMillis(4);
-    private long offloadAutoTriggerSizeThresholdBytes = -1;
     private long metadataOperationsTimeoutSeconds = 60;
     private long readEntryTimeoutSeconds = 120;
     private long addEntryTimeoutSeconds = 120;
@@ -408,48 +406,6 @@ public class ManagedLedgerConfig {
         return retentionSizeInMB;
     }
 
-    /**
-     * When a ledger is offloaded from bookkeeper storage to longterm storage, the bookkeeper ledger
-     * is not deleted immediately. Instead we wait for a grace period before deleting from bookkeeper.
-     * The offloadLedgerDeleteLag sets this grace period.
-     *
-     * @param lagTime period to wait before deleting offloaded ledgers from bookkeeper
-     * @param unit timeunit for lagTime
-     */
-    public ManagedLedgerConfig setOffloadLedgerDeletionLag(long lagTime, TimeUnit unit) {
-        this.offloadLedgerDeletionLagMs = unit.toMillis(lagTime);
-        return this;
-    }
-
-    /**
-     * Number of milliseconds before an offloaded ledger will be deleted from bookkeeper.
-     *
-     * @return the offload ledger deletion lag time in milliseconds
-     */
-    public long getOffloadLedgerDeletionLagMillis() {
-        return offloadLedgerDeletionLagMs;
-    }
-
-    /**
-     * Size, in bytes, at which the managed ledger will start to automatically offload ledgers to longterm storage.
-     * A negative value disables autotriggering. A threshold of 0 offloads data as soon as possible.
-     * Offloading will not occur if no offloader has been set {@link #setLedgerOffloader(LedgerOffloader)}.
-     * Automatical offloading occurs when the ledger is rolled, and the ledgers up to that point exceed the threshold.
-     *
-     * @param threshold Threshold in bytes at which offload is automatically triggered
-     */
-    public ManagedLedgerConfig setOffloadAutoTriggerSizeThresholdBytes(long threshold) {
-        this.offloadAutoTriggerSizeThresholdBytes = threshold;
-        return this;
-    }
-
-    /**
-     * Size, in bytes, at which offloading will automatically be triggered for this managed ledger.
-     * @return the trigger threshold, in bytes
-     */
-    public long getOffloadAutoTriggerSizeThresholdBytes() {
-        return this.offloadAutoTriggerSizeThresholdBytes;
-    }
 
     /**
      * Skip reading non-recoverable/unreadable data-ledger under managed-ledger's list. It helps when data-ledgers gets
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
index 08b5e5e..a64f8f5 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
@@ -1851,8 +1851,11 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
     }
 
     private void maybeOffloadInBackground(CompletableFuture<PositionImpl> promise) {
-        if (config.getOffloadAutoTriggerSizeThresholdBytes() >= 0) {
-            executor.executeOrdered(name, safeRun(() -> maybeOffload(promise)));
+        if (config.getLedgerOffloader() != null && config.getLedgerOffloader() != NullLedgerOffloader.INSTANCE
+                && config.getLedgerOffloader().getOffloadPolicies() != null) {
+            if (config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes() >= 0) {
+                executor.executeOrdered(name, safeRun(() -> maybeOffload(promise)));
+            }
         }
     }
 
@@ -1871,39 +1874,43 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
                     }
                 });
 
-            long threshold = config.getOffloadAutoTriggerSizeThresholdBytes();
-            long sizeSummed = 0;
-            long alreadyOffloadedSize = 0;
-            long toOffloadSize = 0;
-
-            ConcurrentLinkedDeque<LedgerInfo> toOffload = new ConcurrentLinkedDeque();
-
-            // go through ledger list from newest to oldest and build a list to offload in oldest to newest order
-            for (Map.Entry<Long, LedgerInfo> e : ledgers.descendingMap().entrySet()) {
-                long size = e.getValue().getSize();
-                sizeSummed += size;
-                boolean alreadyOffloaded = e.getValue().hasOffloadContext()
-                    && e.getValue().getOffloadContext().getComplete();
-                if (alreadyOffloaded) {
-                    alreadyOffloadedSize += size;
-                } else if (sizeSummed > threshold) {
-                    toOffloadSize += size;
-                    toOffload.addFirst(e.getValue());
+            if (config.getLedgerOffloader() != null && config.getLedgerOffloader() != NullLedgerOffloader.INSTANCE
+                    && config.getLedgerOffloader().getOffloadPolicies() != null) {
+                long threshold = config.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes();
+
+                long sizeSummed = 0;
+                long alreadyOffloadedSize = 0;
+                long toOffloadSize = 0;
+
+                ConcurrentLinkedDeque<LedgerInfo> toOffload = new ConcurrentLinkedDeque();
+
+                // go through ledger list from newest to oldest and build a list to offload in oldest to newest order
+                for (Map.Entry<Long, LedgerInfo> e : ledgers.descendingMap().entrySet()) {
+                    long size = e.getValue().getSize();
+                    sizeSummed += size;
+                    boolean alreadyOffloaded = e.getValue().hasOffloadContext()
+                            && e.getValue().getOffloadContext().getComplete();
+                    if (alreadyOffloaded) {
+                        alreadyOffloadedSize += size;
+                    } else if (sizeSummed > threshold) {
+                        toOffloadSize += size;
+                        toOffload.addFirst(e.getValue());
+                    }
                 }
-            }
 
-            if (toOffload.size() > 0) {
-                log.info("[{}] Going to automatically offload ledgers {}"
-                         + ", total size = {}, already offloaded = {}, to offload = {}",
-                         name, toOffload.stream().map(l -> l.getLedgerId()).collect(Collectors.toList()),
-                         sizeSummed, alreadyOffloadedSize, toOffloadSize);
-            } else {
-                // offloadLoop will complete immediately with an empty list to offload
-                log.debug("[{}] Nothing to offload, total size = {}, already offloaded = {}, threshold = {}",
-                          name, sizeSummed, alreadyOffloadedSize, threshold);
-            }
+                if (toOffload.size() > 0) {
+                    log.info("[{}] Going to automatically offload ledgers {}"
+                                    + ", total size = {}, already offloaded = {}, to offload = {}",
+                            name, toOffload.stream().map(l -> l.getLedgerId()).collect(Collectors.toList()),
+                            sizeSummed, alreadyOffloadedSize, toOffloadSize);
+                } else {
+                    // offloadLoop will complete immediately with an empty list to offload
+                    log.debug("[{}] Nothing to offload, total size = {}, already offloaded = {}, threshold = {}",
+                            name, sizeSummed, alreadyOffloadedSize, threshold);
+                }
 
-            offloadLoop(unlockingPromise, toOffload, PositionImpl.latest, Optional.empty());
+                offloadLoop(unlockingPromise, toOffload, PositionImpl.latest, Optional.empty());
+            }
         }
     }
 
@@ -1925,8 +1932,15 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
 
     private boolean isOffloadedNeedsDelete(OffloadContext offload) {
         long elapsedMs = clock.millis() - offload.getTimestamp();
-        return offload.getComplete() && !offload.getBookkeeperDeleted()
-                && elapsedMs > config.getOffloadLedgerDeletionLagMillis();
+
+        if (config.getLedgerOffloader() != null && config.getLedgerOffloader() != NullLedgerOffloader.INSTANCE
+                && config.getLedgerOffloader().getOffloadPolicies() != null) {
+            return offload.getComplete() && !offload.getBookkeeperDeleted()
+                    && elapsedMs > config.getLedgerOffloader()
+                    .getOffloadPolicies().getManagedLedgerOffloadDeletionLagInMillis();
+        } else {
+            return false;
+        }
     }
 
     /**
diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java
index 02fff69..8fbc588 100644
--- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java
@@ -48,7 +48,7 @@ public class OffloadLedgerDeleteTest extends MockedBookKeeperTestCase {
         config.setMinimumRolloverTime(0, TimeUnit.SECONDS);
         config.setRetentionTime(10, TimeUnit.MINUTES);
         config.setRetentionSizeInMB(10);
-        config.setOffloadLedgerDeletionLag(5, TimeUnit.MINUTES);
+        offloader.getOffloadPolicies().setManagedLedgerOffloadDeletionLagInMillis(new Long(300000));
         config.setLedgerOffloader(offloader);
         config.setClock(clock);
 
@@ -109,8 +109,8 @@ public class OffloadLedgerDeleteTest extends MockedBookKeeperTestCase {
         config.setMaxEntriesPerLedger(10);
         config.setMinimumRolloverTime(0, TimeUnit.SECONDS);
         config.setRetentionTime(5, TimeUnit.MINUTES);
-        config.setOffloadLedgerDeletionLag(10, TimeUnit.MINUTES);
         config.setRetentionSizeInMB(10);
+        offloader.getOffloadPolicies().setManagedLedgerOffloadDeletionLagInMillis(new Long(600000));
         config.setLedgerOffloader(offloader);
         config.setClock(clock);
 
@@ -157,7 +157,7 @@ public class OffloadLedgerDeleteTest extends MockedBookKeeperTestCase {
         config.setMaxEntriesPerLedger(10);
         config.setMinimumRolloverTime(0, TimeUnit.SECONDS);
         config.setRetentionTime(10, TimeUnit.MINUTES);
-        config.setOffloadLedgerDeletionLag(5, TimeUnit.MINUTES);
+        offloader.getOffloadPolicies().setManagedLedgerOffloadDeletionLagInMillis(new Long(300000));
         config.setLedgerOffloader(offloader);
         config.setClock(clock);
 
diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java
index 997f3f6..97bab56 100644
--- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java
@@ -608,9 +608,12 @@ public class OffloadPrefixTest extends MockedBookKeeperTestCase {
         config.setMaxEntriesPerLedger(10);
         config.setMinimumRolloverTime(0, TimeUnit.SECONDS);
         config.setRetentionTime(0, TimeUnit.MINUTES);
+        offloader.getOffloadPolicies().setManagedLedgerOffloadDeletionLagInMillis(new Long(100));
+        offloader.getOffloadPolicies().setManagedLedgerOffloadThresholdInBytes(100);
         config.setLedgerOffloader(offloader);
         ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("my_test_ledger", config);
         ManagedCursor cursor = ledger.openCursor("foobar");
+
         for (int i = 0; i < 15; i++) {
             String content = "entry-" + i;
             ledger.addEntry(content.getBytes());
@@ -746,9 +749,9 @@ public class OffloadPrefixTest extends MockedBookKeeperTestCase {
         MockLedgerOffloader offloader = new MockLedgerOffloader();
         ManagedLedgerConfig config = new ManagedLedgerConfig();
         config.setMaxEntriesPerLedger(10);
-        config.setOffloadAutoTriggerSizeThresholdBytes(100);
         config.setRetentionTime(10, TimeUnit.MINUTES);
         config.setRetentionSizeInMB(10);
+        offloader.getOffloadPolicies().setManagedLedgerOffloadThresholdInBytes(100);
         config.setLedgerOffloader(offloader);
 
         ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("my_test_ledger", config);
@@ -782,9 +785,9 @@ public class OffloadPrefixTest extends MockedBookKeeperTestCase {
 
         ManagedLedgerConfig config = new ManagedLedgerConfig();
         config.setMaxEntriesPerLedger(10);
-        config.setOffloadAutoTriggerSizeThresholdBytes(100);
         config.setRetentionTime(10, TimeUnit.MINUTES);
         config.setRetentionSizeInMB(10);
+        offloader.getOffloadPolicies().setManagedLedgerOffloadThresholdInBytes(100);
         config.setLedgerOffloader(offloader);
 
         ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("my_test_ledger", config);
@@ -843,9 +846,9 @@ public class OffloadPrefixTest extends MockedBookKeeperTestCase {
 
         ManagedLedgerConfig config = new ManagedLedgerConfig();
         config.setMaxEntriesPerLedger(10);
-        config.setOffloadAutoTriggerSizeThresholdBytes(100);
         config.setRetentionTime(10, TimeUnit.MINUTES);
         config.setRetentionSizeInMB(10);
+        offloader.getOffloadPolicies().setManagedLedgerOffloadThresholdInBytes(100);
         config.setLedgerOffloader(offloader);
 
         ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("my_test_ledger", config);
@@ -894,9 +897,9 @@ public class OffloadPrefixTest extends MockedBookKeeperTestCase {
 
         ManagedLedgerConfig config = new ManagedLedgerConfig();
         config.setMaxEntriesPerLedger(10);
-        config.setOffloadAutoTriggerSizeThresholdBytes(100);
         config.setRetentionTime(10, TimeUnit.MINUTES);
         config.setRetentionSizeInMB(10);
+        offloader.getOffloadPolicies().setManagedLedgerOffloadThresholdInBytes(100);
         config.setLedgerOffloader(offloader);
 
         ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("my_test_ledger", config);
@@ -926,13 +929,12 @@ public class OffloadPrefixTest extends MockedBookKeeperTestCase {
 
     @Test
     public void offloadAsSoonAsClosed() throws Exception {
-
         MockLedgerOffloader offloader = new MockLedgerOffloader();
         ManagedLedgerConfig config = new ManagedLedgerConfig();
         config.setMaxEntriesPerLedger(10);
-        config.setOffloadAutoTriggerSizeThresholdBytes(0);
         config.setRetentionTime(10, TimeUnit.MINUTES);
         config.setRetentionSizeInMB(10);
+        offloader.getOffloadPolicies().setManagedLedgerOffloadThresholdInBytes(0);
         config.setLedgerOffloader(offloader);
 
         ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("my_test_ledger", config);
@@ -988,6 +990,12 @@ public class OffloadPrefixTest extends MockedBookKeeperTestCase {
             return deletes.keySet();
         }
 
+        OffloadPolicies offloadPolicies = OffloadPolicies.create("S3", "", "", "",
+                OffloadPolicies.DEFAULT_MAX_BLOCK_SIZE_IN_BYTES,
+                OffloadPolicies.DEFAULT_READ_BUFFER_SIZE_IN_BYTES,
+                OffloadPolicies.DEFAULT_OFFLOAD_THRESHOLD_IN_BYTES,
+                OffloadPolicies.DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS);
+
         @Override
         public String getOffloadDriverName() {
             return "mock";
@@ -1029,7 +1037,7 @@ public class OffloadPrefixTest extends MockedBookKeeperTestCase {
 
         @Override
         public OffloadPolicies getOffloadPolicies() {
-            return null;
+            return offloadPolicies;
         }
 
         @Override
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
index 619a5cd..01927f2 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
@@ -2046,7 +2046,12 @@ public abstract class NamespacesBase extends AdminResource {
 
     protected long internalGetOffloadThreshold() {
         validateAdminAccessForTenant(namespaceName.getTenant());
-        return getNamespacePolicies(namespaceName).offload_threshold;
+        Policies policies = getNamespacePolicies(namespaceName);
+        if (policies.offload_policies == null) {
+            return policies.offload_threshold;
+        } else {
+            return policies.offload_policies.getManagedLedgerOffloadThresholdInBytes();
+        }
     }
 
     protected void internalSetOffloadThreshold(long newThreshold) {
@@ -2057,8 +2062,13 @@ public abstract class NamespacesBase extends AdminResource {
             Stat nodeStat = new Stat();
             final String path = path(POLICIES, namespaceName.toString());
             byte[] content = globalZk().getData(path, null, nodeStat);
+
             Policies policies = jsonMapper().readValue(content, Policies.class);
+            if (policies.offload_policies != null) {
+                policies.offload_policies.setManagedLedgerOffloadThresholdInBytes(newThreshold);
+            }
             policies.offload_threshold = newThreshold;
+
             globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion());
             policiesCache().invalidate(path(POLICIES, namespaceName.toString()));
             log.info("[{}] Successfully updated offloadThreshold configuration: namespace={}, value={}",
@@ -2083,7 +2093,12 @@ public abstract class NamespacesBase extends AdminResource {
 
     protected Long internalGetOffloadDeletionLag() {
         validateAdminAccessForTenant(namespaceName.getTenant());
-        return getNamespacePolicies(namespaceName).offload_deletion_lag_ms;
+        Policies policies = getNamespacePolicies(namespaceName);
+        if (policies.offload_policies == null) {
+            return policies.offload_deletion_lag_ms;
+        } else {
+            return policies.offload_policies.getManagedLedgerOffloadDeletionLagInMillis();
+        }
     }
 
     protected void internalSetOffloadDeletionLag(Long newDeletionLagMs) {
@@ -2094,8 +2109,13 @@ public abstract class NamespacesBase extends AdminResource {
             Stat nodeStat = new Stat();
             final String path = path(POLICIES, namespaceName.toString());
             byte[] content = globalZk().getData(path, null, nodeStat);
+
             Policies policies = jsonMapper().readValue(content, Policies.class);
+            if (policies.offload_policies != null) {
+                policies.offload_policies.setManagedLedgerOffloadDeletionLagInMillis(newDeletionLagMs);
+            }
             policies.offload_deletion_lag_ms = newDeletionLagMs;
+
             globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion());
             policiesCache().invalidate(path(POLICIES, namespaceName.toString()));
             log.info("[{}] Successfully updated offloadDeletionLagMs configuration: namespace={}, value={}",
@@ -2234,6 +2254,20 @@ public abstract class NamespacesBase extends AdminResource {
             final String path = path(POLICIES, namespaceName.toString());
             byte[] content = globalZk().getData(path, null, nodeStat);
             Policies policies = jsonMapper().readValue(content, Policies.class);
+
+            if (offloadPolicies.getManagedLedgerOffloadDeletionLagInMillis()
+                    .equals(OffloadPolicies.DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS)) {
+                offloadPolicies.setManagedLedgerOffloadDeletionLagInMillis(policies.offload_deletion_lag_ms);
+            } else {
+                policies.offload_deletion_lag_ms = offloadPolicies.getManagedLedgerOffloadDeletionLagInMillis();
+            }
+            if (offloadPolicies.getManagedLedgerOffloadThresholdInBytes() ==
+                    OffloadPolicies.DEFAULT_OFFLOAD_THRESHOLD_IN_BYTES) {
+                offloadPolicies.setManagedLedgerOffloadThresholdInBytes(policies.offload_threshold);
+            } else {
+                policies.offload_threshold = offloadPolicies.getManagedLedgerOffloadThresholdInBytes();
+            }
+
             policies.offload_policies = offloadPolicies;
             globalZk().setData(path, jsonMapper().writeValueAsBytes(policies), nodeStat.getVersion(),
                     (rc, path1, ctx, stat) -> {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
index 085bae4..3f0f9cf 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
@@ -1032,19 +1032,18 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
             managedLedgerConfig.setRetentionSizeInMB(retentionPolicies.getRetentionSizeInMB());
 
             OffloadPolicies offloadPolicies = policies.map(p -> p.offload_policies).orElse(null);
+
+            if (offloadPolicies == null) {
+                offloadPolicies = new OffloadPolicies();
+                offloadPolicies.setManagedLedgerOffloadDriver(pulsar.getConfiguration().getManagedLedgerOffloadDriver());
+                offloadPolicies.setManagedLedgerOffloadThresholdInBytes(
+                        pulsar.getConfiguration().getManagedLedgerOffloadAutoTriggerSizeThresholdBytes()
+                );
+                offloadPolicies.setManagedLedgerOffloadDeletionLagInMillis(
+                        pulsar.getConfiguration().getManagedLedgerOffloadDeletionLagMs()
+                );
+            }
             managedLedgerConfig.setLedgerOffloader(pulsar.getManagedLedgerOffloader(namespace, offloadPolicies));
-            policies.ifPresent(p -> {
-                    long lag = serviceConfig.getManagedLedgerOffloadDeletionLagMs();
-                    if (p.offload_deletion_lag_ms != null) {
-                        lag = p.offload_deletion_lag_ms;
-                    }
-                    long bytes = serviceConfig.getManagedLedgerOffloadAutoTriggerSizeThresholdBytes();
-                    if (p.offload_threshold != -1L) {
-                        bytes = p.offload_threshold;
-                    }
-                    managedLedgerConfig.setOffloadLedgerDeletionLag(lag, TimeUnit.MILLISECONDS);
-                    managedLedgerConfig.setOffloadAutoTriggerSizeThresholdBytes(bytes);
-                });
 
             future.complete(managedLedgerConfig);
         }, (exception) -> future.completeExceptionally(exception)));
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java
index 25c45df..76272ef 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java
@@ -149,7 +149,7 @@ public class AdminApiOffloadTest extends MockedPulsarServiceBaseTest {
         String endpoint = "test-endpoint";
 
         OffloadPolicies offload1 = OffloadPolicies.create(
-                driver, region, bucket, endpoint, 100, 100);
+                driver, region, bucket, endpoint, 100, 100, -1, null);
         admin.namespaces().setOffloadPolicies(namespaceName, offload1);
         OffloadPolicies offload2 = admin.namespaces().getOffloadPolicies(namespaceName);
         Assert.assertEquals(offload1, offload2);
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 a607921..5f39812 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
@@ -43,8 +43,12 @@ import java.net.URI;
 import java.net.URL;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
 
 import javax.ws.rs.ClientErrorException;
 import javax.ws.rs.WebApplicationException;
@@ -54,6 +58,8 @@ import javax.ws.rs.core.Response.Status;
 import javax.ws.rs.core.UriBuilder;
 import javax.ws.rs.core.UriInfo;
 
+import org.apache.bookkeeper.client.api.ReadHandle;
+import org.apache.bookkeeper.mledger.LedgerOffloader;
 import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
 import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.pulsar.broker.admin.v1.Namespaces;
@@ -75,6 +81,7 @@ import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.policies.data.BundlesData;
 import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.OffloadPolicies;
 import org.apache.pulsar.common.policies.data.PersistencePolicies;
 import org.apache.pulsar.common.policies.data.Policies;
 import org.apache.pulsar.common.policies.data.RetentionPolicies;
@@ -1073,6 +1080,74 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
         admin.tenants().deleteTenant("my-tenants");
     }
 
+    class MockLedgerOffloader implements LedgerOffloader {
+        ConcurrentHashMap<Long, UUID> offloads = new ConcurrentHashMap<Long, UUID>();
+        ConcurrentHashMap<Long, UUID> deletes = new ConcurrentHashMap<Long, UUID>();
+
+        Set<Long> offloadedLedgers() {
+            return offloads.keySet();
+        }
+
+        Set<Long> deletedOffloads() {
+            return deletes.keySet();
+        }
+
+        OffloadPolicies offloadPolicies;
+
+        public MockLedgerOffloader(OffloadPolicies offloadPolicies) {
+            this.offloadPolicies = offloadPolicies;
+        }
+
+        @Override
+        public String getOffloadDriverName() {
+            return "mock";
+        }
+
+        @Override
+        public CompletableFuture<Void> offload(ReadHandle ledger,
+                                               UUID uuid,
+                                               Map<String, String> extraMetadata) {
+            CompletableFuture<Void> promise = new CompletableFuture<>();
+            if (offloads.putIfAbsent(ledger.getId(), uuid) == null) {
+                promise.complete(null);
+            } else {
+                promise.completeExceptionally(new Exception("Already exists exception"));
+            }
+            return promise;
+        }
+
+        @Override
+        public CompletableFuture<ReadHandle> readOffloaded(long ledgerId, UUID uuid,
+                                                           Map<String, String> offloadDriverMetadata) {
+            CompletableFuture<ReadHandle> promise = new CompletableFuture<>();
+            promise.completeExceptionally(new UnsupportedOperationException());
+            return promise;
+        }
+
+        @Override
+        public CompletableFuture<Void> deleteOffloaded(long ledgerId, UUID uuid,
+                                                       Map<String, String> offloadDriverMetadata) {
+            CompletableFuture<Void> promise = new CompletableFuture<>();
+            if (offloads.remove(ledgerId, uuid)) {
+                deletes.put(ledgerId, uuid);
+                promise.complete(null);
+            } else {
+                promise.completeExceptionally(new Exception("Not found"));
+            }
+            return promise;
+        };
+
+        @Override
+        public OffloadPolicies getOffloadPolicies() {
+            return offloadPolicies;
+        }
+
+        @Override
+        public void close() {
+
+        }
+    }
+
     @Test
     public void testSetOffloadThreshold() throws Exception {
         TopicName topicName = TopicName.get("persistent", this.testTenant, "offload", "offload-topic");
@@ -1088,25 +1163,54 @@ public class NamespacesTest extends MockedPulsarServiceBaseTest {
         assertEquals(-1, admin.namespaces().getOffloadThreshold(namespace));
         // the ledger config should have the expected value
         ManagedLedgerConfig ledgerConf = pulsar.getBrokerService().getManagedLedgerConfig(topicName).get();
-        assertEquals(ledgerConf.getOffloadAutoTriggerSizeThresholdBytes(), 1);
+        MockLedgerOffloader offloader = new MockLedgerOffloader(OffloadPolicies.create("S3", "", "", "",
+                OffloadPolicies.DEFAULT_MAX_BLOCK_SIZE_IN_BYTES,
+                OffloadPolicies.DEFAULT_READ_BUFFER_SIZE_IN_BYTES,
+                admin.namespaces().getOffloadThreshold(namespace),
+                pulsar.getConfiguration().getManagedLedgerOffloadDeletionLagMs()));
+        ledgerConf.setLedgerOffloader(offloader);
+        assertEquals(ledgerConf.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes(),
+                -1);
 
         // set an override for the namespace
         admin.namespaces().setOffloadThreshold(namespace, 100);
         assertEquals(100, admin.namespaces().getOffloadThreshold(namespace));
         ledgerConf = pulsar.getBrokerService().getManagedLedgerConfig(topicName).get();
-        assertEquals(ledgerConf.getOffloadAutoTriggerSizeThresholdBytes(), 100);
+        admin.namespaces().getOffloadPolicies(namespace);
+        offloader = new MockLedgerOffloader(OffloadPolicies.create("S3", "", "", "",
+                OffloadPolicies.DEFAULT_MAX_BLOCK_SIZE_IN_BYTES,
+                OffloadPolicies.DEFAULT_READ_BUFFER_SIZE_IN_BYTES,
+                admin.namespaces().getOffloadThreshold(namespace),
+                pulsar.getConfiguration().getManagedLedgerOffloadDeletionLagMs()));
+        ledgerConf.setLedgerOffloader(offloader);
+        assertEquals(ledgerConf.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes(),
+                100);
 
         // set another negative value to disable
         admin.namespaces().setOffloadThreshold(namespace, -2);
         assertEquals(-2, admin.namespaces().getOffloadThreshold(namespace));
         ledgerConf = pulsar.getBrokerService().getManagedLedgerConfig(topicName).get();
-        assertEquals(ledgerConf.getOffloadAutoTriggerSizeThresholdBytes(), -2);
+        offloader = new MockLedgerOffloader(OffloadPolicies.create("S3", "", "", "",
+                OffloadPolicies.DEFAULT_MAX_BLOCK_SIZE_IN_BYTES,
+                OffloadPolicies.DEFAULT_READ_BUFFER_SIZE_IN_BYTES,
+                admin.namespaces().getOffloadThreshold(namespace),
+                pulsar.getConfiguration().getManagedLedgerOffloadDeletionLagMs()));
+        ledgerConf.setLedgerOffloader(offloader);
+        assertEquals(ledgerConf.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes(),
+                -2);
 
         // set back to -1 and fall back to default
         admin.namespaces().setOffloadThreshold(namespace, -1);
         assertEquals(-1, admin.namespaces().getOffloadThreshold(namespace));
         ledgerConf = pulsar.getBrokerService().getManagedLedgerConfig(topicName).get();
-        assertEquals(ledgerConf.getOffloadAutoTriggerSizeThresholdBytes(), 1);
+        offloader = new MockLedgerOffloader(OffloadPolicies.create("S3", "", "", "",
+                OffloadPolicies.DEFAULT_MAX_BLOCK_SIZE_IN_BYTES,
+                OffloadPolicies.DEFAULT_READ_BUFFER_SIZE_IN_BYTES,
+                admin.namespaces().getOffloadThreshold(namespace),
+                pulsar.getConfiguration().getManagedLedgerOffloadDeletionLagMs()));
+        ledgerConf.setLedgerOffloader(offloader);
+        assertEquals(ledgerConf.getLedgerOffloader().getOffloadPolicies().getManagedLedgerOffloadThresholdInBytes(),
+                -1);
 
         // cleanup
         admin.topics().delete(topicName.toString(), true);
diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java
index d9bd2f7..9318f15 100644
--- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java
+++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java
@@ -489,10 +489,11 @@ public class PulsarAdminToolTest {
         namespaces.run(split("clear-offload-deletion-lag myprop/clust/ns1"));
         verify(mockNamespaces).clearOffloadDeleteLag("myprop/clust/ns1");
 
-        namespaces.run(split("set-offload-policies myprop/clust/ns1 -r test-region -d aws-s3 -b test-bucket -e http://test.endpoint -mbs 32M -rbs 5M"));
+        namespaces.run(split("set-offload-policies myprop/clust/ns1 -r test-region -d aws-s3 -b test-bucket -e http://test.endpoint -mbs 32M -rbs 5M -oat 10M -oae 10s"));
         verify(mockNamespaces).setOffloadPolicies("myprop/clust/ns1",
                 OffloadPolicies.create("aws-s3", "test-region", "test-bucket",
-                        "http://test.endpoint", 32 * 1024 * 1024, 5 * 1024 * 1024));
+                        "http://test.endpoint", 32 * 1024 * 1024, 5 * 1024 * 1024,
+                        10 * 1024 * 1024, 10000L));
 
         namespaces.run(split("get-offload-policies myprop/clust/ns1"));
         verify(mockNamespaces).getOffloadPolicies("myprop/clust/ns1");
diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java
index 0105d0e..3dd4a6f 100644
--- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java
+++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java
@@ -1338,6 +1338,18 @@ public class CmdNamespaces extends CmdBase {
                 required = false)
         private String readBufferSizeStr;
 
+        @Parameter(
+                names = {"--offloadAfterElapsed", "-oae"},
+                description = "Offload after elapsed in minutes (or minutes, hours,days,weeks eg: 100m, 3h, 2d, 5w).",
+                required = false)
+        private String offloadAfterElapsedStr;
+
+        @Parameter(
+                names = {"--offloadAfterThreshold", "-oat"},
+                description = "Offload after threshold size (eg: 1M, 5M)",
+                required = false)
+        private String offloadAfterThresholdStr;
+
         private final String[] DRIVER_NAMES = {"S3", "aws-s3", "google-cloud-storage"};
 
         public boolean driverSupported(String driver) {
@@ -1399,8 +1411,27 @@ public class CmdNamespaces extends CmdBase {
                 }
             }
 
+            Long offloadAfterElapsedInMillis = OffloadPolicies.DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS;
+            if (StringUtils.isNotEmpty(offloadAfterElapsedStr)) {
+                Long offloadAfterElapsed = TimeUnit.SECONDS.toMillis(RelativeTimeUtil.parseRelativeTimeInSeconds(offloadAfterElapsedStr));
+                if (positiveCheck("OffloadAfterElapsed", offloadAfterElapsed)
+                        && maxValueCheck("OffloadAfterElapsed", offloadAfterElapsed, Long.MAX_VALUE)) {
+                    offloadAfterElapsedInMillis = new Long(offloadAfterElapsed);
+                }
+            }
+
+            long offloadAfterThresholdInBytes = OffloadPolicies.DEFAULT_OFFLOAD_THRESHOLD_IN_BYTES;
+            if (StringUtils.isNotEmpty(offloadAfterThresholdStr)) {
+                long offloadAfterThreshold = validateSizeString(offloadAfterThresholdStr);
+                if (positiveCheck("OffloadAfterThreshold", offloadAfterThreshold)
+                        && maxValueCheck("OffloadAfterThreshold", offloadAfterThreshold, Long.MAX_VALUE)) {
+                    offloadAfterThresholdInBytes = new Long(offloadAfterThreshold);
+                }
+            }
+
             OffloadPolicies offloadPolicies = OffloadPolicies.create(driver, region, bucket, endpoint,
-                    maxBlockSizeInBytes, readBufferSizeInBytes);
+                    maxBlockSizeInBytes, readBufferSizeInBytes, offloadAfterThresholdInBytes,
+                    offloadAfterElapsedInMillis);
             admin.namespaces().setOffloadPolicies(namespace, offloadPolicies);
         }
     }
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPolicies.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPolicies.java
index f46b44f..5ccb75c 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPolicies.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPolicies.java
@@ -39,11 +39,15 @@ public class OffloadPolicies {
     public final static int DEFAULT_OFFLOAD_MAX_THREADS = 2;
     public final static String[] DRIVER_NAMES = {"S3", "aws-s3", "google-cloud-storage", "filesystem"};
     public final static String DEFAULT_OFFLOADER_DIRECTORY = "./offloaders";
+    public final static long DEFAULT_OFFLOAD_THRESHOLD_IN_BYTES = -1;
+    public final static Long DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS = null;
 
     // common config
     private String offloadersDirectory = DEFAULT_OFFLOADER_DIRECTORY;
     private String managedLedgerOffloadDriver = null;
     private int managedLedgerOffloadMaxThreads = DEFAULT_OFFLOAD_MAX_THREADS;
+    private long managedLedgerOffloadThresholdInBytes = DEFAULT_OFFLOAD_THRESHOLD_IN_BYTES;
+    private Long managedLedgerOffloadDeletionLagInMillis = DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS;
 
     // s3 config, set by service configuration or cli
     private String s3ManagedLedgerOffloadRegion = null;
@@ -68,9 +72,13 @@ public class OffloadPolicies {
     private String fileSystemURI = null;
 
     public static OffloadPolicies create(String driver, String region, String bucket, String endpoint,
-                                         int maxBlockSizeInBytes, int readBufferSizeInBytes) {
+                                         int maxBlockSizeInBytes, int readBufferSizeInBytes,
+                                         long offloadThresholdInBytes, Long offloadDeletionLagInMillis) {
         OffloadPolicies offloadPolicies = new OffloadPolicies();
         offloadPolicies.setManagedLedgerOffloadDriver(driver);
+        offloadPolicies.setManagedLedgerOffloadThresholdInBytes(offloadThresholdInBytes);
+        offloadPolicies.setManagedLedgerOffloadDeletionLagInMillis(offloadDeletionLagInMillis);
+
         if (driver.equalsIgnoreCase(DRIVER_NAMES[0]) || driver.equalsIgnoreCase(DRIVER_NAMES[1])) {
             offloadPolicies.setS3ManagedLedgerOffloadRegion(region);
             offloadPolicies.setS3ManagedLedgerOffloadBucket(bucket);
@@ -153,6 +161,8 @@ public class OffloadPolicies {
         return Objects.hash(
                 managedLedgerOffloadDriver,
                 managedLedgerOffloadMaxThreads,
+                managedLedgerOffloadThresholdInBytes,
+                managedLedgerOffloadDeletionLagInMillis,
                 s3ManagedLedgerOffloadRegion,
                 s3ManagedLedgerOffloadBucket,
                 s3ManagedLedgerOffloadServiceEndpoint,
@@ -180,6 +190,10 @@ public class OffloadPolicies {
         OffloadPolicies other = (OffloadPolicies) obj;
         return Objects.equals(managedLedgerOffloadDriver, other.getManagedLedgerOffloadDriver())
                 && Objects.equals(managedLedgerOffloadMaxThreads, other.getManagedLedgerOffloadMaxThreads())
+                && Objects.equals(managedLedgerOffloadThresholdInBytes,
+                    other.getManagedLedgerOffloadThresholdInBytes())
+                && Objects.equals(managedLedgerOffloadDeletionLagInMillis,
+                    other.getManagedLedgerOffloadDeletionLagInMillis())
                 && Objects.equals(s3ManagedLedgerOffloadRegion, other.getS3ManagedLedgerOffloadRegion())
                 && Objects.equals(s3ManagedLedgerOffloadBucket, other.getS3ManagedLedgerOffloadBucket())
                 && Objects.equals(s3ManagedLedgerOffloadServiceEndpoint,
@@ -208,6 +222,8 @@ public class OffloadPolicies {
         return MoreObjects.toStringHelper(this)
                 .add("managedLedgerOffloadDriver", managedLedgerOffloadDriver)
                 .add("managedLedgerOffloadMaxThreads", managedLedgerOffloadMaxThreads)
+                .add("managedLedgerOffloadThresholdInBytes", managedLedgerOffloadThresholdInBytes)
+                .add("managedLedgerOffloadDeletionLagInMillis", managedLedgerOffloadDeletionLagInMillis)
                 .add("s3ManagedLedgerOffloadRegion", s3ManagedLedgerOffloadRegion)
                 .add("s3ManagedLedgerOffloadBucket", s3ManagedLedgerOffloadBucket)
                 .add("s3ManagedLedgerOffloadServiceEndpoint", s3ManagedLedgerOffloadServiceEndpoint)
diff --git a/site2/docs/reference-pulsar-admin.md b/site2/docs/reference-pulsar-admin.md
index e97acc0..ef0dfa7 100644
--- a/site2/docs/reference-pulsar-admin.md
+++ b/site2/docs/reference-pulsar-admin.md
@@ -2271,3 +2271,5 @@ Options
 |`-e`, `--endpoint`|Alternative endpoint to connect to||
 |`-mbs`, `--maxBlockSize`|Max block size|64MB|
 |`-rbs`, `--readBufferSize`|Read buffer size|1MB|
+|`-oat`, `--offloadAfterThreshold`|Offload after threshold size (eg: 1M, 5M)||
+|`-oae`, `--offloadAfterElapsed`|Offload after elapsed in millis (or minutes, hours,days,weeks eg: 100m, 3h, 2d, 5w).||


[pulsar] 13/38: Fix localrunner netty dependency issue and add default log4j2 config file (#6779)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit d1fe63604731dca1c6e8b229e2a039f211227cfe
Author: Boyang Jerry Peng <je...@gmail.com>
AuthorDate: Tue Apr 21 22:18:48 2020 -0700

    Fix localrunner netty dependency issue and add default log4j2 config file (#6779)
    
    ### Motivation
    
    When pulling in pulsar-functions-local-runner as a dependency and attempting to run functions locally, netty libraries are missing and will cause class not found exception.
    
    There is also no default log4j2 config file for pulsar-functions-local-runner.  We should add a config file to log to console by default.
    (cherry picked from commit 9dc648a8d9f645d1b500d2955b3d15538610ec4c)
---
 pulsar-functions/localrun-shaded/pom.xml           | 14 ---------
 .../localrun/src/main/resources/log4j2.xml         | 33 ++++++++++++++++++++++
 2 files changed, 33 insertions(+), 14 deletions(-)

diff --git a/pulsar-functions/localrun-shaded/pom.xml b/pulsar-functions/localrun-shaded/pom.xml
index 6a89792..d5e8cb7 100644
--- a/pulsar-functions/localrun-shaded/pom.xml
+++ b/pulsar-functions/localrun-shaded/pom.xml
@@ -58,20 +58,6 @@
                                 <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
                                 <transformer implementation="org.apache.maven.plugins.shade.resource.PluginXmlResourceTransformer" />
                             </transformers>
-                            <artifactSet>
-                                <excludes>
-                                    <exclude>io.netty:netty-common</exclude>
-                                    <exclude>io.netty:netty-buffer</exclude>
-                                    <exclude>io.netty:netty-codec-http2</exclude>
-                                    <exclude>io.netty:netty-codec-http</exclude>
-                                    <exclude>io.netty:netty-codec-socks</exclude>
-                                    <exclude>io.netty:netty-codec</exclude>
-                                    <exclude>io.netty:netty-handler</exclude>
-                                    <exclude>io.netty:netty-handler-proxy</exclude>
-                                    <exclude>io.netty:netty-transport</exclude>
-                                    <exclude>io.netty:netty-resolver</exclude>
-                                </excludes>
-                            </artifactSet>
                             <filters>
                                 <filter>
                                     <!-- Shading signed JARs will fail without
diff --git a/pulsar-functions/localrun/src/main/resources/log4j2.xml b/pulsar-functions/localrun/src/main/resources/log4j2.xml
new file mode 100644
index 0000000..c986285
--- /dev/null
+++ b/pulsar-functions/localrun/src/main/resources/log4j2.xml
@@ -0,0 +1,33 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    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.
+
+-->
+<Configuration status="WARN">
+    <Appenders>
+        <Console name="Console" target="SYSTEM_OUT">
+            <PatternLayout pattern="%d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n"/>
+        </Console>
+    </Appenders>
+    <Loggers>
+        <Root level="info">
+            <AppenderRef ref="Console"/>
+        </Root>
+    </Loggers>
+</Configuration>
\ No newline at end of file


[pulsar] 03/38: ISSUE-6612 FIX: parse long field in GenricJsonRecord (#6612) (#6622)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 8035ae04a3f7f96e9829957dd5d3185ad4049b6c
Author: Lin Yiming <ro...@gmail.com>
AuthorDate: Mon Apr 6 03:17:12 2020 +0800

    ISSUE-6612 FIX: parse long field in GenricJsonRecord (#6612) (#6622)
    
    Fixes #6612
    
    ### Motivation
    
    If message sent in json schema, long field will be decoded as int if its value below Integer.MAX_VALUE, other wise decoded as string.
    For example, the json message below:
    ```json
    {
        "timestamp": 1585204833128
    }
    ```
    will be decoded as
    
    ```json
    {
        "timestamp": "1585204833128"
    }
    ```
    
    ### Modifications
    
    Add field type check in GenericJsonRecord
    (cherry picked from commit 1aad3b73efe2a2afe0cd2598987f112416117159)
---
 .../impl/schema/generic/GenericJsonRecord.java     | 12 +++--
 .../impl/schema/generic/GenericJsonRecordTest.java | 57 ++++++++++++++++++++++
 2 files changed, 65 insertions(+), 4 deletions(-)

diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericJsonRecord.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericJsonRecord.java
index 6f57631..650ca46 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericJsonRecord.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericJsonRecord.java
@@ -55,12 +55,16 @@ class GenericJsonRecord extends VersionedGenericRecord {
             return new GenericJsonRecord(schemaVersion, fields, fn);
         } else if (fn.isBoolean()) {
             return fn.asBoolean();
-        } else if (fn.isInt()) {
-            return fn.asInt();
         } else if (fn.isFloatingPointNumber()) {
             return fn.asDouble();
-        } else if (fn.isDouble()) {
-            return fn.asDouble();
+        } else if (fn.isBigInteger()) {
+            if (fn.canConvertToLong()) {
+                return fn.asLong();
+            } else {
+                return fn.asText();
+            }
+        } else if (fn.isNumber()) {
+            return fn.numberValue();
         } else {
             return fn.asText();
         }
diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/generic/GenericJsonRecordTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/generic/GenericJsonRecordTest.java
new file mode 100644
index 0000000..0de3e3e
--- /dev/null
+++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/generic/GenericJsonRecordTest.java
@@ -0,0 +1,57 @@
+/**
+ * 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.client.impl.schema.generic;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.testng.annotations.Test;
+
+import java.util.Collections;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+
+public class GenericJsonRecordTest {
+
+    @Test
+    public void decodeLongField() throws Exception{
+        String jsonStr = "{\"timestamp\":1585204833128, \"count\":2, \"value\": 1.1, \"on\":true}";
+        byte[] jsonStrBytes = jsonStr.getBytes();
+        ObjectMapper objectMapper = new ObjectMapper();
+        JsonNode jn = objectMapper.readTree(new String(jsonStrBytes, 0, jsonStrBytes.length, UTF_8));
+        GenericJsonRecord record = new GenericJsonRecord(null, Collections.emptyList(), jn);
+
+        Object longValue = record.getField("timestamp");
+        assertTrue(longValue instanceof Long);
+        assertEquals(1585204833128L, longValue);
+
+        Object intValue = record.getField("count");
+        assertTrue(intValue instanceof Integer);
+        assertEquals(2, intValue);
+
+        Object value = record.getField("value");
+        assertTrue(value instanceof Double);
+        assertEquals(1.1, value);
+
+        Object boolValue = record.getField("on");
+        assertTrue((boolean)boolValue);
+    }
+}
\ No newline at end of file


[pulsar] 14/38: Extract common method in tiered storage to managed-ledger module (#6533)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 48b96139007587972c7449ac7512b3d4b2441ad1
Author: Yijie Shen <he...@gmail.com>
AuthorDate: Sun Mar 15 21:18:12 2020 +0800

    Extract common method in tiered storage to managed-ledger module (#6533)
    
    Move common ledger metadata SerDe methods to managed-ledger.
    
    (cherry picked from commit f0880f255fd2bbabb5652b1ba38e65ff430fb6a8)
---
 .../bookkeeper/mledger/offload/OffloadUtils.java   |  92 +++++++++++
 .../impl/FileStoreBackedReadHandleImpl.java        |  60 +------
 .../impl/FileSystemManagedLedgerOffloader.java     |  36 +---
 .../offload/jcloud/impl/OffloadIndexBlockImpl.java | 181 +--------------------
 4 files changed, 100 insertions(+), 269 deletions(-)

diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/OffloadUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/OffloadUtils.java
index c3a0cc6..b1fbbfc 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/OffloadUtils.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/OffloadUtils.java
@@ -19,12 +19,27 @@
 package org.apache.bookkeeper.mledger.offload;
 
 import com.google.common.collect.Maps;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
+
+import com.google.protobuf.ByteString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerMetadataBuilder;
+import org.apache.bookkeeper.client.api.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.mledger.proto.MLDataFormats.KeyValue;
 import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo;
 import org.apache.bookkeeper.mledger.proto.MLDataFormats.OffloadContext;
 import org.apache.bookkeeper.mledger.proto.MLDataFormats.OffloadDriverMetadata;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.proto.DataFormats;
 
+@Slf4j
 public final class OffloadUtils {
 
     private OffloadUtils() {}
@@ -87,4 +102,81 @@ public final class OffloadUtils {
                         .setValue(v)
                         .build()));
     }
+
+    public static byte[] buildLedgerMetadataFormat(LedgerMetadata metadata) {
+        DataFormats.LedgerMetadataFormat.Builder builder = DataFormats.LedgerMetadataFormat.newBuilder();
+        builder.setQuorumSize(metadata.getWriteQuorumSize())
+                .setAckQuorumSize(metadata.getAckQuorumSize())
+                .setEnsembleSize(metadata.getEnsembleSize())
+                .setLength(metadata.getLength())
+                .setState(metadata.isClosed() ? DataFormats.LedgerMetadataFormat.State.CLOSED : DataFormats.LedgerMetadataFormat.State.OPEN)
+                .setLastEntryId(metadata.getLastEntryId())
+                .setCtime(metadata.getCtime())
+                .setDigestType(BookKeeper.DigestType.toProtoDigestType(
+                        BookKeeper.DigestType.fromApiDigestType(metadata.getDigestType())));
+
+        for (Map.Entry<String, byte[]> e : metadata.getCustomMetadata().entrySet()) {
+            builder.addCustomMetadataBuilder()
+                    .setKey(e.getKey()).setValue(ByteString.copyFrom(e.getValue()));
+        }
+
+        for (Map.Entry<Long, ? extends List<BookieSocketAddress>> e : metadata.getAllEnsembles().entrySet()) {
+            builder.addSegmentBuilder()
+                    .setFirstEntryId(e.getKey())
+                    .addAllEnsembleMember(e.getValue().stream().map(a -> a.toString()).collect(Collectors.toList()));
+        }
+
+        return builder.build().toByteArray();
+    }
+
+    public static LedgerMetadata parseLedgerMetadata(byte[] bytes) throws IOException {
+        DataFormats.LedgerMetadataFormat ledgerMetadataFormat = DataFormats.LedgerMetadataFormat.newBuilder().mergeFrom(bytes).build();
+        LedgerMetadataBuilder builder = LedgerMetadataBuilder.create()
+                .withLastEntryId(ledgerMetadataFormat.getLastEntryId())
+                .withPassword(ledgerMetadataFormat.getPassword().toByteArray())
+                .withClosedState()
+                .withMetadataFormatVersion(2)
+                .withLength(ledgerMetadataFormat.getLength())
+                .withAckQuorumSize(ledgerMetadataFormat.getAckQuorumSize())
+                .withCreationTime(ledgerMetadataFormat.getCtime())
+                .withWriteQuorumSize(ledgerMetadataFormat.getQuorumSize())
+                .withEnsembleSize(ledgerMetadataFormat.getEnsembleSize());
+        ledgerMetadataFormat.getSegmentList().forEach(segment -> {
+            ArrayList<BookieSocketAddress> addressArrayList = new ArrayList<>();
+            segment.getEnsembleMemberList().forEach(address -> {
+                try {
+                    addressArrayList.add(new BookieSocketAddress(address));
+                } catch (IOException e) {
+                    log.error("Exception when create BookieSocketAddress. ", e);
+                }
+            });
+            builder.newEnsembleEntry(segment.getFirstEntryId(), addressArrayList);
+        });
+
+        if (ledgerMetadataFormat.getCustomMetadataCount() > 0) {
+            Map<String, byte[]> customMetadata = Maps.newHashMap();
+            ledgerMetadataFormat.getCustomMetadataList().forEach(
+                    entry -> customMetadata.put(entry.getKey(), entry.getValue().toByteArray()));
+            builder.withCustomMetadata(customMetadata);
+        }
+
+        switch (ledgerMetadataFormat.getDigestType()) {
+            case HMAC:
+                builder.withDigestType(DigestType.MAC);
+                break;
+            case CRC32:
+                builder.withDigestType(DigestType.CRC32);
+                break;
+            case CRC32C:
+                builder.withDigestType(DigestType.CRC32C);
+                break;
+            case DUMMY:
+                builder.withDigestType(DigestType.DUMMY);
+                break;
+            default:
+                throw new IllegalArgumentException("Unable to convert digest type " + ledgerMetadataFormat.getDigestType());
+        }
+
+        return builder.build();
+    }
 }
diff --git a/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileStoreBackedReadHandleImpl.java b/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileStoreBackedReadHandleImpl.java
index b1663d2..64399d4 100644
--- a/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileStoreBackedReadHandleImpl.java
+++ b/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileStoreBackedReadHandleImpl.java
@@ -18,12 +18,9 @@
  */
 package org.apache.bookkeeper.mledger.offload.filesystem.impl;
 
-import com.google.common.collect.Maps;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.PooledByteBufAllocator;
 import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerMetadataBuilder;
-import org.apache.bookkeeper.client.api.DigestType;
 import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
 import org.apache.bookkeeper.client.api.LedgerEntries;
 import org.apache.bookkeeper.client.api.LedgerEntry;
@@ -32,8 +29,6 @@ import org.apache.bookkeeper.client.api.ReadHandle;
 import org.apache.bookkeeper.client.impl.LedgerEntriesImpl;
 import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
 
-import org.apache.bookkeeper.net.BookieSocketAddress;
-import org.apache.bookkeeper.proto.DataFormats;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.MapFile;
@@ -43,11 +38,12 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ScheduledExecutorService;
 
+import static org.apache.bookkeeper.mledger.offload.OffloadUtils.parseLedgerMetadata;
+
 public class FileStoreBackedReadHandleImpl implements ReadHandle {
     private static final Logger log = LoggerFactory.getLogger(FileStoreBackedReadHandleImpl.class);
     private final ExecutorService executor;
@@ -184,56 +180,4 @@ public class FileStoreBackedReadHandleImpl implements ReadHandle {
     public static ReadHandle open(ScheduledExecutorService executor, MapFile.Reader reader, long ledgerId) throws IOException {
             return new FileStoreBackedReadHandleImpl(executor, reader, ledgerId);
     }
-
-    private static LedgerMetadata parseLedgerMetadata(byte[] bytes) throws IOException {
-        DataFormats.LedgerMetadataFormat ledgerMetadataFormat = DataFormats.LedgerMetadataFormat.newBuilder().mergeFrom(bytes).build();
-        LedgerMetadataBuilder builder = LedgerMetadataBuilder.create()
-                .withLastEntryId(ledgerMetadataFormat.getLastEntryId())
-                .withPassword(ledgerMetadataFormat.getPassword().toByteArray())
-                .withClosedState()
-                .withMetadataFormatVersion(2)
-                .withLength(ledgerMetadataFormat.getLength())
-                .withAckQuorumSize(ledgerMetadataFormat.getAckQuorumSize())
-                .withCreationTime(ledgerMetadataFormat.getCtime())
-                .withWriteQuorumSize(ledgerMetadataFormat.getQuorumSize())
-                .withEnsembleSize(ledgerMetadataFormat.getEnsembleSize());
-        ledgerMetadataFormat.getSegmentList().forEach(segment -> {
-            ArrayList<BookieSocketAddress> addressArrayList = new ArrayList<>();
-            segment.getEnsembleMemberList().forEach(address -> {
-                try {
-                    addressArrayList.add(new BookieSocketAddress(address));
-                } catch (IOException e) {
-                    log.error("Exception when create BookieSocketAddress. ", e);
-                }
-            });
-            builder.newEnsembleEntry(segment.getFirstEntryId(), addressArrayList);
-        });
-
-        if (ledgerMetadataFormat.getCustomMetadataCount() > 0) {
-            Map<String, byte[]> customMetadata = Maps.newHashMap();
-            ledgerMetadataFormat.getCustomMetadataList().forEach(
-                    entry -> customMetadata.put(entry.getKey(), entry.getValue().toByteArray()));
-            builder.withCustomMetadata(customMetadata);
-        }
-
-        switch (ledgerMetadataFormat.getDigestType()) {
-            case HMAC:
-                builder.withDigestType(DigestType.MAC);
-                break;
-            case CRC32:
-                builder.withDigestType(DigestType.CRC32);
-                break;
-            case CRC32C:
-                builder.withDigestType(DigestType.CRC32C);
-                break;
-            case DUMMY:
-                builder.withDigestType(DigestType.DUMMY);
-                break;
-            default:
-                throw new IllegalArgumentException("Unable to convert digest type " + ledgerMetadataFormat.getDigestType());
-        }
-
-        return builder.build();
-    }
-
 }
diff --git a/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemManagedLedgerOffloader.java b/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemManagedLedgerOffloader.java
index eda747d..bbee828 100644
--- a/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemManagedLedgerOffloader.java
+++ b/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemManagedLedgerOffloader.java
@@ -20,17 +20,12 @@ package org.apache.bookkeeper.mledger.offload.filesystem.impl;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
-import com.google.protobuf.ByteString;
-import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.api.LedgerEntries;
 import org.apache.bookkeeper.client.api.LedgerEntry;
-import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.ReadHandle;
 import org.apache.bookkeeper.common.util.OrderedScheduler;
 import org.apache.bookkeeper.mledger.LedgerOffloader;
 import org.apache.bookkeeper.mledger.offload.filesystem.FileSystemLedgerOffloaderFactory;
-import org.apache.bookkeeper.net.BookieSocketAddress;
-import org.apache.bookkeeper.proto.DataFormats;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -44,7 +39,6 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 
@@ -52,8 +46,8 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.stream.Collectors;
 
+import static org.apache.bookkeeper.mledger.offload.OffloadUtils.buildLedgerMetadataFormat;
 
 public class FileSystemManagedLedgerOffloader implements LedgerOffloader {
 
@@ -189,7 +183,7 @@ public class FileSystemManagedLedgerOffloader implements LedgerOffloader {
                 //store the ledgerMetadata in -1 index
                 key.set(METADATA_KEY_INDEX);
                 byte[] ledgerMetadata = buildLedgerMetadataFormat(readHandle.getLedgerMetadata());
-                value.set(buildLedgerMetadataFormat(readHandle.getLedgerMetadata()), 0, ledgerMetadata.length);
+                value.set(ledgerMetadata, 0, ledgerMetadata.length);
                 dataWriter.append(key, value);
                 AtomicLong haveOffloadEntryNumber = new AtomicLong(0);
                 long needToOffloadFirstEntryNumber = 0;
@@ -307,32 +301,6 @@ public class FileSystemManagedLedgerOffloader implements LedgerOffloader {
         return promise;
     }
 
-    private static byte[] buildLedgerMetadataFormat(LedgerMetadata metadata) {
-        DataFormats.LedgerMetadataFormat.Builder builder = DataFormats.LedgerMetadataFormat.newBuilder();
-        builder.setQuorumSize(metadata.getWriteQuorumSize())
-                .setAckQuorumSize(metadata.getAckQuorumSize())
-                .setEnsembleSize(metadata.getEnsembleSize())
-                .setLength(metadata.getLength())
-                .setState(metadata.isClosed() ? DataFormats.LedgerMetadataFormat.State.CLOSED : DataFormats.LedgerMetadataFormat.State.OPEN)
-                .setLastEntryId(metadata.getLastEntryId())
-                .setCtime(metadata.getCtime())
-                .setDigestType(BookKeeper.DigestType.toProtoDigestType(
-                        BookKeeper.DigestType.fromApiDigestType(metadata.getDigestType())));
-
-        for (Map.Entry<String, byte[]> e : metadata.getCustomMetadata().entrySet()) {
-            builder.addCustomMetadataBuilder()
-                    .setKey(e.getKey()).setValue(ByteString.copyFrom(e.getValue()));
-        }
-
-        for (Map.Entry<Long, ? extends List<BookieSocketAddress>> e : metadata.getAllEnsembles().entrySet()) {
-            builder.addSegmentBuilder()
-                    .setFirstEntryId(e.getKey())
-                    .addAllEnsembleMember(e.getValue().stream().map(a -> a.toString()).collect(Collectors.toList()));
-        }
-
-        return builder.build().toByteArray();
-    }
-
     @Override
     public OffloadPolicies getOffloadPolicies() {
         return offloadPolicies;
diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockImpl.java
index 16492bc..42a5d8d 100644
--- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockImpl.java
+++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockImpl.java
@@ -18,10 +18,7 @@
  */
 package org.apache.bookkeeper.mledger.offload.jcloud.impl;
 
-import static com.google.common.base.Preconditions.checkState;
-
 import com.google.common.collect.Maps;
-import com.google.protobuf.ByteString;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufInputStream;
@@ -31,25 +28,20 @@ import io.netty.util.Recycler.Handle;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
 import java.util.TreeMap;
-import java.util.stream.Collectors;
 
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.api.DigestType;
 import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlock;
 import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexEntry;
-import org.apache.bookkeeper.net.BookieSocketAddress;
-import org.apache.bookkeeper.proto.DataFormats;
-import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat;
 import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static com.google.common.base.Preconditions.checkState;
+import static org.apache.bookkeeper.mledger.offload.OffloadUtils.buildLedgerMetadataFormat;
+import static org.apache.bookkeeper.mledger.offload.OffloadUtils.parseLedgerMetadata;
+
 public class OffloadIndexBlockImpl implements OffloadIndexBlock {
     private static final Logger log = LoggerFactory.getLogger(OffloadIndexBlockImpl.class);
 
@@ -136,32 +128,6 @@ public class OffloadIndexBlockImpl implements OffloadIndexBlock {
         return this.dataHeaderLength;
     }
 
-    private static byte[] buildLedgerMetadataFormat(LedgerMetadata metadata) {
-        LedgerMetadataFormat.Builder builder = LedgerMetadataFormat.newBuilder();
-        builder.setQuorumSize(metadata.getWriteQuorumSize())
-            .setAckQuorumSize(metadata.getAckQuorumSize())
-            .setEnsembleSize(metadata.getEnsembleSize())
-            .setLength(metadata.getLength())
-            .setState(metadata.isClosed() ? LedgerMetadataFormat.State.CLOSED : LedgerMetadataFormat.State.OPEN)
-            .setLastEntryId(metadata.getLastEntryId())
-            .setCtime(metadata.getCtime())
-            .setDigestType(BookKeeper.DigestType.toProtoDigestType(
-                BookKeeper.DigestType.fromApiDigestType(metadata.getDigestType())));
-
-        for (Map.Entry<String, byte[]> e : metadata.getCustomMetadata().entrySet()) {
-            builder.addCustomMetadataBuilder()
-                .setKey(e.getKey()).setValue(ByteString.copyFrom(e.getValue()));
-        }
-
-        for (Map.Entry<Long, ? extends List<BookieSocketAddress>> e : metadata.getAllEnsembles().entrySet()) {
-            builder.addSegmentBuilder()
-                .setFirstEntryId(e.getKey())
-                .addAllEnsembleMember(e.getValue().stream().map(a -> a.toString()).collect(Collectors.toList()));
-        }
-
-        return builder.build().toByteArray();
-    }
-
     /**
      * Get the content of the index block as InputStream.
      * Read out in format:
@@ -203,145 +169,6 @@ public class OffloadIndexBlockImpl implements OffloadIndexBlock {
         return new OffloadIndexBlock.IndexInputStream(new ByteBufInputStream(out, true), indexBlockLength);
     }
 
-    static private class InternalLedgerMetadata implements LedgerMetadata {
-        private LedgerMetadataFormat ledgerMetadataFormat;
-
-        private int ensembleSize;
-        private int writeQuorumSize;
-        private int ackQuorumSize;
-        private long lastEntryId;
-        private long length;
-        private DataFormats.LedgerMetadataFormat.DigestType digestType;
-        private long ctime;
-        private byte[] password;
-        private State state;
-        private Map<String, byte[]> customMetadata = Maps.newHashMap();
-        private TreeMap<Long, ArrayList<BookieSocketAddress>> ensembles = new TreeMap<Long, ArrayList<BookieSocketAddress>>();
-
-        InternalLedgerMetadata(LedgerMetadataFormat ledgerMetadataFormat) {
-            this.ensembleSize = ledgerMetadataFormat.getEnsembleSize();
-            this.writeQuorumSize = ledgerMetadataFormat.getQuorumSize();
-            this.ackQuorumSize = ledgerMetadataFormat.getAckQuorumSize();
-            this.lastEntryId = ledgerMetadataFormat.getLastEntryId();
-            this.length = ledgerMetadataFormat.getLength();
-            this.digestType = ledgerMetadataFormat.getDigestType();
-            this.ctime = ledgerMetadataFormat.getCtime();
-            this.state = State.CLOSED;
-            this.password = ledgerMetadataFormat.getPassword().toByteArray();
-
-            if (ledgerMetadataFormat.getCustomMetadataCount() > 0) {
-                ledgerMetadataFormat.getCustomMetadataList().forEach(
-                    entry -> this.customMetadata.put(entry.getKey(), entry.getValue().toByteArray()));
-            }
-
-            ledgerMetadataFormat.getSegmentList().forEach(segment -> {
-                ArrayList<BookieSocketAddress> addressArrayList = new ArrayList<BookieSocketAddress>();
-                segment.getEnsembleMemberList().forEach(address -> {
-                    try {
-                        addressArrayList.add(new BookieSocketAddress(address));
-                    } catch (IOException e) {
-                        log.error("Exception when create BookieSocketAddress. ", e);
-                    }
-                });
-                this.ensembles.put(segment.getFirstEntryId(), addressArrayList);
-            });
-        }
-
-        @Override
-        public boolean hasPassword() { return true; }
-
-        @Override
-        public byte[] getPassword() { return password; }
-
-        @Override
-        public State getState() { return state; }
-
-        @Override
-        public int getMetadataFormatVersion() { return 2; }
-
-        @Override
-        public long getCToken() {
-            return 0;
-        }
-
-        @Override
-        public int getEnsembleSize() {
-            return this.ensembleSize;
-        }
-
-        @Override
-        public int getWriteQuorumSize() {
-            return this.writeQuorumSize;
-        }
-
-        @Override
-        public int getAckQuorumSize() {
-            return this.ackQuorumSize;
-        }
-
-        @Override
-        public long getLastEntryId() {
-            return this.lastEntryId;
-        }
-
-        @Override
-        public long getLength() {
-            return this.length;
-        }
-
-        @Override
-        public DigestType getDigestType() {
-            switch (this.digestType) {
-                case HMAC:
-                    return DigestType.MAC;
-                case CRC32:
-                    return DigestType.CRC32;
-                case CRC32C:
-                    return DigestType.CRC32C;
-                case DUMMY:
-                    return DigestType.DUMMY;
-                default:
-                    throw new IllegalArgumentException("Unable to convert digest type " + digestType);
-            }
-        }
-
-        @Override
-        public long getCtime() {
-            return this.ctime;
-        }
-
-        @Override
-        public boolean isClosed() {
-            return this.state == State.CLOSED;
-        }
-
-        @Override
-        public Map<String, byte[]> getCustomMetadata() {
-            return this.customMetadata;
-        }
-
-        @Override
-        public List<BookieSocketAddress> getEnsembleAt(long entryId) {
-            return ensembles.get(ensembles.headMap(entryId + 1).lastKey());
-        }
-
-        @Override
-        public NavigableMap<Long, ? extends List<BookieSocketAddress>> getAllEnsembles() {
-            return this.ensembles;
-        }
-
-        @Override
-        public String toSafeString() {
-            return toString();
-        }
-    }
-
-    private static LedgerMetadata parseLedgerMetadata(byte[] bytes) throws IOException {
-        LedgerMetadataFormat.Builder builder = LedgerMetadataFormat.newBuilder();
-        builder.mergeFrom(bytes);
-        return new InternalLedgerMetadata(builder.build());
-    }
-
     private OffloadIndexBlock fromStream(InputStream stream) throws IOException {
         DataInputStream dis = new DataInputStream(stream);
         int magic = dis.readInt();


[pulsar] 35/38: Add note for bookkeeper explicit LAC (#6908)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 026a610db59e3c69863f83c46731dc473806bd6a
Author: lipenghui <pe...@apache.org>
AuthorDate: Fri May 8 00:22:28 2020 +0800

    Add note for bookkeeper explicit LAC (#6908)
    
    (cherry picked from commit b9e96098062b5968a7de24e44644688dff66d474)
---
 site2/docs/sql-deployment-configurations.md | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git a/site2/docs/sql-deployment-configurations.md b/site2/docs/sql-deployment-configurations.md
index 65653f0..b986a54 100644
--- a/site2/docs/sql-deployment-configurations.md
+++ b/site2/docs/sql-deployment-configurations.md
@@ -152,4 +152,7 @@ presto> SELECT * FROM system.runtime.nodes;
  2       | http://192.168.2.3:8081 | testversion  | false       | active 
 ```
 
-For more information about deployment in Presto, refer to [Presto deployment](https://prestodb.io/docs/current/installation/deployment.html).
\ No newline at end of file
+For more information about deployment in Presto, refer to [Presto deployment](https://prestodb.io/docs/current/installation/deployment.html).
+
+> Note  
+> The broker does not advance LAC, so when Pulsar SQL bypass broker to query data, it can only read entries up to the LAC that all the bookies learned. You can enable periodically write LAC on the broker by setting "bookkeeperExplicitLacIntervalInMills" in the broker.conf.
\ No newline at end of file


[pulsar] 04/38: Fix typo in io-debezium-source.md (#6729)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 7e951e31cebcfa737bf186d6046202d42fe26090
Author: ran <ga...@126.com>
AuthorDate: Mon Apr 13 17:11:36 2020 +0800

    Fix typo in io-debezium-source.md (#6729)
    
    Fix typo in io-debezium-source.md
    (cherry picked from commit 81f3bc823d81144acbeeaa229e93e091abd9370f)
---
 site2/docs/io-debezium-source.md | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/site2/docs/io-debezium-source.md b/site2/docs/io-debezium-source.md
index 7bdb1e9..bdb3e29 100644
--- a/site2/docs/io-debezium-source.md
+++ b/site2/docs/io-debezium-source.md
@@ -1,6 +1,6 @@
 ---
 id: io-debezium-source
-title: Debezium source onnector
+title: Debezium source connector
 sidebar_label: Debezium source connector
 ---
 
@@ -346,4 +346,4 @@ If you encounter the above problems in synchronizing data, please refer to [this
 
 ```$xslt
 max.queue.size=
-```
\ No newline at end of file
+```


[pulsar] 25/38: Avoid creating partitioned topic for partition name (#6846)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit a2d5d5b59cfb84843eed15d806e0441b0d64a442
Author: lipenghui <pe...@apache.org>
AuthorDate: Thu Apr 30 10:13:49 2020 +0800

    Avoid creating partitioned topic for partition name (#6846)
    
    Fixes #6840
    
    Motivation
    Avoid creating partitioned topic for partition name
    
    Verifying this change
    New unit test added.
    (cherry picked from commit bb41702501e90a0789a39a1cb921b6b82eb55d07)
---
 .../pulsar/broker/service/BrokerService.java       |  1 +
 .../pulsar/broker/admin/TopicAutoCreationTest.java | 88 ++++++++++++++++++++++
 2 files changed, 89 insertions(+)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
index ffd75b6..808f0c3 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
@@ -1863,6 +1863,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
                                 // If topic is already exist, creating partitioned topic is not allowed.
                                 if (metadata.partitions == 0
                                         && !topicExists
+                                        && !topicName.isPartitioned()
                                         && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
                                         && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
                                     return pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java
new file mode 100644
index 0000000..2067526
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java
@@ -0,0 +1,88 @@
+/**
+ * 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 org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.junit.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.List;
+import java.util.UUID;
+
+public class TopicAutoCreationTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setAllowAutoTopicCreationType("partitioned");
+        conf.setAllowAutoTopicCreation(true);
+        conf.setDefaultNumPartitions(3);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testPartitionedTopicAutoCreation() throws PulsarAdminException, PulsarClientException {
+        final String namespaceName = "my-property/my-ns";
+        final String topic = "persistent://" + namespaceName + "/test-partitioned-topi-auto-creation-"
+                + UUID.randomUUID().toString();
+
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .create();
+
+        List<String> partitionedTopics = admin.topics().getPartitionedTopicList(namespaceName);
+        List<String> topics = admin.topics().getList(namespaceName);
+        Assert.assertEquals(partitionedTopics.size(), 1);
+        Assert.assertEquals(topics.size(), 3);
+
+        producer.close();
+        for (String t : topics) {
+            admin.topics().delete(t);
+        }
+
+        admin.topics().deletePartitionedTopic(topic);
+
+
+        final String partition = "persistent://" + namespaceName + "/test-partitioned-topi-auto-creation-partition-0";
+
+        producer = pulsarClient.newProducer()
+                .topic(partition)
+                .create();
+
+        partitionedTopics = admin.topics().getPartitionedTopicList(namespaceName);
+        topics = admin.topics().getList(namespaceName);
+        Assert.assertEquals(partitionedTopics.size(), 0);
+        Assert.assertEquals(topics.size(), 1);
+
+        producer.close();
+    }
+}


[pulsar] 19/38: Fix check backlogged cursors without consumer (#6766)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 5b74f81555a5ff12c4889d0de023b538cebdc136
Author: zhaorongsheng <zh...@users.noreply.github.com>
AuthorDate: Thu Apr 23 02:42:22 2020 +0800

    Fix check backlogged cursors without consumer (#6766)
    
    * fix the backlogged cursors bug
    
    * add some test
    
    * change test case
    
    * change test case
    
    * change test case
    
    * change check logical
    
    * move backlogged cursor check after consumer add
    
    Co-authored-by: zhaorongsheng <zh...@bigo.sg>(cherry picked from commit 4d59f2b8a002c48676ca67bbb137ca0078f7fa8d)
---
 .../apache/bookkeeper/mledger/ManagedLedger.java   |   6 -
 .../mledger/ManagedLedgerFactoryConfig.java        |   5 -
 .../bookkeeper/mledger/impl/ManagedLedgerImpl.java |  15 ---
 .../bookkeeper/mledger/impl/ManagedLedgerTest.java |  80 ------------
 .../pulsar/broker/ManagedLedgerClientFactory.java  |   1 -
 .../apache/pulsar/broker/service/PulsarStats.java  |   2 +-
 .../org/apache/pulsar/broker/service/Topic.java    |   6 +
 .../service/nonpersistent/NonPersistentTopic.java  |   5 +
 .../broker/service/persistent/PersistentTopic.java |  20 ++-
 .../pulsar/broker/service/PersistentTopicTest.java | 142 ++++++++++++++++++++-
 .../client/api/SimpleProducerConsumerTest.java     |   4 +-
 11 files changed, 172 insertions(+), 114 deletions(-)

diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java
index 50439f5..8f52905 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java
@@ -328,12 +328,6 @@ public interface ManagedLedger {
      */
     long getOffloadedSize();
 
-    /**
-     * Activate cursors those caught up backlog-threshold entries and deactivate slow cursors which are creating
-     * backlog.
-     */
-    void checkBackloggedCursors();
-
     void asyncTerminate(TerminateCallback callback, Object ctx);
 
     /**
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java
index b5d0a7c..4d1eb31 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java
@@ -48,11 +48,6 @@ public class ManagedLedgerFactoryConfig {
     private long cacheEvictionTimeThresholdMillis = 1000;
 
     /**
-     * Threshould to consider a cursor as "backlogged"
-     */
-    private long thresholdBackloggedCursor = 1000;
-
-    /**
      * Whether we should make a copy of the entry payloads when inserting in cache
      */
     private boolean copyEntriesInCache = false;
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
index bf7ad2b..12ca6cd 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
@@ -234,8 +234,6 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
             .newUpdater(ManagedLedgerImpl.class, "addOpCount");
     private volatile long addOpCount = 0;
 
-    private final long backloggedCursorThresholdEntries;
-
     // last read-operation's callback to check read-timeout on it.
     private volatile ReadEntryCallbackWrapper lastReadCallback = null;
     private static final AtomicReferenceFieldUpdater<ManagedLedgerImpl, ReadEntryCallbackWrapper> LAST_READ_CALLBACK_UPDATER = AtomicReferenceFieldUpdater
@@ -271,7 +269,6 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
         this.waitingCursors = Queues.newConcurrentLinkedQueue();
         this.uninitializedCursors = Maps.newHashMap();
         this.clock = config.getClock();
-        this.backloggedCursorThresholdEntries = factory.getConfig().getThresholdBackloggedCursor();
 
         // Get the next rollover time. Add a random value upto 5% to avoid rollover multiple ledgers at the same time
         this.maximumRolloverTimeMs = (long) (config.getMaximumRolloverTimeMs() * (1 + random.nextDouble() * 5 / 100.0));
@@ -908,18 +905,6 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
     }
 
     @Override
-    public void checkBackloggedCursors() {
-        // activate caught up cursors
-        cursors.forEach(cursor -> {
-            if (cursor.getNumberOfEntries() < backloggedCursorThresholdEntries) {
-                cursor.setActive();
-            } else {
-                cursor.setInactive();
-            }
-        });
-    }
-
-    @Override
     public long getEstimatedBacklogSize() {
 
         PositionImpl pos = getMarkDeletePositionOfSlowestConsumer();
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 c891a89..c2886cf 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
@@ -2070,71 +2070,6 @@ public class ManagedLedgerTest extends MockedBookKeeperTestCase {
     }
 
     @Test
-    public void testBacklogCursor() throws Exception {
-        int backloggedThreshold = 10;
-        ManagedLedgerFactoryConfig factoryConf = new ManagedLedgerFactoryConfig();
-        factoryConf.setThresholdBackloggedCursor(backloggedThreshold);
-        ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(bkc, zkc, factoryConf);
-        ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("cache_backlog_ledger");
-
-        // Open Cursor also adds cursor into activeCursor-container
-        ManagedCursor cursor1 = ledger.openCursor("c1");
-        ManagedCursor cursor2 = ledger.openCursor("c2");
-
-        CountDownLatch latch = new CountDownLatch(backloggedThreshold);
-        for (int i = 0; i < backloggedThreshold + 1; i++) {
-            String content = "entry"; // 5 bytes
-            ByteBuf entry = getMessageWithMetadata(content.getBytes());
-            ledger.asyncAddEntry(entry, new AddEntryCallback() {
-                @Override
-                public void addComplete(Position position, Object ctx) {
-                    latch.countDown();
-                    entry.release();
-                }
-
-                @Override
-                public void addFailed(ManagedLedgerException exception, Object ctx) {
-                    latch.countDown();
-                    entry.release();
-                }
-
-            }, null);
-        }
-        latch.await();
-
-        // Verify: cursors are active as :haven't started deactivateBacklogCursor scan
-        assertTrue(cursor1.isActive());
-        assertTrue(cursor2.isActive());
-
-        // deactivate backlog cursors
-        ledger.checkBackloggedCursors();
-
-        // both cursors have to be inactive
-        assertFalse(cursor1.isActive());
-        assertFalse(cursor2.isActive());
-
-        // read entries so, cursor1 reaches maxBacklog threshold again to be active again
-        List<Entry> entries1 = cursor1.readEntries(50);
-        for (Entry entry : entries1) {
-            log.info("Read entry. Position={} Content='{}'", entry.getPosition(), new String(entry.getData()));
-            entry.release();
-        }
-
-        // activate cursors which caught up maxbacklog threshold
-        ledger.checkBackloggedCursors();
-
-        // verify: cursor1 has consumed messages so, under maxBacklog threshold => active
-        assertTrue(cursor1.isActive());
-
-        // verify: cursor2 has not consumed messages so, above maxBacklog threshold => inactive
-        assertFalse(cursor2.isActive());
-
-        ledger.close();
-
-        factory.shutdown();
-    }
-
-    @Test
     public void testConcurrentOpenCursor() throws Exception {
         ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testConcurrentOpenCursor");
 
@@ -2271,21 +2206,6 @@ public class ManagedLedgerTest extends MockedBookKeeperTestCase {
         }
     }
 
-    public ByteBuf getMessageWithMetadata(byte[] data) throws IOException {
-        MessageMetadata messageData = MessageMetadata.newBuilder().setPublishTime(System.currentTimeMillis())
-                .setProducerName("prod-name").setSequenceId(0).build();
-        ByteBuf payload = Unpooled.wrappedBuffer(data, 0, data.length);
-
-        int msgMetadataSize = messageData.getSerializedSize();
-        int headersSize = 4 + msgMetadataSize;
-        ByteBuf headers = PulsarByteBufAllocator.DEFAULT.buffer(headersSize, headersSize);
-        ByteBufCodedOutputStream outStream = ByteBufCodedOutputStream.get(headers);
-        headers.writeInt(msgMetadataSize);
-        messageData.writeTo(outStream);
-        outStream.recycle();
-        return ByteBufPair.coalesce(ByteBufPair.get(headers, payload));
-    }
-
     @Test
     public void testConsumerSubscriptionInitializePosition() throws Exception{
         final int MAX_ENTRY_PER_LEDGER = 2;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java
index bb286ad..2f640e6 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java
@@ -55,7 +55,6 @@ public class ManagedLedgerClientFactory implements Closeable {
         managedLedgerFactoryConfig.setNumManagedLedgerSchedulerThreads(conf.getManagedLedgerNumSchedulerThreads());
         managedLedgerFactoryConfig.setCacheEvictionFrequency(conf.getManagedLedgerCacheEvictionFrequency());
         managedLedgerFactoryConfig.setCacheEvictionTimeThresholdMillis(conf.getManagedLedgerCacheEvictionTimeThresholdMillis());
-        managedLedgerFactoryConfig.setThresholdBackloggedCursor(conf.getManagedLedgerCursorBackloggedThreshold());
         managedLedgerFactoryConfig.setCopyEntriesInCache(conf.isManagedLedgerCacheCopyEntries());
 
         this.defaultBkClient = bookkeeperProvider.create(conf, zkClient, Optional.empty(), null);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java
index 678abf3..66c5b04 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java
@@ -138,7 +138,7 @@ public class PulsarStats implements Closeable {
                                 }
                                 // this task: helps to activate inactive-backlog-cursors which have caught up and
                                 // connected, also deactivate active-backlog-cursors which has backlog
-                                ((PersistentTopic) topic).getManagedLedger().checkBackloggedCursors();
+                                ((PersistentTopic) topic).checkBackloggedCursors();
                             }else if (topic instanceof NonPersistentTopic) {
                                 tempNonPersistentTopics.add((NonPersistentTopic) topic);
                             } else {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java
index 890e02a..aa147d4 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java
@@ -129,6 +129,12 @@ public interface Topic {
 
     void checkInactiveSubscriptions();
 
+    /**
+     * Activate cursors those caught up backlog-threshold entries and deactivate slow cursors which are creating
+     * backlog.
+     */
+    void checkBackloggedCursors();
+
     void checkMessageExpiry();
 
     void checkMessageDeduplicationInfo();
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java
index 2792234..332abfb 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java
@@ -865,6 +865,11 @@ public class NonPersistentTopic extends AbstractTopic implements Topic {
     }
 
     @Override
+    public void checkBackloggedCursors() {
+        // no-op
+    }
+
+    @Override
     public CompletableFuture<Void> onPoliciesUpdate(Policies data) {
         if (log.isDebugEnabled()) {
             log.debug("[{}] isEncryptionRequired changes: {} -> {}", topic, isEncryptionRequired,
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
index 18d371b..b7470fa 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
@@ -157,6 +157,7 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal
     private Optional<DispatchRateLimiter> dispatchRateLimiter = Optional.empty();
     private Optional<SubscribeRateLimiter> subscribeRateLimiter = Optional.empty();
     public volatile long delayedDeliveryTickTimeMillis = 1000;
+    private final long backloggedCursorThresholdEntries;
     public volatile boolean delayedDeliveryEnabled = false;
     public static final int MESSAGE_RATE_BACKOFF_MS = 1000;
 
@@ -215,6 +216,7 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal
         USAGE_COUNT_UPDATER.set(this, 0);
         this.delayedDeliveryEnabled = brokerService.pulsar().getConfiguration().isDelayedDeliveryEnabled();
         this.delayedDeliveryTickTimeMillis = brokerService.pulsar().getConfiguration().getDelayedDeliveryTickTimeMillis();
+        this.backloggedCursorThresholdEntries = brokerService.pulsar().getConfiguration().getManagedLedgerCursorBackloggedThreshold();
 
         initializeDispatchRateLimiterIfNeeded(Optional.empty());
 
@@ -270,6 +272,7 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal
         this.subscriptions = new ConcurrentOpenHashMap<>(16, 1);
         this.replicators = new ConcurrentOpenHashMap<>(16, 1);
         this.compactedTopic = new CompactedTopicImpl(brokerService.pulsar().getBookKeeperClient());
+        this.backloggedCursorThresholdEntries = brokerService.pulsar().getConfiguration().getManagedLedgerCursorBackloggedThreshold();
     }
 
     private void initializeDispatchRateLimiterIfNeeded(Optional<Policies> policies) {
@@ -589,12 +592,12 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal
 
         subscriptionFuture.thenAccept(subscription -> {
             try {
-                ledger.checkBackloggedCursors();
-
                 Consumer consumer = new Consumer(subscription, subType, topic, consumerId, priorityLevel, consumerName,
                                                  maxUnackedMessages, cnx, cnx.getRole(), metadata, readCompacted, initialPosition, keySharedMeta);
                 subscription.addConsumer(consumer);
 
+                checkBackloggedCursors();
+
                 if (!cnx.isActive()) {
                     consumer.close();
                     if (log.isDebugEnabled()) {
@@ -1692,6 +1695,19 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal
         });
     }
 
+    @Override
+    public void checkBackloggedCursors() {
+        // activate caught up cursors which include consumers
+        subscriptions.forEach((subName, subscription) -> {
+            if (!subscription.getConsumers().isEmpty()
+                && subscription.getCursor().getNumberOfEntries() < backloggedCursorThresholdEntries) {
+                subscription.getCursor().setActive();
+            } else {
+                subscription.getCursor().setInactive();
+            }
+        });
+    }
+
     /**
      * Check whether the topic should be retained (based on time), even tough there are no producers/consumers and it's
      * marked as inactive.
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java
index bc92ee4..87a8d98 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java
@@ -21,7 +21,7 @@ package org.apache.pulsar.broker.service;
 import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.createMockBookKeeper;
 import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.createMockZooKeeper;
 import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
-import static org.mockito.Mockito.any;
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.anyString;
 import static org.mockito.Mockito.eq;
 import static org.mockito.Mockito.matches;
@@ -40,12 +40,14 @@ import static org.testng.Assert.assertNull;
 import static org.testng.Assert.assertTrue;
 import static org.testng.Assert.fail;
 
+import java.io.IOException;
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -68,13 +70,17 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback;
 import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenCursorCallback;
 import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenLedgerCallback;
 import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.Entry;
 import org.apache.bookkeeper.mledger.ManagedCursor;
 import org.apache.bookkeeper.mledger.ManagedLedger;
 import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
 import org.apache.bookkeeper.mledger.ManagedLedgerException;
 import org.apache.bookkeeper.mledger.ManagedLedgerFactory;
+import org.apache.bookkeeper.mledger.Position;
 import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
 import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.test.MockedBookKeeperTestCase;
 import org.apache.pulsar.broker.NoOpShutdownService;
 import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.ServiceConfiguration;
@@ -93,6 +99,7 @@ import org.apache.pulsar.client.api.PulsarClient;
 import org.apache.pulsar.client.api.Schema;
 import org.apache.pulsar.client.impl.PulsarClientImpl;
 import org.apache.pulsar.client.impl.conf.ProducerConfigurationData;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition;
@@ -100,8 +107,11 @@ import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
 import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
 import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.Policies;
+import org.apache.pulsar.common.protocol.ByteBufPair;
 import org.apache.pulsar.common.protocol.schema.SchemaVersion;
+import org.apache.pulsar.common.util.Codec;
 import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap;
+import org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream;
 import org.apache.pulsar.compaction.CompactedTopic;
 import org.apache.pulsar.compaction.Compactor;
 import org.apache.pulsar.zookeeper.ZooKeeperCache;
@@ -126,7 +136,7 @@ import io.netty.buffer.Unpooled;
 
 /**
  */
-public class PersistentTopicTest {
+public class PersistentTopicTest extends MockedBookKeeperTestCase {
     private PulsarService pulsar;
     private BrokerService brokerService;
     private ManagedLedgerFactory mlFactoryMock;
@@ -1505,4 +1515,132 @@ public class PersistentTopicTest {
         topic.checkCompaction();
         verify(compactor, times(0)).compact(anyString());
     }
+
+    @Test
+    public void testBacklogCursor() throws Exception {
+        int backloggedThreshold = 10;
+        pulsar.getConfiguration().setManagedLedgerCursorBackloggedThreshold(backloggedThreshold);
+
+        ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("cache_backlog_ledger");
+        PersistentTopic topic = new PersistentTopic(successTopicName, ledger, brokerService);
+
+        // STEP1: prepare cursors
+        // Open cursor1, add it into activeCursor-container and add it into subscription consumer list
+        ManagedCursor cursor1 = ledger.openCursor("c1");
+        PersistentSubscription sub1 = new PersistentSubscription(topic, "sub-1", cursor1, false);
+        Consumer consumer1 = new Consumer(sub1, SubType.Exclusive, topic.getName(), 1 /* consumer id */, 0, "Cons1"/* consumer name */,
+            50000, serverCnx, "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null);
+        topic.getSubscriptions().put(Codec.decode(cursor1.getName()), sub1);
+        sub1.addConsumer(consumer1);
+        // Open cursor2, add it into activeCursor-container and add it into subscription consumer list
+        ManagedCursor cursor2 = ledger.openCursor("c2");
+        PersistentSubscription sub2 = new PersistentSubscription(topic, "sub-2", cursor2, false);
+        Consumer consumer2 = new Consumer(sub2, SubType.Exclusive, topic.getName(), 2 /* consumer id */, 0, "Cons2"/* consumer name */,
+            50000, serverCnx, "myrole-2", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null);
+        topic.getSubscriptions().put(Codec.decode(cursor2.getName()), sub2);
+        sub2.addConsumer(consumer2);
+        // Open cursor3, add it into activeCursor-container and do not add it into subscription consumer list
+        ManagedCursor cursor3 = ledger.openCursor("c3");
+        PersistentSubscription sub3 = new PersistentSubscription(topic, "sub-3", cursor3, false);
+        Consumer consumer3 = new Consumer(sub2, SubType.Exclusive, topic.getName(), 3 /* consumer id */, 0, "Cons2"/* consumer name */,
+            50000, serverCnx, "myrole-3", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null);
+        topic.getSubscriptions().put(Codec.decode(cursor3.getName()), sub3);
+
+        // Case1: cursors are active as haven't started deactivateBacklogCursor scan
+        assertTrue(cursor1.isActive());
+        assertTrue(cursor2.isActive());
+        assertTrue(cursor3.isActive());
+
+        // deactivate cursor which consumer list is empty
+        topic.checkBackloggedCursors();
+
+        // Case2: cursor3 change to be inactive as it does not include consumer
+        assertTrue(cursor1.isActive());
+        assertTrue(cursor2.isActive());
+        assertFalse(cursor3.isActive());
+
+        // Write messages to ledger
+        CountDownLatch latch = new CountDownLatch(backloggedThreshold);
+        for (int i = 0; i < backloggedThreshold + 1; i++) {
+            String content = "entry"; // 5 bytes
+            ByteBuf entry = getMessageWithMetadata(content.getBytes());
+            ledger.asyncAddEntry(entry, new AddEntryCallback() {
+                @Override
+                public void addComplete(Position position, Object ctx) {
+                    latch.countDown();
+                    entry.release();
+                }
+
+                @Override
+                public void addFailed(ManagedLedgerException exception, Object ctx) {
+                    latch.countDown();
+                    entry.release();
+                }
+
+            }, null);
+        }
+        latch.await();
+
+        assertTrue(cursor1.isActive());
+        assertTrue(cursor2.isActive());
+        assertFalse(cursor3.isActive());
+
+        // deactivate backlog cursors
+        topic.checkBackloggedCursors();
+
+        // Case3: cursor1 and cursor2 change to be inactive because of the backlog
+        assertFalse(cursor1.isActive());
+        assertFalse(cursor2.isActive());
+        assertFalse(cursor3.isActive());
+
+        // read entries so, cursor1 reaches maxBacklog threshold again to be active again
+        List<Entry> entries1 = cursor1.readEntries(50);
+        for (Entry entry : entries1) {
+            log.info("Read entry. Position={} Content='{}'", entry.getPosition(), new String(entry.getData()));
+            entry.release();
+        }
+        List<Entry> entries3 = cursor3.readEntries(50);
+        for (Entry entry : entries3) {
+            log.info("Read entry. Position={} Content='{}'", entry.getPosition(), new String(entry.getData()));
+            entry.release();
+        }
+
+        // activate cursors which caught up maxbacklog threshold
+        topic.checkBackloggedCursors();
+
+        // Case4:
+        // cursor1 has consumed messages so, under maxBacklog threshold => active
+        assertTrue(cursor1.isActive());
+        // cursor2 has not consumed messages so, above maxBacklog threshold => inactive
+        assertFalse(cursor2.isActive());
+        // cursor3 has not consumer so do not change to active
+        assertFalse(cursor3.isActive());
+
+        // add consumer to sub3 and read entries
+        sub3.addConsumer(consumer3);
+        entries3 = cursor3.readEntries(50);
+        for (Entry entry : entries3) {
+            log.info("Read entry. Position={} Content='{}'", entry.getPosition(), new String(entry.getData()));
+            entry.release();
+        }
+
+        topic.checkBackloggedCursors();
+        // Case5: cursor3 has consumer so change to active
+        assertTrue(cursor3.isActive());
+    }
+
+    private ByteBuf getMessageWithMetadata(byte[] data) throws IOException {
+        MessageMetadata messageData = MessageMetadata.newBuilder().setPublishTime(System.currentTimeMillis())
+            .setProducerName("prod-name").setSequenceId(0).build();
+        ByteBuf payload = Unpooled.wrappedBuffer(data, 0, data.length);
+
+        int msgMetadataSize = messageData.getSerializedSize();
+        int headersSize = 4 + msgMetadataSize;
+        ByteBuf headers = PulsarByteBufAllocator.DEFAULT.buffer(headersSize, headersSize);
+        ByteBufCodedOutputStream outStream = ByteBufCodedOutputStream.get(headers);
+        headers.writeInt(msgMetadataSize);
+        messageData.writeTo(outStream);
+        outStream.recycle();
+        return ByteBufPair.coalesce(ByteBufPair.get(headers, payload));
+    }
 }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java
index ad039e1..51a7eef 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java
@@ -1040,7 +1040,7 @@ public class SimpleProducerConsumerTest extends ProducerConsumerBase {
         Thread.sleep(maxMessageCacheRetentionTimeMillis);
 
         // 4. deactivate subscriber which has built the backlog
-        ledger.checkBackloggedCursors();
+        topicRef.checkBackloggedCursors();
         Thread.sleep(100);
 
         // 5. verify: active subscribers
@@ -1055,7 +1055,7 @@ public class SimpleProducerConsumerTest extends ProducerConsumerBase {
             subscriber2.acknowledge(msg);
         }
 
-        ledger.checkBackloggedCursors();
+        topicRef.checkBackloggedCursors();
 
         activeSubscriber.clear();
         ledger.getActiveCursors().forEach(c -> activeSubscriber.add(c.getName()));


[pulsar] 05/38: Fixed double delete on a namespace (#6713)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 40b6c916bd891ffdcde407d834ecf714cd5bc6e3
Author: Matteo Merli <mm...@apache.org>
AuthorDate: Tue Apr 14 08:46:06 2020 -0700

    Fixed double delete on a namespace (#6713)
    
    (cherry picked from commit 88f401c534f597ce8b66939d1490974bfde60441)
---
 .../apache/pulsar/broker/admin/impl/NamespacesBase.java |  9 ++++++++-
 .../org/apache/pulsar/broker/admin/AdminApiTest.java    | 17 +++++++++++++++++
 2 files changed, 25 insertions(+), 1 deletion(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
index 638d397..619a5cd 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
@@ -90,6 +90,7 @@ import org.apache.pulsar.common.policies.data.SubscriptionAuthMode;
 import org.apache.pulsar.common.util.FutureUtil;
 import org.apache.pulsar.common.util.ObjectMapperFactory;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.data.Stat;
 
 import org.slf4j.Logger;
@@ -257,7 +258,13 @@ public abstract class NamespacesBase extends AdminResource {
                 final String globalZkPolicyPath = path(POLICIES, namespaceName.toString());
                 final String lcaolZkPolicyPath = joinPath(LOCAL_POLICIES_ROOT, namespaceName.toString());
                 globalZk().delete(globalZkPolicyPath, -1);
-                localZk().delete(lcaolZkPolicyPath, -1);
+
+                try {
+                    localZk().delete(lcaolZkPolicyPath, -1);
+                } catch (NoNodeException nne) {
+                    // If the z-node with the modified information is not there anymore, we're already good
+                }
+
                 policiesCache().invalidate(globalZkPolicyPath);
                 localCacheService().policiesCache().invalidate(lcaolZkPolicyPath);
             } catch (Exception e) {
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
index ae98235..0ab6311 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
@@ -2253,6 +2253,23 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
     }
 
     @Test
+    public void testCreateAndDeleteNamespaceWithBundles() throws Exception {
+        admin.clusters().createCluster("usw", new ClusterData());
+        TenantInfo tenantInfo = new TenantInfo(Sets.newHashSet("role1", "role2"),
+                Sets.newHashSet("test", "usw"));
+        admin.tenants().updateTenant("prop-xyz", tenantInfo);
+
+        String ns = "prop-xyz/ns-" + System.nanoTime();
+
+        admin.namespaces().createNamespace(ns, 24);
+        admin.namespaces().deleteNamespace(ns);
+
+        // Re-create and re-delete
+        admin.namespaces().createNamespace(ns, 32);
+        admin.namespaces().deleteNamespace(ns);
+    }
+
+    @Test
     public void testBacklogSizeShouldBeZeroWhenConsumerAckedAllMessages() throws Exception {
         final String topic = "persistent://prop-xyz/ns1/testBacklogSizeShouldBeZeroWhenConsumerAckedAllMessages";
         Consumer<byte[]> consumer = pulsarClient.newConsumer()


[pulsar] 20/38: Fix some empty message related problems in the compacted topic. (#6795)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit b0ca8e8a8848fbbe1040e10c175c299a5c712182
Author: lipenghui <pe...@apache.org>
AuthorDate: Thu Apr 23 15:43:16 2020 +0800

    Fix some empty message related problems in the compacted topic. (#6795)
    
    ### Motivation
    
    Fix some empty message related problems in the compacted topic.
    
    1. Fix message delete of a key for the batch message.
    2. Fix compaction for all empty messages in the topic. If all messages are empty, the compaction should delete all messages in the compacted topic. Without this fix, the compact task failure with NPE and the consumer can get all messages.
    3. Seek to the compaction horizon when the last compaction task deletes all messages from the compacted topic(all previous messages are deleted by empty message). Without this fix, the consumer will stuck because of no entries in the ledger that the compacted topic used.
    
    ### Verifying this change
    
    Add unit test for the changes
    (cherry picked from commit 53407fc598286690790727635bb5067a7ac108e7)
---
 .../pulsar/client/impl/RawBatchConverter.java      |  10 +-
 .../pulsar/compaction/CompactedTopicImpl.java      |  46 +--
 .../pulsar/compaction/TwoPhaseCompactor.java       |  19 +-
 .../apache/pulsar/client/impl/RawReaderTest.java   |  19 +-
 .../apache/pulsar/compaction/CompactionTest.java   | 340 +++++++++++++++++++++
 5 files changed, 397 insertions(+), 37 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java
index 8c21a73..4ad65af 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java
@@ -29,7 +29,7 @@ import java.util.List;
 import java.util.Optional;
 import java.util.function.BiPredicate;
 
-import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.ImmutableTriple;
 import org.apache.pulsar.client.api.MessageId;
 import org.apache.pulsar.client.api.RawMessage;
 import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
@@ -52,7 +52,7 @@ public class RawBatchConverter {
         }
     }
 
-    public static List<ImmutablePair<MessageId,String>> extractIdsAndKeys(RawMessage msg)
+    public static List<ImmutableTriple<MessageId, String, Integer>> extractIdsAndKeysAndSize(RawMessage msg)
             throws IOException {
         checkArgument(msg.getMessageIdData().getBatchIndex() == -1);
 
@@ -66,7 +66,7 @@ public class RawBatchConverter {
         ByteBuf uncompressedPayload = codec.decode(payload, uncompressedSize);
         metadata.recycle();
 
-        List<ImmutablePair<MessageId,String>> idsAndKeys = new ArrayList<>();
+        List<ImmutableTriple<MessageId, String, Integer>> idsAndKeysAndSize = new ArrayList<>();
 
         for (int i = 0; i < batchSize; i++) {
             SingleMessageMetadata.Builder singleMessageMetadataBuilder = SingleMessageMetadata.newBuilder();
@@ -78,13 +78,13 @@ public class RawBatchConverter {
                                                   msg.getMessageIdData().getPartition(),
                                                   i);
             if (!singleMessageMetadataBuilder.getCompactedOut()) {
-                idsAndKeys.add(ImmutablePair.of(id, singleMessageMetadataBuilder.getPartitionKey()));
+                idsAndKeysAndSize.add(ImmutableTriple.of(id, singleMessageMetadataBuilder.getPartitionKey(), singleMessageMetadataBuilder.getPayloadSize()));
             }
             singleMessageMetadataBuilder.recycle();
             singleMessagePayload.release();
         }
         uncompressedPayload.release();
-        return idsAndKeys;
+        return idsAndKeysAndSize;
     }
 
     /**
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java
index 22efe8e..2d430a7 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java
@@ -24,6 +24,7 @@ import com.google.common.collect.ComparisonChain;
 
 import io.netty.buffer.ByteBuf;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Enumeration;
 import java.util.List;
 import java.util.NoSuchElementException;
@@ -88,28 +89,35 @@ public class CompactedTopicImpl implements CompactedTopic {
                 cursor.asyncReadEntriesOrWait(numberOfEntriesToRead, callback, ctx);
             } else {
                 compactedTopicContext.thenCompose(
-                        (context) -> {
-                            return findStartPoint(cursorPosition, context.ledger.getLastAddConfirmed(), context.cache)
-                                .thenCompose((startPoint) -> {
-                                        if (startPoint == NEWER_THAN_COMPACTED) {
-                                            cursor.asyncReadEntriesOrWait(numberOfEntriesToRead, callback, ctx);
-                                            return CompletableFuture.completedFuture(null);
-                                        } else {
-                                            long endPoint = Math.min(context.ledger.getLastAddConfirmed(),
-                                                                     startPoint + numberOfEntriesToRead);
-                                            return readEntries(context.ledger, startPoint, endPoint)
-                                                .thenAccept((entries) -> {
-                                                        Entry lastEntry = entries.get(entries.size() - 1);
-                                                        cursor.seek(lastEntry.getPosition().getNext());
-                                                        callback.readEntriesComplete(entries, ctx);
-                                                    });
-                                        }
+                    (context) -> findStartPoint(cursorPosition, context.ledger.getLastAddConfirmed(), context.cache)
+                        .thenCompose((startPoint) -> {
+                            if (startPoint == NEWER_THAN_COMPACTED && compactionHorizon.compareTo(cursorPosition) < 0) {
+                                cursor.asyncReadEntriesOrWait(numberOfEntriesToRead, callback, ctx);
+                                return CompletableFuture.completedFuture(null);
+                            } else {
+                                long endPoint = Math.min(context.ledger.getLastAddConfirmed(),
+                                                         startPoint + numberOfEntriesToRead);
+                                if (startPoint == NEWER_THAN_COMPACTED) {
+                                    cursor.seek(compactionHorizon.getNext());
+                                    callback.readEntriesComplete(Collections.emptyList(), ctx);
+                                }
+                                return readEntries(context.ledger, startPoint, endPoint)
+                                    .thenAccept((entries) -> {
+                                        Entry lastEntry = entries.get(entries.size() - 1);
+                                        cursor.seek(lastEntry.getPosition().getNext());
+                                        callback.readEntriesComplete(entries, ctx);
                                     });
-                                })
+                            }
+                        }))
                     .exceptionally((exception) -> {
+                        if (exception.getCause() instanceof NoSuchElementException) {
+                            cursor.seek(compactionHorizon.getNext());
+                            callback.readEntriesComplete(Collections.emptyList(), ctx);
+                        } else {
                             callback.readEntriesFailed(new ManagedLedgerException(exception), ctx);
-                            return null;
-                        });
+                        }
+                        return null;
+                    });
             }
         }
     }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java
index a275bb5..df7c79b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java
@@ -37,6 +37,7 @@ import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.mledger.impl.LedgerMetadataUtils;
+import org.apache.commons.lang3.tuple.ImmutableTriple;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.pulsar.broker.ServiceConfiguration;
 import org.apache.pulsar.client.api.MessageId;
@@ -128,8 +129,17 @@ public class TwoPhaseCompactor extends Compactor {
                         boolean deletedMessage = false;
                         if (RawBatchConverter.isReadableBatch(m)) {
                             try {
-                                RawBatchConverter.extractIdsAndKeys(m)
-                                    .forEach(e -> latestForKey.put(e.getRight(), e.getLeft()));
+                                for (ImmutableTriple<MessageId, String, Integer> e :
+                                        RawBatchConverter.extractIdsAndKeysAndSize(m)) {
+                                    if (e != null) {
+                                        if (e.getRight() > 0) {
+                                            latestForKey.put(e.getMiddle(), e.getLeft());
+                                        } else {
+                                            deletedMessage = true;
+                                            latestForKey.remove(e.getMiddle());
+                                        }
+                                    }
+                                }
                             } catch (IOException ioe) {
                                 log.info("Error decoding batch for message {}. Whole batch will be included in output",
                                          id, ioe);
@@ -149,7 +159,8 @@ public class TwoPhaseCompactor extends Compactor {
                         MessageId first = firstMessageId.orElse(deletedMessage ? null : id);
                         MessageId to = deletedMessage ? toMessageId.orElse(null) : id;
                         if (id.compareTo(lastMessageId) == 0) {
-                            loopPromise.complete(new PhaseOneResult(first, to, lastMessageId, latestForKey));
+                            loopPromise.complete(new PhaseOneResult(first == null ? id : first, to == null ? id : to,
+                                    lastMessageId, latestForKey));
                         } else {
                             phaseOneLoop(reader,
                                          Optional.ofNullable(first),
@@ -230,7 +241,7 @@ public class TwoPhaseCompactor extends Compactor {
                         if (RawBatchConverter.isReadableBatch(m)) {
                             try {
                                 messageToAdd = RawBatchConverter.rebatchMessage(
-                                        m, (key, subid) -> latestForKey.get(key).equals(subid));
+                                        m, (key, subid) -> subid.equals(latestForKey.get(key)));
                             } catch (IOException ioe) {
                                 log.info("Error decoding batch for message {}. Whole batch will be included in output",
                                         id, ioe);
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java
index 5ae4618..2bdd488 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java
@@ -35,6 +35,7 @@ import java.util.concurrent.TimeoutException;
 
 import org.apache.bookkeeper.mledger.ManagedLedger;
 import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.ImmutableTriple;
 import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
 import org.apache.pulsar.broker.service.persistent.PersistentTopic;
 import org.apache.pulsar.client.api.MessageId;
@@ -252,10 +253,10 @@ public class RawReaderTest extends MockedPulsarServiceBaseTest {
         while (true) {
             try (RawMessage m = reader.readNextAsync().get(1, TimeUnit.SECONDS)) {
                 Assert.assertTrue(RawBatchConverter.isReadableBatch(m));
-                List<ImmutablePair<MessageId, String>> batchKeys = RawBatchConverter.extractIdsAndKeys(m);
+                List<ImmutableTriple<MessageId, String, Integer>> batchKeys = RawBatchConverter.extractIdsAndKeysAndSize(m);
                 // Assert each key is unique
-                for (ImmutablePair<MessageId, String> pair : batchKeys) {
-                    String key = pair.right;
+                for (ImmutableTriple<MessageId, String, Integer> pair : batchKeys) {
+                    String key = pair.middle;
                     Assert.assertTrue(
                             keys.add(key),
                             "Received duplicated key '" + key + "' : already received keys = " + keys);
@@ -285,7 +286,7 @@ public class RawReaderTest extends MockedPulsarServiceBaseTest {
 
         RawReader reader = RawReader.create(pulsarClient, topic, subscription).get();
         try (RawMessage m = reader.readNextAsync().get()) {
-            List<ImmutablePair<MessageId,String>> idsAndKeys = RawBatchConverter.extractIdsAndKeys(m);
+            List<ImmutableTriple<MessageId, String, Integer>> idsAndKeys = RawBatchConverter.extractIdsAndKeysAndSize(m);
 
             Assert.assertEquals(idsAndKeys.size(), 3);
 
@@ -294,9 +295,9 @@ public class RawReaderTest extends MockedPulsarServiceBaseTest {
             Assert.assertTrue(idsAndKeys.get(1).getLeft().compareTo(idsAndKeys.get(2).getLeft()) < 0);
 
             // assert keys are as expected
-            Assert.assertEquals(idsAndKeys.get(0).getRight(), "key1");
-            Assert.assertEquals(idsAndKeys.get(1).getRight(), "key2");
-            Assert.assertEquals(idsAndKeys.get(2).getRight(), "key3");
+            Assert.assertEquals(idsAndKeys.get(0).getMiddle(), "key1");
+            Assert.assertEquals(idsAndKeys.get(1).getMiddle(), "key2");
+            Assert.assertEquals(idsAndKeys.get(2).getMiddle(), "key3");
         } finally {
             reader.closeAsync().get();
         }
@@ -321,9 +322,9 @@ public class RawReaderTest extends MockedPulsarServiceBaseTest {
         RawReader reader = RawReader.create(pulsarClient, topic, subscription).get();
         try (RawMessage m1 = reader.readNextAsync().get()) {
             RawMessage m2 = RawBatchConverter.rebatchMessage(m1, (key, id) -> key.equals("key2")).get();
-            List<ImmutablePair<MessageId,String>> idsAndKeys = RawBatchConverter.extractIdsAndKeys(m2);
+            List<ImmutableTriple<MessageId, String, Integer>> idsAndKeys = RawBatchConverter.extractIdsAndKeysAndSize(m2);
             Assert.assertEquals(idsAndKeys.size(), 1);
-            Assert.assertEquals(idsAndKeys.get(0).getRight(), "key2");
+            Assert.assertEquals(idsAndKeys.get(0).getMiddle(), "key2");
             m2.close();
             Assert.assertEquals(m1.getHeadersAndPayload().refCnt(), 1);
         } finally {
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java
index 233246c..2505a6e 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java
@@ -21,6 +21,8 @@ package org.apache.pulsar.compaction;
 import static org.mockito.Mockito.anyLong;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
 import static org.testng.Assert.assertNull;
 import static org.testng.Assert.assertTrue;
 
@@ -37,6 +39,9 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Random;
 import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
@@ -55,12 +60,15 @@ import org.apache.pulsar.client.api.Message;
 import org.apache.pulsar.client.api.MessageId;
 import org.apache.pulsar.client.api.MessageRoutingMode;
 import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerBuilder;
 import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.client.api.Reader;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
 import org.apache.pulsar.client.impl.BatchMessageIdImpl;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.RetentionPolicies;
 import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.apache.pulsar.common.util.FutureUtil;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
@@ -1308,4 +1316,336 @@ public class CompactionTest extends MockedPulsarServiceBaseTest {
         }
     }
 
+    @Test(timeOut = 20000, dataProvider = "lastDeletedBatching")
+    public void testAllEmptyCompactionLedger(boolean batchEnabled) throws Exception {
+        final String topic = "persistent://my-property/use/my-ns/testAllEmptyCompactionLedger" + UUID.randomUUID().toString();
+
+        final int messages = 10;
+
+        // 1.create producer and publish message to the topic.
+        ProducerBuilder<byte[]> builder = pulsarClient.newProducer().topic(topic);
+        if (!batchEnabled) {
+            builder.enableBatching(false);
+        } else {
+            builder.batchingMaxMessages(messages / 5);
+        }
+
+        Producer<byte[]> producer = builder.create();
+
+        List<CompletableFuture<MessageId>> futures = new ArrayList<>(messages);
+        for (int i = 0; i < messages; i++) {
+            futures.add(producer.newMessage().keyBytes("1".getBytes()).value("".getBytes()).sendAsync());
+        }
+
+        FutureUtil.waitForAll(futures).get();
+
+        // 2.compact the topic.
+        Compactor compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler);
+        compactor.compact(topic).get();
+
+        // consumer with readCompacted enabled only get compacted entries
+        try (Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topic).subscriptionName("sub1")
+                .readCompacted(true).subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe()) {
+            Message<byte[]> m = consumer.receive(2, TimeUnit.SECONDS);
+            assertNull(m);
+        }
+    }
+
+    @Test(timeOut = 20000)
+    public void testBatchAndNonBatchWithoutEmptyPayload() throws PulsarClientException, ExecutionException, InterruptedException {
+        final String topic = "persistent://my-property/use/my-ns/testBatchAndNonBatchWithoutEmptyPayload" + UUID.randomUUID().toString();
+
+        // 1.create producer and publish message to the topic.
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)
+                .batchingMaxPublishDelay(1, TimeUnit.DAYS)
+                .create();
+
+        final String k1 = "k1";
+        final String k2 = "k2";
+        producer.newMessage().key(k1).value("0".getBytes()).send();
+        List<CompletableFuture<MessageId>> futures = new ArrayList<>(7);
+        for (int i = 0; i < 2; i++) {
+            futures.add(producer.newMessage().key(k1).value((i + 1 + "").getBytes()).sendAsync());
+        }
+        producer.flush();
+        producer.newMessage().key(k1).value("3".getBytes()).send();
+        for (int i = 0; i < 2; i++) {
+            futures.add(producer.newMessage().key(k1).value((i + 4 + "").getBytes()).sendAsync());
+        }
+        producer.flush();
+
+        for (int i = 0; i < 3; i++) {
+            futures.add(producer.newMessage().key(k2).value((i + "").getBytes()).sendAsync());
+        }
+
+        producer.newMessage().key(k2).value("3".getBytes()).send();
+        producer.flush();
+        FutureUtil.waitForAll(futures).get();
+
+        // 2.compact the topic.
+        Compactor compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler);
+        compactor.compact(topic).get();
+
+        // consumer with readCompacted enabled only get compacted entries
+        try (Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topic).subscriptionName("sub1")
+                .readCompacted(true).subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe()) {
+            Message<byte[]> m1 = consumer.receive(2, TimeUnit.SECONDS);
+            Message<byte[]> m2 = consumer.receive(2, TimeUnit.SECONDS);
+            assertNotNull(m1);
+            assertNotNull(m2);
+            assertEquals(m1.getKey(), k1);
+            assertEquals(new String(m1.getValue()), "5");
+            assertEquals(m2.getKey(), k2);
+            assertEquals(new String(m2.getValue()), "3");
+            Message<byte[]> none = consumer.receive(2, TimeUnit.SECONDS);
+            assertNull(none);
+        }
+    }
+    @Test(timeOut = 20000)
+    public void testBatchAndNonBatchWithEmptyPayload() throws PulsarClientException, ExecutionException, InterruptedException {
+        final String topic = "persistent://my-property/use/my-ns/testBatchAndNonBatchWithEmptyPayload" + UUID.randomUUID().toString();
+
+        // 1.create producer and publish message to the topic.
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)
+                .batchingMaxPublishDelay(1, TimeUnit.DAYS)
+                .create();
+
+        final String k1 = "k1";
+        final String k2 = "k2";
+        final String k3 = "k3";
+        producer.newMessage().key(k1).value("0".getBytes()).send();
+        List<CompletableFuture<MessageId>> futures = new ArrayList<>(7);
+        for (int i = 0; i < 2; i++) {
+            futures.add(producer.newMessage().key(k1).value((i + 1 + "").getBytes()).sendAsync());
+        }
+        producer.flush();
+        producer.newMessage().key(k1).value("3".getBytes()).send();
+        for (int i = 0; i < 2; i++) {
+            futures.add(producer.newMessage().key(k1).value((i + 4 + "").getBytes()).sendAsync());
+        }
+        producer.flush();
+
+        for (int i = 0; i < 3; i++) {
+            futures.add(producer.newMessage().key(k2).value((i + 10 + "").getBytes()).sendAsync());
+        }
+        producer.flush();
+
+        producer.newMessage().key(k2).value("".getBytes()).send();
+
+        producer.newMessage().key(k3).value("0".getBytes()).send();
+
+        FutureUtil.waitForAll(futures).get();
+
+        // 2.compact the topic.
+        Compactor compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler);
+        compactor.compact(topic).get();
+
+        // consumer with readCompacted enabled only get compacted entries
+        try (Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topic).subscriptionName("sub1")
+                .readCompacted(true).subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe()) {
+            Message<byte[]> m1 = consumer.receive();
+            Message<byte[]> m2 = consumer.receive();
+            assertNotNull(m1);
+            assertNotNull(m2);
+            assertEquals(m1.getKey(), k1);
+            assertEquals(m2.getKey(), k3);
+            assertEquals(new String(m1.getValue()), "5");
+            assertEquals(new String(m2.getValue()), "0");
+            Message<byte[]> none = consumer.receive(2, TimeUnit.SECONDS);
+            assertNull(none);
+        }
+    }
+
+    @Test(timeOut = 20000)
+    public void testBatchAndNonBatchEndOfEmptyPayload() throws PulsarClientException, ExecutionException, InterruptedException {
+        final String topic = "persistent://my-property/use/my-ns/testBatchAndNonBatchWithEmptyPayload" + UUID.randomUUID().toString();
+
+        // 1.create producer and publish message to the topic.
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)
+                .batchingMaxPublishDelay(1, TimeUnit.DAYS)
+                .create();
+
+        final String k1 = "k1";
+        final String k2 = "k2";
+        producer.newMessage().key(k1).value("0".getBytes()).send();
+        List<CompletableFuture<MessageId>> futures = new ArrayList<>(7);
+        for (int i = 0; i < 2; i++) {
+            futures.add(producer.newMessage().key(k1).value((i + 1 + "").getBytes()).sendAsync());
+        }
+        producer.flush();
+        producer.newMessage().key(k1).value("3".getBytes()).send();
+        for (int i = 0; i < 2; i++) {
+            futures.add(producer.newMessage().key(k1).value((i + 4 + "").getBytes()).sendAsync());
+        }
+        producer.flush();
+
+        for (int i = 0; i < 3; i++) {
+            futures.add(producer.newMessage().key(k2).value((i + 10 + "").getBytes()).sendAsync());
+        }
+        producer.flush();
+
+        producer.newMessage().key(k2).value("".getBytes()).send();
+
+        FutureUtil.waitForAll(futures).get();
+
+        // 2.compact the topic.
+        Compactor compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler);
+        compactor.compact(topic).get();
+
+        // consumer with readCompacted enabled only get compacted entries
+        try (Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topic).subscriptionName("sub1")
+                .readCompacted(true).subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe()) {
+            Message<byte[]> m1 = consumer.receive();
+            assertNotNull(m1);
+            assertEquals(m1.getKey(), k1);
+            assertEquals(new String(m1.getValue()), "5");
+            Message<byte[]> none = consumer.receive(2, TimeUnit.SECONDS);
+            assertNull(none);
+        }
+    }
+
+    @Test(timeOut = 20000, dataProvider = "lastDeletedBatching")
+    public void testCompactMultipleTimesWithoutEmptyMessage(boolean batchEnabled) throws PulsarClientException, ExecutionException, InterruptedException {
+        final String topic = "persistent://my-property/use/my-ns/testCompactMultipleTimesWithoutEmptyMessage" + UUID.randomUUID().toString();
+
+        final int messages = 10;
+        final String key = "1";
+
+        // 1.create producer and publish message to the topic.
+        ProducerBuilder<byte[]> builder = pulsarClient.newProducer().topic(topic);
+        if (!batchEnabled) {
+            builder.enableBatching(false);
+        } else {
+            builder.batchingMaxMessages(messages / 5);
+        }
+
+        Producer<byte[]> producer = builder.create();
+
+        List<CompletableFuture<MessageId>> futures = new ArrayList<>(messages);
+        for (int i = 0; i < messages; i++) {
+            futures.add(producer.newMessage().key(key).value((i + "").getBytes()).sendAsync());
+        }
+
+        FutureUtil.waitForAll(futures).get();
+
+        // 2.compact the topic.
+        Compactor compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler);
+        compactor.compact(topic).get();
+
+        // 3. Send more ten messages
+        futures.clear();
+        for (int i = 0; i < messages; i++) {
+            futures.add(producer.newMessage().key(key).value((i + 10 + "").getBytes()).sendAsync());
+        }
+        FutureUtil.waitForAll(futures).get();
+
+        // 4.compact again.
+        compactor.compact(topic).get();
+
+        try (Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topic).subscriptionName("sub1")
+                .readCompacted(true).subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe()) {
+            Message<byte[]> m1 = consumer.receive();
+            assertNotNull(m1);
+            assertEquals(m1.getKey(), key);
+            assertEquals(new String(m1.getValue()), "19");
+            Message<byte[]> none = consumer.receive(2, TimeUnit.SECONDS);
+            assertNull(none);
+        }
+    }
+
+    @Test(timeOut = 2000000, dataProvider = "lastDeletedBatching")
+    public void testReadUnCompacted(boolean batchEnabled) throws PulsarClientException, ExecutionException, InterruptedException {
+        final String topic = "persistent://my-property/use/my-ns/testReadUnCompacted" + UUID.randomUUID().toString();
+
+        final int messages = 10;
+        final String key = "1";
+
+        // 1.create producer and publish message to the topic.
+        ProducerBuilder<byte[]> builder = pulsarClient.newProducer().topic(topic);
+        if (!batchEnabled) {
+            builder.enableBatching(false);
+        } else {
+            builder.batchingMaxMessages(messages / 5);
+        }
+
+        Producer<byte[]> producer = builder.create();
+
+        List<CompletableFuture<MessageId>> futures = new ArrayList<>(messages);
+        for (int i = 0; i < messages; i++) {
+            futures.add(producer.newMessage().key(key).value((i + "").getBytes()).sendAsync());
+        }
+
+        FutureUtil.waitForAll(futures).get();
+
+        // 2.compact the topic.
+        Compactor compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler);
+        compactor.compact(topic).get();
+
+        // 3. Send more ten messages
+        futures.clear();
+        for (int i = 0; i < messages; i++) {
+            futures.add(producer.newMessage().key(key).value((i + 10 + "").getBytes()).sendAsync());
+        }
+        FutureUtil.waitForAll(futures).get();
+        try (Consumer<byte[]> consumer = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub1")
+                .readCompacted(true)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe()) {
+            for (int i = 0; i < 11; i++) {
+                Message<byte[]> received = consumer.receive();
+                assertNotNull(received);
+                assertEquals(received.getKey(), key);
+                assertEquals(new String(received.getValue()), i + 9 + "");
+                consumer.acknowledge(received);
+            }
+            Message<byte[]> none = consumer.receive(2, TimeUnit.SECONDS);
+            assertNull(none);
+        }
+
+        // 4.Send empty message to delete the key-value in the compacted topic.
+        producer.newMessage().key(key).value(("").getBytes()).send();
+
+        // 5.compact the topic.
+        compactor.compact(topic).get();
+
+        try (Consumer<byte[]> consumer = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub2")
+                .readCompacted(true)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe()) {
+            Message<byte[]> none = consumer.receive(2, TimeUnit.SECONDS);
+            assertNull(none);
+        }
+
+        for (int i = 0; i < messages; i++) {
+            futures.add(producer.newMessage().key(key).value((i + 20 + "").getBytes()).sendAsync());
+        }
+        FutureUtil.waitForAll(futures).get();
+
+        try (Consumer<byte[]> consumer = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub3")
+                .readCompacted(true)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscribe()) {
+            for (int i = 0; i < 10; i++) {
+                Message<byte[]> received = consumer.receive();
+                assertNotNull(received);
+                assertEquals(received.getKey(), key);
+                assertEquals(new String(received.getValue()), i + 20 + "");
+                consumer.acknowledge(received);
+            }
+            Message<byte[]> none = consumer.receive(2, TimeUnit.SECONDS);
+            assertNull(none);
+        }
+    }
 }
\ No newline at end of file


[pulsar] 16/38: Avoid prefetch too much data when offloading data to HDFS (#6717)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 74c668a35130b85cf67256ab10466dbc06f99e50
Author: pheecian <ph...@gmail.com>
AuthorDate: Wed Apr 22 17:06:22 2020 +0800

    Avoid prefetch too much data when offloading data to HDFS (#6717)
    
    Fixes #6692
    
    ### Motivation
    avoid prefetch too much data when offloading, which may lead to OOM;
    fix object not close issue, which is also mentioned by congbobo184 https://github.com/apache/pulsar/pull/6697
    
    *Explain here the context, and why you're making that change. What is the problem you're trying to solve.*
    
    ### Does this pull request potentially affect one of the following parts:
    
    *If `yes` was chosen, please highlight the changes*
    
      - Dependencies (does it add or upgrade a dependency): (no)
      - The public API: (no)
      - The schema: (no)
      - The default values of configurations: (no)
      - The wire protocol: (no)
      - The rest endpoints: (no)
      - The admin cli options: (no)
      - Anything that affects deployment: (no)
    
    ### Documentation
    
      - Does this pull request introduce a new feature? (no)
    (cherry picked from commit 514b6af7586633424739cfc3c6131b0d0afec9e4)
---
 conf/broker.conf                                   |  3 +
 .../apache/pulsar/broker/ServiceConfiguration.java |  6 ++
 .../common/policies/data/OffloadPolicies.java      |  5 ++
 .../impl/FileSystemManagedLedgerOffloader.java     | 73 +++++++++++++++++-----
 4 files changed, 70 insertions(+), 17 deletions(-)

diff --git a/conf/broker.conf b/conf/broker.conf
index 83d009d..b1e6c1a 100644
--- a/conf/broker.conf
+++ b/conf/broker.conf
@@ -818,6 +818,9 @@ managedLedgerOffloadDriver=
 # Maximum number of thread pool threads for ledger offloading
 managedLedgerOffloadMaxThreads=2
 
+# Maximum prefetch rounds for ledger reading for offloading
+managedLedgerOffloadPrefetchRounds=1
+
 # Use Open Range-Set to cache unacked messages
 managedLedgerUnackedRangesOpenCacheSetEnabled=true
 
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
index d7d0d63..6ad8b46 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
@@ -1431,6 +1431,12 @@ public class ServiceConfiguration implements PulsarConfiguration {
     )
     private int managedLedgerOffloadMaxThreads = 2;
 
+    @FieldContext(
+            category = CATEGORY_STORAGE_OFFLOADING,
+            doc = "Maximum prefetch rounds for ledger reading for offloading"
+    )
+    private int managedLedgerOffloadPrefetchRounds = 1;
+
     /**** --- Transaction config variables --- ****/
     @FieldContext(
             category = CATEGORY_TRANSACTION,
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPolicies.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPolicies.java
index 5ccb75c..4936923 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPolicies.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPolicies.java
@@ -37,6 +37,7 @@ public class OffloadPolicies {
     public final static int DEFAULT_MAX_BLOCK_SIZE_IN_BYTES = 64 * 1024 * 1024;   // 64MB
     public final static int DEFAULT_READ_BUFFER_SIZE_IN_BYTES = 1024 * 1024;      // 1MB
     public final static int DEFAULT_OFFLOAD_MAX_THREADS = 2;
+    public final static int DEFAULT_OFFLOAD_MAX_PREFETCH_ROUNDS = 1;
     public final static String[] DRIVER_NAMES = {"S3", "aws-s3", "google-cloud-storage", "filesystem"};
     public final static String DEFAULT_OFFLOADER_DIRECTORY = "./offloaders";
     public final static long DEFAULT_OFFLOAD_THRESHOLD_IN_BYTES = -1;
@@ -46,6 +47,7 @@ public class OffloadPolicies {
     private String offloadersDirectory = DEFAULT_OFFLOADER_DIRECTORY;
     private String managedLedgerOffloadDriver = null;
     private int managedLedgerOffloadMaxThreads = DEFAULT_OFFLOAD_MAX_THREADS;
+    private int managedLedgerOffloadPrefetchRounds = DEFAULT_OFFLOAD_MAX_PREFETCH_ROUNDS;
     private long managedLedgerOffloadThresholdInBytes = DEFAULT_OFFLOAD_THRESHOLD_IN_BYTES;
     private Long managedLedgerOffloadDeletionLagInMillis = DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS;
 
@@ -161,6 +163,7 @@ public class OffloadPolicies {
         return Objects.hash(
                 managedLedgerOffloadDriver,
                 managedLedgerOffloadMaxThreads,
+                managedLedgerOffloadPrefetchRounds,
                 managedLedgerOffloadThresholdInBytes,
                 managedLedgerOffloadDeletionLagInMillis,
                 s3ManagedLedgerOffloadRegion,
@@ -190,6 +193,7 @@ public class OffloadPolicies {
         OffloadPolicies other = (OffloadPolicies) obj;
         return Objects.equals(managedLedgerOffloadDriver, other.getManagedLedgerOffloadDriver())
                 && Objects.equals(managedLedgerOffloadMaxThreads, other.getManagedLedgerOffloadMaxThreads())
+                && Objects.equals(managedLedgerOffloadPrefetchRounds, other.getManagedLedgerOffloadPrefetchRounds())
                 && Objects.equals(managedLedgerOffloadThresholdInBytes,
                     other.getManagedLedgerOffloadThresholdInBytes())
                 && Objects.equals(managedLedgerOffloadDeletionLagInMillis,
@@ -222,6 +226,7 @@ public class OffloadPolicies {
         return MoreObjects.toStringHelper(this)
                 .add("managedLedgerOffloadDriver", managedLedgerOffloadDriver)
                 .add("managedLedgerOffloadMaxThreads", managedLedgerOffloadMaxThreads)
+                .add("managedLedgerOffloadPrefetchRounds", managedLedgerOffloadPrefetchRounds)
                 .add("managedLedgerOffloadThresholdInBytes", managedLedgerOffloadThresholdInBytes)
                 .add("managedLedgerOffloadDeletionLagInMillis", managedLedgerOffloadDeletionLagInMillis)
                 .add("s3ManagedLedgerOffloadRegion", s3ManagedLedgerOffloadRegion)
diff --git a/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemManagedLedgerOffloader.java b/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemManagedLedgerOffloader.java
index bbee828..5438459 100644
--- a/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemManagedLedgerOffloader.java
+++ b/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileSystemManagedLedgerOffloader.java
@@ -20,6 +20,7 @@ package org.apache.bookkeeper.mledger.offload.filesystem.impl;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
+import io.netty.util.Recycler;
 import org.apache.bookkeeper.client.api.LedgerEntries;
 import org.apache.bookkeeper.client.api.LedgerEntry;
 import org.apache.bookkeeper.client.api.ReadHandle;
@@ -45,6 +46,7 @@ import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Executors;
+import java.util.concurrent.Semaphore;
 import java.util.concurrent.atomic.AtomicLong;
 
 import static org.apache.bookkeeper.mledger.offload.OffloadUtils.buildLedgerMetadataFormat;
@@ -68,6 +70,7 @@ public class FileSystemManagedLedgerOffloader implements LedgerOffloader {
     public static boolean driverSupported(String driver) {
         return DRIVER_NAMES.equals(driver);
     }
+
     @Override
     public String getOffloadDriverName() {
         return driverName;
@@ -82,7 +85,7 @@ public class FileSystemManagedLedgerOffloader implements LedgerOffloader {
         this.configuration = new Configuration();
         if (conf.getFileSystemProfilePath() != null) {
             String[] paths = conf.getFileSystemProfilePath().split(",");
-            for (int i =0 ; i < paths.length; i++) {
+            for (int i = 0; i < paths.length; i++) {
                 configuration.addResource(new Path(paths[i]));
             }
         }
@@ -106,6 +109,7 @@ public class FileSystemManagedLedgerOffloader implements LedgerOffloader {
                 .numThreads(conf.getManagedLedgerOffloadMaxThreads())
                 .name("offload-assignment").build();
     }
+
     @VisibleForTesting
     public FileSystemManagedLedgerOffloader(OffloadPolicies conf, OrderedScheduler scheduler, String testHDFSPath, String baseDir) throws IOException {
         this.offloadPolicies = conf;
@@ -137,7 +141,7 @@ public class FileSystemManagedLedgerOffloader implements LedgerOffloader {
     @Override
     public CompletableFuture<Void> offload(ReadHandle readHandle, UUID uuid, Map<String, String> extraMetadata) {
         CompletableFuture<Void> promise = new CompletableFuture<>();
-        scheduler.chooseThread(readHandle.getId()).submit(new LedgerReader(readHandle, uuid, extraMetadata, promise, storageBasePath, configuration, assignmentScheduler));
+        scheduler.chooseThread(readHandle.getId()).submit(new LedgerReader(readHandle, uuid, extraMetadata, promise, storageBasePath, configuration, assignmentScheduler, offloadPolicies.getManagedLedgerOffloadPrefetchRounds()));
         return promise;
     }
 
@@ -151,9 +155,10 @@ public class FileSystemManagedLedgerOffloader implements LedgerOffloader {
         private final Configuration configuration;
         volatile Exception fileSystemWriteException = null;
         private OrderedScheduler assignmentScheduler;
+        private int managedLedgerOffloadPrefetchRounds = 1;
 
         private LedgerReader(ReadHandle readHandle, UUID uuid, Map<String, String> extraMetadata, CompletableFuture<Void> promise,
-                             String storageBasePath, Configuration configuration, OrderedScheduler assignmentScheduler) {
+                             String storageBasePath, Configuration configuration, OrderedScheduler assignmentScheduler, int managedLedgerOffloadPrefetchRounds) {
             this.readHandle = readHandle;
             this.uuid = uuid;
             this.extraMetadata = extraMetadata;
@@ -161,6 +166,7 @@ public class FileSystemManagedLedgerOffloader implements LedgerOffloader {
             this.storageBasePath = storageBasePath;
             this.configuration = configuration;
             this.assignmentScheduler = assignmentScheduler;
+            this.managedLedgerOffloadPrefetchRounds = managedLedgerOffloadPrefetchRounds;
         }
 
         @Override
@@ -188,13 +194,17 @@ public class FileSystemManagedLedgerOffloader implements LedgerOffloader {
                 AtomicLong haveOffloadEntryNumber = new AtomicLong(0);
                 long needToOffloadFirstEntryNumber = 0;
                 CountDownLatch countDownLatch;
+                //avoid prefetch too much data into memory
+                Semaphore semaphore = new Semaphore(managedLedgerOffloadPrefetchRounds);
                 do {
                     long end = Math.min(needToOffloadFirstEntryNumber + ENTRIES_PER_READ - 1, readHandle.getLastAddConfirmed());
                     log.debug("read ledger entries. start: {}, end: {}", needToOffloadFirstEntryNumber, end);
                     LedgerEntries ledgerEntriesOnce = readHandle.readAsync(needToOffloadFirstEntryNumber, end).get();
+                    semaphore.acquire();
                     countDownLatch = new CountDownLatch(1);
-                    assignmentScheduler.chooseThread(ledgerId).submit(new FileSystemWriter(ledgerEntriesOnce, dataWriter,
-                            countDownLatch, haveOffloadEntryNumber, this)).addListener(() -> {}, Executors.newSingleThreadExecutor());
+                    assignmentScheduler.chooseThread(ledgerId).submit(FileSystemWriter.create(ledgerEntriesOnce, dataWriter, semaphore,
+                            countDownLatch, haveOffloadEntryNumber, this)).addListener(() -> {
+                    }, Executors.newSingleThreadExecutor());
                     needToOffloadFirstEntryNumber = end + 1;
                 } while (needToOffloadFirstEntryNumber - 1 != readHandle.getLastAddConfirmed() && fileSystemWriteException == null);
                 countDownLatch.await();
@@ -216,24 +226,50 @@ public class FileSystemManagedLedgerOffloader implements LedgerOffloader {
 
     private static class FileSystemWriter implements Runnable {
 
-        private final LedgerEntries ledgerEntriesOnce;
+        private LedgerEntries ledgerEntriesOnce;
 
         private final LongWritable key = new LongWritable();
         private final BytesWritable value = new BytesWritable();
 
-        private final MapFile.Writer dataWriter;
-        private final CountDownLatch countDownLatch;
-        private final AtomicLong haveOffloadEntryNumber;
-        private final LedgerReader ledgerReader;
+        private MapFile.Writer dataWriter;
+        private CountDownLatch countDownLatch;
+        private AtomicLong haveOffloadEntryNumber;
+        private LedgerReader ledgerReader;
+        private Semaphore semaphore;
+        private Recycler.Handle<FileSystemWriter> recyclerHandle;
+
+        private FileSystemWriter(Recycler.Handle<FileSystemWriter> recyclerHandle) {
+            this.recyclerHandle = recyclerHandle;
+        }
+
+        private static final Recycler<FileSystemWriter> RECYCLER = new Recycler<FileSystemWriter>() {
+            @Override
+            protected FileSystemWriter newObject(Recycler.Handle<FileSystemWriter> handle) {
+                return new FileSystemWriter(handle);
+            }
+        };
+
+        private void recycle() {
+            this.dataWriter = null;
+            this.countDownLatch = null;
+            this.haveOffloadEntryNumber = null;
+            this.ledgerReader = null;
+            this.ledgerEntriesOnce = null;
+            this.semaphore = null;
+            recyclerHandle.recycle(this);
+        }
 
 
-        private FileSystemWriter(LedgerEntries ledgerEntriesOnce, MapFile.Writer dataWriter,
-                                 CountDownLatch countDownLatch, AtomicLong haveOffloadEntryNumber, LedgerReader ledgerReader) {
-            this.ledgerEntriesOnce = ledgerEntriesOnce;
-            this.dataWriter = dataWriter;
-            this.countDownLatch = countDownLatch;
-            this.haveOffloadEntryNumber = haveOffloadEntryNumber;
-            this.ledgerReader = ledgerReader;
+        public static FileSystemWriter create(LedgerEntries ledgerEntriesOnce, MapFile.Writer dataWriter, Semaphore semaphore,
+                                              CountDownLatch countDownLatch, AtomicLong haveOffloadEntryNumber, LedgerReader ledgerReader) {
+            FileSystemWriter writer = RECYCLER.get();
+            writer.ledgerReader = ledgerReader;
+            writer.dataWriter = dataWriter;
+            writer.countDownLatch = countDownLatch;
+            writer.haveOffloadEntryNumber = haveOffloadEntryNumber;
+            writer.ledgerEntriesOnce = ledgerEntriesOnce;
+            writer.semaphore = semaphore;
+            return writer;
         }
 
         @Override
@@ -255,6 +291,9 @@ public class FileSystemManagedLedgerOffloader implements LedgerOffloader {
                 }
             }
             countDownLatch.countDown();
+            ledgerEntriesOnce.close();
+            semaphore.release();
+            this.recycle();
         }
     }
 


[pulsar] 32/38: [broker] Increase timeout for loading topics (#6750)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 121cf08922987d98a54a9655e169a9651b09486d
Author: Addison Higham <ad...@gmail.com>
AuthorDate: Thu May 7 01:55:14 2020 -0600

    [broker] Increase timeout for loading topics (#6750)
    
    In #6489, a timeout was introduced to make sure calls into the
    BrokerService finish or error out. However, this timeout is too low by
    default when loading topics that have many replicated clusters.
    
    Loading replicated topics is quite an expensive operation, involve
    global ZK lookups and the start of many sub-processes. While we would
    hope it finishes in 60 seconds we want to safe.
    
    Long term, it may make sense to break out this operation into more
    steps where each step can have it's own timeout
    
    Co-authored-by: Addison Higham <ah...@instructure.com>(cherry picked from commit 6854b007aaba90128530808035d0402b27e93846)
---
 .../main/java/org/apache/pulsar/broker/ServiceConfiguration.java    | 6 ++++++
 .../main/java/org/apache/pulsar/broker/service/BrokerService.java   | 3 ++-
 2 files changed, 8 insertions(+), 1 deletion(-)

diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
index 8d9a2b7..6b92061 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
@@ -215,6 +215,12 @@ public class ServiceConfiguration implements PulsarConfiguration {
     private long brokerShutdownTimeoutMs = 60000;
 
     @FieldContext(
+            category = CATEGORY_SERVER,
+            doc = "Amount of seconds to timeout when loading a topic. In situations with many geo-replicated clusters, this may need raised."
+    )
+    private long topicLoadTimeoutSeconds = 60;
+
+    @FieldContext(
         category = CATEGORY_POLICIES,
         doc = "Enable backlog quota check. Enforces actions on topic when the quota is reached"
     )
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
index 808f0c3..7a36d0b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
@@ -851,7 +851,8 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
     protected CompletableFuture<Optional<Topic>> loadOrCreatePersistentTopic(final String topic,
             boolean createIfMissing) throws RuntimeException {
         checkTopicNsOwnership(topic);
-        final CompletableFuture<Optional<Topic>> topicFuture = futureWithDeadline();
+        final CompletableFuture<Optional<Topic>> topicFuture = futureWithDeadline(pulsar.getConfiguration().getTopicLoadTimeoutSeconds(),
+                TimeUnit.SECONDS, new TimeoutException("Failed to load topic within timeout"));
         if (!pulsar.getConfiguration().isEnablePersistentTopics()) {
             if (log.isDebugEnabled()) {
                 log.debug("Broker is unable to load persistent topic {}", topic);


[pulsar] 38/38: Add Tls with keystore type config support (#6853)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 9e72dfb7f88f529ad264974fa277894926b4a6af
Author: Jia Zhai <zh...@apache.org>
AuthorDate: Fri May 8 15:41:19 2020 +0800

    Add Tls with keystore type config support (#6853)
    
    Fixes #6640
    
    Add Tls with keystore type config.
    
    Add Tls with keystore type config.
    
    - Unit test passed
    
    (cherry picked from commit 367ce7829827e4b0853e1f5a50566192bb82bf54)
---
 conf/broker.conf                                   |  56 ++++
 conf/client.conf                                   |  11 +
 conf/standalone.conf                               | 100 +++++-
 .../apache/pulsar/broker/ServiceConfiguration.java |  98 ++++++
 .../authentication/AuthenticationDataHttps.java    |   1 -
 .../OneStageAuthenticationState.java               |   2 +-
 .../java/org/apache/pulsar/PulsarStandalone.java   |  25 +-
 .../org/apache/pulsar/broker/PulsarService.java    |  30 +-
 .../pulsar/broker/service/BrokerService.java       |  20 +-
 .../broker/service/PulsarChannelInitializer.java   |  62 ++--
 .../org/apache/pulsar/broker/web/WebService.java   |  35 +-
 .../pulsar/client/api/TlsProducerConsumerTest.java |   6 +-
 .../client/impl/AdminApiKeyStoreTlsAuthTest.java   | 229 +++++++++++++
 .../KeyStoreTlsProducerConsumerTestWithAuth.java   | 267 ++++++++++++++++
 ...KeyStoreTlsProducerConsumerTestWithoutAuth.java | 255 +++++++++++++++
 .../apache/pulsar/client/impl/KeyStoreTlsTest.java |  80 +++++
 .../authentication/keystoretls/broker.keystore.jks | Bin 0 -> 2767 bytes
 .../keystoretls/broker.truststore.jks              | Bin 0 -> 731 bytes
 .../keystoretls/brokerKeyStorePW.txt               |   1 +
 .../keystoretls/brokerTrustStorePW.txt             |   1 +
 .../authentication/keystoretls/client.keystore.jks | Bin 0 -> 2767 bytes
 .../keystoretls/client.truststore.jks              | Bin 0 -> 731 bytes
 .../keystoretls/clientKeyStorePW.txt               |   1 +
 .../keystoretls/clientTrustStorePW.txt             |   1 +
 .../pulsar/client/admin/PulsarAdminBuilder.java    |  60 +++-
 .../admin/internal/PulsarAdminBuilderImpl.java     |  49 ++-
 .../admin/internal/http/AsyncHttpConnector.java    |  46 ++-
 .../client/api/AuthenticationDataProvider.java     |   9 +
 .../apache/pulsar/client/api/ClientBuilder.java    |  64 ++++
 .../apache/pulsar/client/api/KeyStoreParams.java   |  41 +--
 .../apache/pulsar/admin/cli/PulsarAdminTool.java   |  22 +-
 .../apache/pulsar/client/cli/PulsarClientTool.java |  18 ++
 .../pulsar/client/impl/ClientBuilderImpl.java      |  47 ++-
 .../org/apache/pulsar/client/impl/HttpClient.java  |  68 ++--
 .../pulsar/client/impl/HttpLookupService.java      |   3 +-
 .../client/impl/PulsarChannelInitializer.java      |  39 ++-
 .../impl/auth/AuthenticationDataKeyStoreTls.java   |  32 +-
 .../impl/auth/AuthenticationKeyStoreTls.java       | 136 ++++++++
 .../pulsar/client/impl/auth/AuthenticationTls.java |   4 +-
 .../client/impl/conf/ClientConfigurationData.java  |  12 +
 pulsar-common/pom.xml                              |   4 +
 .../common/util/ClientSslContextRefresher.java     |  67 ----
 .../common/util/DefaultSslContextBuilder.java      |  18 +-
 .../util/NettyClientSslContextRefresher.java       |  74 +++++
 ...lder.java => NettyServerSslContextBuilder.java} |  33 +-
 .../common/util/SslContextAutoRefreshBuilder.java  |  46 +--
 .../util/keystoretls/KeyStoreSSLContext.java       | 355 +++++++++++++++++++++
 .../util/keystoretls/NetSslContextBuilder.java     |  92 ++++++
 .../NettySSLContextAutoRefreshBuilder.java         | 144 +++++++++
 .../keystoretls/SSLContextValidatorEngine.java     | 176 ++++++++++
 .../SslContextFactoryWithAutoRefresh.java          |  63 ++++
 .../common/util/keystoretls/package-info.java      |  34 +-
 .../src/test/resources/broker.keystore.jks         | Bin 0 -> 2767 bytes
 .../src/test/resources/broker.truststore.jks       | Bin 0 -> 731 bytes
 .../src/test/resources/brokerKeyStorePW.txt        |   1 +
 .../src/test/resources/brokerTrustStorePW.txt      |   1 +
 pulsar-common/src/test/resources/ca-cert           |  16 +
 pulsar-common/src/test/resources/ca-cert.srl       |   1 +
 pulsar-common/src/test/resources/ca-key            |  30 ++
 pulsar-common/src/test/resources/cert-file         |  17 +
 pulsar-common/src/test/resources/cert-signed       |  22 ++
 .../src/test/resources/client.keystore.jks         | Bin 0 -> 2767 bytes
 .../src/test/resources/client.truststore.jks       | Bin 0 -> 731 bytes
 .../src/test/resources/clientKeyStorePW.txt        |   1 +
 .../src/test/resources/clientTrustStorePW.txt      |   1 +
 .../src/test/resources/old/broker.keystore.jks     | Bin 0 -> 2928 bytes
 .../src/test/resources/old/broker.truststore.jks   | Bin 0 -> 797 bytes
 .../src/test/resources/old/brokerKeyStorePW.txt    |   1 +
 .../src/test/resources/old/brokerTrustStorePW.txt  |   1 +
 .../src/test/resources/old/client.keystore.jks     | Bin 0 -> 2926 bytes
 .../src/test/resources/old/client.truststore.jks   | Bin 0 -> 797 bytes
 .../src/test/resources/old/clientKeyStorePW.txt    |   1 +
 .../src/test/resources/old/clientTrustStorePW.txt  |   1 +
 .../service/ServiceChannelInitializer.java         |  47 ++-
 .../discovery/service/server/ServerManager.java    |  35 +-
 .../discovery/service/server/ServiceConfig.java    | 222 ++-----------
 .../pulsar/proxy/server/DirectProxyHandler.java    |  14 +-
 .../pulsar/proxy/server/ProxyConfiguration.java    | 104 +++++-
 .../pulsar/proxy/server/ProxyConnection.java       |  21 +-
 .../proxy/server/ServiceChannelInitializer.java    |  90 +++++-
 .../org/apache/pulsar/proxy/server/WebServer.java  |  37 ++-
 .../proxy/server/ProxyKeyStoreTlsTestWithAuth.java | 202 ++++++++++++
 .../server/ProxyKeyStoreTlsTestWithoutAuth.java    | 186 +++++++++++
 .../authentication/keystoretls/broker.keystore.jks | Bin 0 -> 2767 bytes
 .../keystoretls/broker.truststore.jks              | Bin 0 -> 731 bytes
 .../keystoretls/brokerKeyStorePW.txt               |   1 +
 .../keystoretls/brokerTrustStorePW.txt             |   1 +
 .../authentication/keystoretls/client.keystore.jks | Bin 0 -> 2767 bytes
 .../keystoretls/client.truststore.jks              | Bin 0 -> 731 bytes
 .../keystoretls/clientKeyStorePW.txt               |   1 +
 .../keystoretls/clientTrustStorePW.txt             |   1 +
 site2/docs/reference-configuration.md              |  12 +
 92 files changed, 3566 insertions(+), 539 deletions(-)

diff --git a/conf/broker.conf b/conf/broker.conf
index b1e6c1a..d0c942a 100644
--- a/conf/broker.conf
+++ b/conf/broker.conf
@@ -389,6 +389,62 @@ tlsCiphers=
 # authentication.
 tlsRequireTrustedClientCertOnConnect=false
 
+### --- KeyStore TLS config variables --- ###
+# Enable TLS with KeyStore type configuration in broker.
+tlsEnabledWithKeyStore=false
+
+# TLS Provider for KeyStore type
+tlsProvider=
+
+# TLS KeyStore type configuration in broker: JKS, PKCS12
+tlsKeyStoreType=JKS
+
+# TLS KeyStore path in broker
+tlsKeyStore=
+
+# TLS KeyStore password for broker
+tlsKeyStorePassword=
+
+# TLS TrustStore type configuration in broker: JKS, PKCS12
+tlsTrustStoreType=JKS
+
+# TLS TrustStore path in broker
+tlsTrustStore=
+
+# TLS TrustStore password in broker
+tlsTrustStorePassword=
+
+# Whether internal client use KeyStore type to authenticate with Pulsar brokers
+brokerClientTlsEnabledWithKeyStore=false
+
+# The TLS Provider used by internal client to authenticate with other Pulsar brokers
+brokerClientSslProvider=
+
+# TLS TrustStore type configuration for internal client: JKS, PKCS12
+# used by the internal client to authenticate with Pulsar brokers
+brokerClientTlsTrustStoreType=JKS
+
+# TLS TrustStore path for internal client
+# used by the internal client to authenticate with Pulsar brokers
+brokerClientTlsTrustStore=
+
+# TLS TrustStore password for internal client,
+# used by the internal client to authenticate with Pulsar brokers
+brokerClientTlsTrustStorePassword=
+
+# Specify the tls cipher the internal client will use to negotiate during TLS Handshake
+# (a comma-separated list of ciphers)
+# e.g.  [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256].
+# used by the internal client to authenticate with Pulsar brokers
+brokerClientTlsCiphers=
+
+# Specify the tls protocols the broker will use to negotiate during TLS handshake
+# (a comma-separated list of protocol names).
+# e.g.  [TLSv1.2, TLSv1.1, TLSv1]
+# used by the internal client to authenticate with Pulsar brokers
+brokerClientTlsProtocols=
+
+
 ### --- Authentication --- ###
 
 # Enable authentication
diff --git a/conf/client.conf b/conf/client.conf
index 887785a..597478e 100644
--- a/conf/client.conf
+++ b/conf/client.conf
@@ -56,3 +56,14 @@ tlsEnableHostnameVerification=false
 # fails, then the cert is untrusted and the connection is dropped.
 tlsTrustCertsFilePath=
 
+# Enable TLS with KeyStore type configuration in broker.
+useKeyStoreTls=false;
+
+# TLS KeyStore type configuration: JKS, PKCS12
+tlsTrustStoreType=JKS
+
+# TLS TrustStore path
+tlsTrustStorePath=
+
+# TLS TrustStore password
+tlsTrustStorePassword=
diff --git a/conf/standalone.conf b/conf/standalone.conf
index 7dff0c3..3d6c12e 100644
--- a/conf/standalone.conf
+++ b/conf/standalone.conf
@@ -169,8 +169,8 @@ dispatchThrottlingRatePerTopicInMsg=0
 # default message-byte dispatch-throttling
 dispatchThrottlingRatePerTopicInByte=0
 
-# Dispatch rate-limiting relative to publish rate. 
-# (Enabling flag will make broker to dynamically update dispatch-rate relatively to publish-rate: 
+# Dispatch rate-limiting relative to publish rate.
+# (Enabling flag will make broker to dynamically update dispatch-rate relatively to publish-rate:
 # throttle-dispatch-rate = (publish-rate + configured dispatch-rate).
 dispatchThrottlingRateRelativeToPublishRate=false
 
@@ -211,6 +211,102 @@ maxConsumersPerTopic=0
 # Using a value of 0, is disabling maxConsumersPerSubscription-limit check.
 maxConsumersPerSubscription=0
 
+### --- TLS --- ###
+# Deprecated - Use webServicePortTls and brokerServicePortTls instead
+tlsEnabled=false
+
+# Tls cert refresh duration in seconds (set 0 to check on every new connection)
+tlsCertRefreshCheckDurationSec=300
+
+# Path for the TLS certificate file
+tlsCertificateFilePath=
+
+# Path for the TLS private key file
+tlsKeyFilePath=
+
+# Path for the trusted TLS certificate file.
+# 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.
+tlsTrustCertsFilePath=
+
+# Accept untrusted TLS certificate from client.
+# If true, a client with a cert which cannot be verified with the
+# 'tlsTrustCertsFilePath' cert will allowed to connect to the server,
+# though the cert will not be used for client authentication.
+tlsAllowInsecureConnection=false
+
+# Specify the tls protocols the broker will use to negotiate during TLS handshake
+# (a comma-separated list of protocol names).
+# Examples:- [TLSv1.2, TLSv1.1, TLSv1]
+tlsProtocols=
+
+# Specify the tls cipher the broker will use to negotiate during TLS Handshake
+# (a comma-separated list of ciphers).
+# Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256]
+tlsCiphers=
+
+# Trusted client certificates are required for to connect TLS
+# Reject the Connection if the Client Certificate is not trusted.
+# In effect, this requires that all connecting clients perform TLS client
+# authentication.
+tlsRequireTrustedClientCertOnConnect=false
+
+### --- KeyStore TLS config variables --- ###
+# Enable TLS with KeyStore type configuration in broker.
+tlsEnabledWithKeyStore=false
+
+# TLS Provider for KeyStore type
+tlsProvider=
+
+# TLS KeyStore type configuration in broker: JKS, PKCS12
+tlsKeyStoreType=JKS
+
+# TLS KeyStore path in broker
+tlsKeyStore=
+
+# TLS KeyStore password for broker
+tlsKeyStorePassword=
+
+# TLS TrustStore type configuration in broker: JKS, PKCS12
+tlsTrustStoreType=JKS
+
+# TLS TrustStore path in broker
+tlsTrustStore=
+
+# TLS TrustStore password for broker
+tlsTrustStorePassword=
+
+# Whether internal client use KeyStore type to authenticate with Pulsar brokers
+brokerClientTlsEnabledWithKeyStore=false
+
+# The TLS Provider used by internal client to authenticate with other Pulsar brokers
+brokerClientSslProvider=
+
+# TLS TrustStore type configuration for internal client: JKS, PKCS12
+# used by the internal client to authenticate with Pulsar brokers
+brokerClientTlsTrustStoreType=JKS
+
+# TLS TrustStore path for internal client
+# used by the internal client to authenticate with Pulsar brokers
+brokerClientTlsTrustStore=
+
+# TLS TrustStore password for internal client,
+# used by the internal client to authenticate with Pulsar brokers
+brokerClientTlsTrustStorePassword=
+
+# Specify the tls cipher the internal client will use to negotiate during TLS Handshake
+# (a comma-separated list of ciphers)
+# e.g.  [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256].
+# used by the internal client to authenticate with Pulsar brokers
+brokerClientTlsCiphers=
+
+# Specify the tls protocols the broker will use to negotiate during TLS handshake
+# (a comma-separated list of protocol names).
+# e.g.  [TLSv1.2, TLSv1.1, TLSv1]
+# used by the internal client to authenticate with Pulsar brokers
+brokerClientTlsProtocols=
+
 ### --- Authentication --- ###
 # Role names that are treated as "proxy roles". If the broker sees a request with
 #role as proxyRoles - it will demand to see a valid original principal.
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
index 6b92061..68d56fe 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
@@ -77,6 +77,8 @@ public class ServiceConfiguration implements PulsarConfiguration {
     @Category
     private static final String CATEGORY_TLS = "TLS";
     @Category
+    private static final String CATEGORY_KEYSTORE_TLS = "KeyStoreTLS";
+    @Category
     private static final String CATEGORY_AUTHENTICATION = "Authentication";
     @Category
     private static final String CATEGORY_AUTHORIZATION = "Authorization";
@@ -1458,6 +1460,102 @@ public class ServiceConfiguration implements PulsarConfiguration {
     private String transactionMetadataStoreProviderClassName =
             "org.apache.pulsar.transaction.coordinator.impl.InMemTransactionMetadataStoreProvider";
 
+    /**** --- KeyStore TLS config variables --- ****/
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "Enable TLS with KeyStore type configuration in broker"
+    )
+    private boolean tlsEnabledWithKeyStore = false;
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS Provider for KeyStore type"
+    )
+    private String tlsProvider = null;
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS KeyStore type configuration in broker: JKS, PKCS12"
+    )
+    private String tlsKeyStoreType = "JKS";
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS KeyStore path in broker"
+    )
+    private String tlsKeyStore = null;
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS KeyStore password for broker"
+    )
+    private String tlsKeyStorePassword = null;
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS TrustStore type configuration in broker: JKS, PKCS12"
+    )
+    private String tlsTrustStoreType = "JKS";
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS TrustStore path in broker"
+    )
+    private String tlsTrustStore = null;
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS TrustStore password for broker"
+    )
+    private String tlsTrustStorePassword = null;
+
+    /**** --- KeyStore TLS config variables used for internal client/admin to auth with other broker--- ****/
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "Whether internal client use KeyStore type to authenticate with other Pulsar brokers"
+    )
+    private boolean brokerClientTlsEnabledWithKeyStore = false;
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "The TLS Provider used by internal client to authenticate with other Pulsar brokers"
+    )
+    private String brokerClientSslProvider = null;
+    // needed when client auth is required
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS TrustStore type configuration for internal client: JKS, PKCS12 "
+                  + " used by the internal client to authenticate with Pulsar brokers"
+    )
+    private String brokerClientTlsTrustStoreType = "JKS";
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS TrustStore path for internal client, "
+                  + " used by the internal client to authenticate with Pulsar brokers"
+    )
+    private String brokerClientTlsTrustStore = null;
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS TrustStore password for internal client, "
+                  + " used by the internal client to authenticate with Pulsar brokers"
+    )
+    private String brokerClientTlsTrustStorePassword = null;
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "Specify the tls cipher the internal client will use to negotiate during TLS Handshake"
+                  + " (a comma-separated list of ciphers).\n\n"
+                  + "Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256].\n"
+                  + " used by the internal client to authenticate with Pulsar brokers"
+    )
+    private Set<String> brokerClientTlsCiphers = Sets.newTreeSet();
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_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] \n"
+                  + " used by the internal client to authenticate with Pulsar brokers"
+    )
+    private Set<String> brokerClientTlsProtocols = Sets.newTreeSet();
+
     /**
      * @deprecated See {@link #getConfigurationStoreServers}
      */
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java
index 03a9bd3..4e1d33b 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java
@@ -34,7 +34,6 @@ public class AuthenticationDataHttps extends AuthenticationDataHttp {
     /*
      * TLS
      */
-
     @Override
     public boolean hasDataFromTls() {
         return (certificates != null);
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/OneStageAuthenticationState.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/OneStageAuthenticationState.java
index 06b1749..f2667c3 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/OneStageAuthenticationState.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/OneStageAuthenticationState.java
@@ -42,7 +42,7 @@ public class OneStageAuthenticationState implements AuthenticationState {
                                        SSLSession sslSession,
                                        AuthenticationProvider provider) throws AuthenticationException {
         this.authenticationDataSource = new AuthenticationDataCommand(
-            new String(authData.getBytes(), UTF_8), remoteAddress, sslSession);;
+            new String(authData.getBytes(), UTF_8), remoteAddress, sslSession);
         this.authRole = provider.authenticate(authenticationDataSource);
     }
 
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java
index 7c76257..be1b276 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java
@@ -33,6 +33,7 @@ import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.ServiceConfiguration;
 import org.apache.pulsar.broker.ServiceConfigurationUtils;
 import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminBuilder;
 import org.apache.pulsar.client.admin.PulsarAdminException;
 import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.policies.data.ClusterData;
@@ -332,11 +333,29 @@ public class PulsarStandalone implements AutoCloseable {
             createSampleNameSpace(clusterData, cluster);
         } else {
             URL webServiceUrlTls = new URL(
-                    String.format("http://%s:%d", config.getAdvertisedAddress(), config.getWebServicePortTls().get()));
+                    String.format("https://%s:%d", config.getAdvertisedAddress(), config.getWebServicePortTls().get()));
             String brokerServiceUrlTls = String.format("pulsar+ssl://%s:%d", config.getAdvertisedAddress(),
                     config.getBrokerServicePortTls().get());
-            admin = PulsarAdmin.builder().serviceHttpUrl(webServiceUrlTls.toString()).authentication(
-                    config.getBrokerClientAuthenticationPlugin(), config.getBrokerClientAuthenticationParameters()).build();
+            PulsarAdminBuilder builder = PulsarAdmin.builder()
+                    .serviceHttpUrl(webServiceUrlTls.toString())
+                    .authentication(
+                            config.getBrokerClientAuthenticationPlugin(),
+                            config.getBrokerClientAuthenticationParameters());
+
+            // set trust store if needed.
+            if (config.isBrokerClientTlsEnabled()) {
+                if (config.isBrokerClientTlsEnabledWithKeyStore()) {
+                    builder.useKeyStoreTls(true)
+                            .tlsTrustStoreType(config.getBrokerClientTlsTrustStoreType())
+                            .tlsTrustStorePath(config.getBrokerClientTlsTrustStore())
+                            .tlsTrustStorePassword(config.getBrokerClientTlsTrustStorePassword());
+                } else {
+                    builder.tlsTrustCertsFilePath(config.getBrokerClientTrustCertsFilePath());
+                }
+                builder.allowTlsInsecureConnection(config.isTlsAllowInsecureConnection());
+            }
+
+            admin = builder.build();
             ClusterData clusterData = new ClusterData(null, webServiceUrlTls.toString(), null, brokerServiceUrlTls);
             createSampleNameSpace(clusterData, cluster);
         }
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 eeb3b1a..62801ee 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
@@ -522,9 +522,9 @@ public class PulsarService implements AutoCloseable {
 
             final String bootstrapMessage = "bootstrap service "
                     + (config.getWebServicePort().isPresent() ? "port = " + config.getWebServicePort().get() : "")
-                    + (config.getWebServicePortTls().isPresent() ? "tls-port = " + config.getWebServicePortTls() : "")
-                    + (config.getBrokerServicePort().isPresent() ? "broker url= " + brokerServiceUrl : "")
-                    + (config.getBrokerServicePortTls().isPresent() ? "broker url= " + brokerServiceUrlTls : "");
+                    + (config.getWebServicePortTls().isPresent() ? ", tls-port = " + config.getWebServicePortTls() : "")
+                    + (config.getBrokerServicePort().isPresent() ? ", broker url= " + brokerServiceUrl : "")
+                    + (config.getBrokerServicePortTls().isPresent() ? ", broker tls url= " + brokerServiceUrlTls : "");
             LOG.info("messaging service is ready");
 
             LOG.info("messaging service is ready, {}, cluster={}, configs={}", bootstrapMessage,
@@ -935,10 +935,17 @@ public class PulsarService implements AutoCloseable {
                     .tlsTrustCertsFilePath(this.getConfiguration().getTlsCertificateFilePath());
 
                 if (this.getConfiguration().isBrokerClientTlsEnabled()) {
-                    builder.tlsTrustCertsFilePath(
-                        isNotBlank(this.getConfiguration().getBrokerClientTrustCertsFilePath())
-                            ? this.getConfiguration().getBrokerClientTrustCertsFilePath()
-                            : this.getConfiguration().getTlsCertificateFilePath());
+                    if (this.getConfiguration().isBrokerClientTlsEnabledWithKeyStore()) {
+                        builder.useKeyStoreTls(true)
+                                .tlsTrustStoreType(this.getConfiguration().getBrokerClientTlsTrustStoreType())
+                                .tlsTrustStorePath(this.getConfiguration().getBrokerClientTlsTrustStore())
+                                .tlsTrustStorePassword(this.getConfiguration().getBrokerClientTlsTrustStorePassword());
+                    } else {
+                        builder.tlsTrustCertsFilePath(
+                                isNotBlank(this.getConfiguration().getBrokerClientTrustCertsFilePath())
+                                        ? this.getConfiguration().getBrokerClientTrustCertsFilePath()
+                                        : this.getConfiguration().getTlsCertificateFilePath());
+                    }
                 }
 
                 if (isNotBlank(this.getConfiguration().getBrokerClientAuthenticationPlugin())) {
@@ -964,7 +971,14 @@ public class PulsarService implements AutoCloseable {
                                 conf.getBrokerClientAuthenticationParameters());
 
                 if (conf.isBrokerClientTlsEnabled()) {
-                    builder.tlsTrustCertsFilePath(conf.getBrokerClientTrustCertsFilePath());
+                    if (conf.isBrokerClientTlsEnabledWithKeyStore()) {
+                        builder.useKeyStoreTls(true)
+                                .tlsTrustStoreType(conf.getBrokerClientTlsTrustStoreType())
+                                .tlsTrustStorePath(conf.getBrokerClientTlsTrustStore())
+                                .tlsTrustStorePassword(conf.getBrokerClientTlsTrustStorePassword());
+                    } else {
+                        builder.tlsTrustCertsFilePath(conf.getBrokerClientTrustCertsFilePath());
+                    }
                     builder.allowTlsInsecureConnection(conf.isTlsAllowInsecureConnection());
                 }
 
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
index 7a36d0b..ce040ed 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
@@ -786,8 +786,17 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
                             .serviceUrl(isNotBlank(data.getBrokerServiceUrlTls()) ? data.getBrokerServiceUrlTls()
                                     : data.getServiceUrlTls())
                             .enableTls(true)
-                            .tlsTrustCertsFilePath(pulsar.getConfiguration().getBrokerClientTrustCertsFilePath())
                             .allowTlsInsecureConnection(pulsar.getConfiguration().isTlsAllowInsecureConnection());
+                    if (pulsar.getConfiguration().isBrokerClientTlsEnabledWithKeyStore()) {
+                        clientBuilder.useKeyStoreTls(true)
+                                .tlsTrustStoreType(pulsar.getConfiguration().getBrokerClientTlsTrustStoreType())
+                                .tlsTrustStorePath(pulsar.getConfiguration().getBrokerClientTlsTrustStore())
+                                .tlsTrustStorePassword(pulsar.getConfiguration()
+                                        .getBrokerClientTlsTrustStorePassword());
+                    } else {
+                        clientBuilder.tlsTrustCertsFilePath(pulsar.getConfiguration()
+                                .getBrokerClientTrustCertsFilePath());
+                    }
                 } else {
                     clientBuilder.serviceUrl(
                             isNotBlank(data.getBrokerServiceUrl()) ? data.getBrokerServiceUrl() : data.getServiceUrl());
@@ -823,8 +832,15 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
                                 conf.getBrokerClientAuthenticationParameters());
 
                 if (isTlsUrl) {
-                    builder.tlsTrustCertsFilePath(conf.getBrokerClientTrustCertsFilePath());
                     builder.allowTlsInsecureConnection(conf.isTlsAllowInsecureConnection());
+                    if (conf.isBrokerClientTlsEnabledWithKeyStore()) {
+                        builder.useKeyStoreTls(true)
+                                .tlsTrustStoreType(conf.getBrokerClientTlsTrustStoreType())
+                                .tlsTrustStorePath(conf.getBrokerClientTlsTrustStore())
+                                .tlsTrustStorePassword(conf.getBrokerClientTlsTrustStorePassword());
+                    } else {
+                        builder.tlsTrustCertsFilePath(conf.getBrokerClientTrustCertsFilePath());
+                    }
                 }
 
                 // most of the admin request requires to make zk-call so, keep the max read-timeout based on
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java
index ce16a7e..2a2d3d5 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java
@@ -20,23 +20,24 @@ package org.apache.pulsar.broker.service;
 
 import static org.apache.bookkeeper.util.SafeRunnable.safeRun;
 
-import java.net.SocketAddress;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.pulsar.broker.PulsarService;
-import org.apache.pulsar.broker.ServiceConfiguration;
-import org.apache.pulsar.common.protocol.ByteBufPair;
-import org.apache.pulsar.common.protocol.Commands;
-import org.apache.pulsar.common.util.NettySslContextBuilder;
-
 import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
-
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
 import io.netty.handler.flow.FlowControlHandler;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslHandler;
+import java.net.SocketAddress;
+import java.util.concurrent.TimeUnit;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.common.protocol.ByteBufPair;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.NettyServerSslContextBuilder;
+import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder;
+import org.apache.pulsar.common.util.keystoretls.NettySSLContextAutoRefreshBuilder;
 
 @Slf4j
 public class PulsarChannelInitializer extends ChannelInitializer<SocketChannel> {
@@ -45,8 +46,10 @@ public class PulsarChannelInitializer extends ChannelInitializer<SocketChannel>
 
     private final PulsarService pulsar;
     private final boolean enableTls;
-    private final NettySslContextBuilder sslCtxRefresher;
+    private final boolean tlsEnabledWithKeyStore;
+    private SslContextAutoRefreshBuilder<SslContext> sslCtxRefresher;
     private final ServiceConfiguration brokerConf;
+    private NettySSLContextAutoRefreshBuilder nettySSLContextAutoRefreshBuilder;
 
     // This cache is used to maintain a list of active connections to iterate over them
     // We keep weak references to have the cache to be auto cleaned up when the connections
@@ -66,13 +69,31 @@ public class PulsarChannelInitializer extends ChannelInitializer<SocketChannel>
         super();
         this.pulsar = pulsar;
         this.enableTls = enableTLS;
+        ServiceConfiguration serviceConfig = pulsar.getConfiguration();
+        this.tlsEnabledWithKeyStore = serviceConfig.isTlsEnabledWithKeyStore();
         if (this.enableTls) {
-            ServiceConfiguration serviceConfig = pulsar.getConfiguration();
-            sslCtxRefresher = new NettySslContextBuilder(serviceConfig.isTlsAllowInsecureConnection(),
-                    serviceConfig.getTlsTrustCertsFilePath(), serviceConfig.getTlsCertificateFilePath(),
-                    serviceConfig.getTlsKeyFilePath(), serviceConfig.getTlsCiphers(), serviceConfig.getTlsProtocols(),
-                    serviceConfig.isTlsRequireTrustedClientCertOnConnect(),
-                    serviceConfig.getTlsCertRefreshCheckDurationSec());
+            if (tlsEnabledWithKeyStore) {
+                nettySSLContextAutoRefreshBuilder = new NettySSLContextAutoRefreshBuilder(
+                        serviceConfig.getTlsProvider(),
+                        serviceConfig.getTlsKeyStoreType(),
+                        serviceConfig.getTlsKeyStore(),
+                        serviceConfig.getTlsKeyStorePassword(),
+                        serviceConfig.isTlsAllowInsecureConnection(),
+                        serviceConfig.getTlsTrustStoreType(),
+                        serviceConfig.getTlsTrustStore(),
+                        serviceConfig.getTlsTrustStorePassword(),
+                        serviceConfig.isTlsRequireTrustedClientCertOnConnect(),
+                        serviceConfig.getTlsCiphers(),
+                        serviceConfig.getTlsProtocols(),
+                        serviceConfig.getTlsCertRefreshCheckDurationSec());
+            } else {
+                sslCtxRefresher = new NettyServerSslContextBuilder(serviceConfig.isTlsAllowInsecureConnection(),
+                        serviceConfig.getTlsTrustCertsFilePath(), serviceConfig.getTlsCertificateFilePath(),
+                        serviceConfig.getTlsKeyFilePath(),
+                        serviceConfig.getTlsCiphers(), serviceConfig.getTlsProtocols(),
+                        serviceConfig.isTlsRequireTrustedClientCertOnConnect(),
+                        serviceConfig.getTlsCertRefreshCheckDurationSec());
+            }
         } else {
             this.sslCtxRefresher = null;
         }
@@ -86,7 +107,12 @@ public class PulsarChannelInitializer extends ChannelInitializer<SocketChannel>
     @Override
     protected void initChannel(SocketChannel ch) throws Exception {
         if (this.enableTls) {
-            ch.pipeline().addLast(TLS_HANDLER, sslCtxRefresher.get().newHandler(ch.alloc()));
+            if (this.tlsEnabledWithKeyStore) {
+                ch.pipeline().addLast(TLS_HANDLER,
+                        new SslHandler(nettySSLContextAutoRefreshBuilder.get().createSSLEngine()));
+            } else {
+                ch.pipeline().addLast(TLS_HANDLER, sslCtxRefresher.get().newHandler(ch.alloc()));
+            }
             ch.pipeline().addLast("ByteBufPairEncoder", ByteBufPair.COPYING_ENCODER);
         } else {
             ch.pipeline().addLast("ByteBufPairEncoder", ByteBufPair.ENCODER);
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 ffaec7c..1a41ce1 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 com.google.common.collect.Lists;
-
 import io.prometheus.client.jetty.JettyStatisticsCollector;
 
 import java.util.ArrayList;
@@ -28,12 +27,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.TimeZone;
-
 import javax.servlet.DispatcherType;
 
 import org.apache.pulsar.broker.PulsarServerException;
 import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.broker.ServiceConfiguration;
 import org.apache.pulsar.common.util.SecurityUtility;
+import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
@@ -97,13 +97,30 @@ public class WebService implements AutoCloseable {
         Optional<Integer> tlsPort = pulsar.getConfiguration().getWebServicePortTls();
         if (tlsPort.isPresent()) {
             try {
-                SslContextFactory sslCtxFactory = SecurityUtility.createSslContextFactory(
-                        pulsar.getConfiguration().isTlsAllowInsecureConnection(),
-                        pulsar.getConfiguration().getTlsTrustCertsFilePath(),
-                        pulsar.getConfiguration().getTlsCertificateFilePath(),
-                        pulsar.getConfiguration().getTlsKeyFilePath(),
-                        pulsar.getConfiguration().isTlsRequireTrustedClientCertOnConnect(), true,
-                        pulsar.getConfiguration().getTlsCertRefreshCheckDurationSec());
+                SslContextFactory sslCtxFactory;
+                ServiceConfiguration config = pulsar.getConfiguration();
+                if (config.isTlsEnabledWithKeyStore()) {
+                    sslCtxFactory = KeyStoreSSLContext.createSslContextFactory(
+                            config.getTlsProvider(),
+                            config.getTlsKeyStoreType(),
+                            config.getTlsKeyStore(),
+                            config.getTlsKeyStorePassword(),
+                            config.isTlsAllowInsecureConnection(),
+                            config.getTlsTrustStoreType(),
+                            config.getTlsTrustStore(),
+                            config.getTlsTrustStorePassword(),
+                            config.isTlsRequireTrustedClientCertOnConnect(),
+                            config.getTlsCertRefreshCheckDurationSec()
+                    );
+                } else {
+                    sslCtxFactory = SecurityUtility.createSslContextFactory(
+                            config.isTlsAllowInsecureConnection(),
+                            config.getTlsTrustCertsFilePath(),
+                            config.getTlsCertificateFilePath(),
+                            config.getTlsKeyFilePath(),
+                            config.isTlsRequireTrustedClientCertOnConnect(), true,
+                            config.getTlsCertRefreshCheckDurationSec());
+                }
                 httpsConnector = new PulsarServerConnector(server, 1, 1, sslCtxFactory);
                 httpsConnector.setPort(tlsPort.get());
                 httpsConnector.setHost(pulsar.getBindAddress());
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java
index 55bc4a7..9f1eac8 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java
@@ -182,14 +182,14 @@ public class TlsProducerConsumerTest extends TlsProducerConsumerBase {
 
     /**
      * It verifies that AuthenticationTls provides cert refresh functionality.
-     * 
+     *
      * <pre>
      *  a. Create Auth with invalid cert
      *  b. Consumer fails with invalid tls certs
      *  c. refresh cert in provider
      *  d. Consumer successfully gets created
      * </pre>
-     * 
+     *
      * @throws Exception
      */
     @Test
@@ -234,5 +234,5 @@ public class TlsProducerConsumerTest extends TlsProducerConsumerBase {
 
     private ByteArrayInputStream getStream(AtomicInteger index, ByteArrayInputStream... streams) {
         return streams[index.intValue()];
-    } 
+    }
 }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AdminApiKeyStoreTlsAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AdminApiKeyStoreTlsAuthTest.java
new file mode 100644
index 0000000..ab2833d
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/AdminApiKeyStoreTlsAuthTest.java
@@ -0,0 +1,229 @@
+/**
+ * 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.client.impl;
+
+import static org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls.mapToString;
+import static org.testng.Assert.fail;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.net.ssl.SSLContext;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.ClientBuilder;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.GenericType;
+import javax.ws.rs.core.MediaType;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.http.conn.ssl.NoopHostnameVerifier;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderTls;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.admin.internal.JacksonConfigurator;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext;
+import org.glassfish.jersey.client.ClientConfig;
+import org.glassfish.jersey.client.ClientProperties;
+import org.glassfish.jersey.jackson.JacksonFeature;
+import org.glassfish.jersey.media.multipart.MultiPartFeature;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class AdminApiKeyStoreTlsAuthTest extends ProducerConsumerBase {
+    protected final String BROKER_KEYSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/broker.keystore.jks";
+    protected final String BROKER_TRUSTSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/broker.truststore.jks";
+    protected final String BROKER_KEYSTORE_PW = "111111";
+    protected final String BROKER_TRUSTSTORE_PW = "111111";
+
+    protected final String CLIENT_KEYSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/client.keystore.jks";
+    protected final String CLIENT_TRUSTSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/client.truststore.jks";
+    protected final String CLIENT_KEYSTORE_PW = "111111";
+    protected final String CLIENT_TRUSTSTORE_PW = "111111";
+
+    protected final String CLIENT_KEYSTORE_CN = "clientuser";
+    protected final String KEYSTORE_TYPE = "JKS";
+
+    private final String clusterName = "test";
+    Set<String> tlsProtocols = Sets.newConcurrentHashSet();
+
+    @BeforeMethod
+    @Override
+    public void setup() throws Exception {
+        conf.setLoadBalancerEnabled(true);
+        conf.setBrokerServicePortTls(Optional.of(0));
+        conf.setWebServicePortTls(Optional.of(0));
+
+        conf.setTlsEnabledWithKeyStore(true);
+        conf.setTlsKeyStoreType(KEYSTORE_TYPE);
+        conf.setTlsKeyStore(BROKER_KEYSTORE_FILE_PATH);
+        conf.setTlsKeyStorePassword(BROKER_KEYSTORE_PW);
+
+        conf.setTlsTrustStoreType(KEYSTORE_TYPE);
+        conf.setTlsTrustStore(CLIENT_TRUSTSTORE_FILE_PATH);
+        conf.setTlsTrustStorePassword(CLIENT_TRUSTSTORE_PW);
+
+        conf.setClusterName(clusterName);
+        conf.setTlsRequireTrustedClientCertOnConnect(true);
+        tlsProtocols.add("TLSv1.2");
+        conf.setTlsProtocols(tlsProtocols);
+
+        // config for authentication and authorization.
+        conf.setSuperUserRoles(Sets.newHashSet(CLIENT_KEYSTORE_CN));
+        conf.setAuthenticationEnabled(true);
+        conf.setAuthorizationEnabled(true);
+        Set<String> providers = new HashSet<>();
+        providers.add(AuthenticationProviderTls.class.getName());
+        conf.setAuthenticationProviders(providers);
+
+        conf.setBrokerClientTlsEnabled(true);
+        conf.setBrokerClientTlsEnabledWithKeyStore(true);
+
+        // set broker client tls auth
+        Map<String, String> authParams = new HashMap<>();
+        authParams.put(AuthenticationKeyStoreTls.KEYSTORE_TYPE, KEYSTORE_TYPE);
+        authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PATH, CLIENT_KEYSTORE_FILE_PATH);
+        authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PW, CLIENT_KEYSTORE_PW);
+        conf.setBrokerClientAuthenticationPlugin(AuthenticationKeyStoreTls.class.getName());
+        conf.setBrokerClientAuthenticationParameters(mapToString(authParams));
+        conf.setBrokerClientTlsTrustStore(BROKER_TRUSTSTORE_FILE_PATH);
+        conf.setBrokerClientTlsTrustStorePassword(BROKER_TRUSTSTORE_PW);
+
+        super.init();
+    }
+
+    @AfterMethod
+    @Override
+    public void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    WebTarget buildWebClient() throws Exception {
+        ClientConfig httpConfig = new ClientConfig();
+        httpConfig.property(ClientProperties.FOLLOW_REDIRECTS, true);
+        httpConfig.property(ClientProperties.ASYNC_THREADPOOL_SIZE, 8);
+        httpConfig.register(MultiPartFeature.class);
+
+        ClientBuilder clientBuilder = ClientBuilder.newBuilder().withConfig(httpConfig)
+            .register(JacksonConfigurator.class).register(JacksonFeature.class);
+
+        SSLContext sslCtx = KeyStoreSSLContext.createClientSslContext(
+                KEYSTORE_TYPE,
+                CLIENT_KEYSTORE_FILE_PATH,
+                CLIENT_KEYSTORE_PW,
+                KEYSTORE_TYPE,
+                BROKER_TRUSTSTORE_FILE_PATH,
+                BROKER_TRUSTSTORE_PW);
+
+        clientBuilder.sslContext(sslCtx).hostnameVerifier(NoopHostnameVerifier.INSTANCE);
+        Client client = clientBuilder.build();
+
+        return client.target(brokerUrlTls.toString());
+    }
+
+    PulsarAdmin buildAdminClient() throws Exception {
+        Map<String, String> authParams = new HashMap<>();
+        authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PATH, CLIENT_KEYSTORE_FILE_PATH);
+        authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PW, CLIENT_KEYSTORE_PW);
+
+        return PulsarAdmin.builder()
+                .serviceHttpUrl(brokerUrlTls.toString())
+                .useKeyStoreTls(true)
+                .tlsTrustStorePath(BROKER_TRUSTSTORE_FILE_PATH)
+                .tlsTrustStorePassword(BROKER_TRUSTSTORE_PW)
+                .allowTlsInsecureConnection(false)
+                .authentication(AuthenticationKeyStoreTls.class.getName(), authParams)
+                .build();
+    }
+
+    @Test
+    public void testSuperUserCanListTenants() throws Exception {
+        try (PulsarAdmin admin = buildAdminClient()) {
+            admin.clusters().createCluster("test", new ClusterData(brokerUrl.toString()));
+            admin.tenants().createTenant("tenant1",
+                                         new TenantInfo(ImmutableSet.of("foobar"),
+                                                        ImmutableSet.of("test")));
+            Assert.assertEquals(ImmutableSet.of("tenant1"), admin.tenants().getTenants());
+        }
+    }
+
+    @Test
+    public void testSuperUserCantListNamespaces() throws Exception {
+        try (PulsarAdmin admin = buildAdminClient()) {
+            admin.clusters().createCluster("test", new ClusterData(brokerUrl.toString()));
+            admin.tenants().createTenant("tenant1",
+                                         new TenantInfo(ImmutableSet.of("proxy"),
+                                                        ImmutableSet.of("test")));
+            admin.namespaces().createNamespace("tenant1/ns1");
+            admin.namespaces().getNamespaces("tenant1").contains("tenant1/ns1");
+        }
+    }
+
+    @Test
+    public void testAuthorizedUserAsOriginalPrincipal() throws Exception {
+        try (PulsarAdmin admin = buildAdminClient()) {
+            admin.clusters().createCluster("test", new ClusterData(brokerUrl.toString()));
+            admin.tenants().createTenant("tenant1",
+                                         new TenantInfo(ImmutableSet.of("proxy", "user1"),
+                                                        ImmutableSet.of("test")));
+            admin.namespaces().createNamespace("tenant1/ns1");
+        }
+        WebTarget root = buildWebClient();
+        Assert.assertEquals(ImmutableSet.of("tenant1/ns1"),
+                            root.path("/admin/v2/namespaces").path("tenant1")
+                            .request(MediaType.APPLICATION_JSON)
+                            .header("X-Original-Principal", "user1")
+                            .get(new GenericType<List<String>>() {}));
+    }
+
+    @Test
+    public void testPersistentList() throws Exception {
+        log.info("-- Starting {} test --", methodName);
+
+        try (PulsarAdmin admin = buildAdminClient()) {
+            admin.clusters().createCluster("test", new ClusterData(brokerUrl.toString()));
+            admin.tenants().createTenant("tenant1",
+                    new TenantInfo(ImmutableSet.of("foobar"),
+                            ImmutableSet.of("test")));
+            Assert.assertEquals(ImmutableSet.of("tenant1"), admin.tenants().getTenants());
+
+            admin.namespaces().createNamespace("tenant1/ns1");
+
+            // this will calls internal admin to list nonpersist topics.
+            admin.topics().getList("tenant1/ns1");
+        } catch (PulsarAdminException ex) {
+            ex.printStackTrace();
+            fail("Should not have thrown an exception");
+        }
+    }
+}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithAuth.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithAuth.java
new file mode 100644
index 0000000..14177e0
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithAuth.java
@@ -0,0 +1,267 @@
+/**
+ * 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.client.impl;
+
+import static org.mockito.Mockito.spy;
+
+import com.google.common.collect.Sets;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderTls;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+// TLS authentication and authorization based on KeyStore type config.
+@Slf4j
+public class KeyStoreTlsProducerConsumerTestWithAuth extends ProducerConsumerBase {
+    protected final String BROKER_KEYSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/broker.keystore.jks";
+    protected final String BROKER_TRUSTSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/broker.truststore.jks";
+    protected final String BROKER_KEYSTORE_PW = "111111";
+    protected final String BROKER_TRUSTSTORE_PW = "111111";
+
+    protected final String CLIENT_KEYSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/client.keystore.jks";
+    protected final String CLIENT_TRUSTSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/client.truststore.jks";
+    protected final String CLIENT_KEYSTORE_PW = "111111";
+    protected final String CLIENT_TRUSTSTORE_PW = "111111";
+
+    protected final String CLIENT_KEYSTORE_CN = "clientuser";
+    protected final String KEYSTORE_TYPE = "JKS";
+
+    private final String clusterName = "use";
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        // TLS configuration for Broker
+        internalSetUpForBroker();
+
+        // Start Broker
+
+        super.init();
+    }
+
+    @AfterMethod
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    protected void internalSetUpForBroker() throws Exception {
+        conf.setBrokerServicePortTls(Optional.of(0));
+        conf.setWebServicePortTls(Optional.of(0));
+        conf.setTlsEnabledWithKeyStore(true);
+
+        conf.setTlsKeyStoreType(KEYSTORE_TYPE);
+        conf.setTlsKeyStore(BROKER_KEYSTORE_FILE_PATH);
+        conf.setTlsKeyStorePassword(BROKER_KEYSTORE_PW);
+
+        conf.setTlsTrustStoreType(KEYSTORE_TYPE);
+        conf.setTlsTrustStore(CLIENT_TRUSTSTORE_FILE_PATH);
+        conf.setTlsTrustStorePassword(CLIENT_TRUSTSTORE_PW);
+
+        conf.setClusterName(clusterName);
+        conf.setTlsRequireTrustedClientCertOnConnect(true);
+
+        // config for authentication and authorization.
+        conf.setSuperUserRoles(Sets.newHashSet(CLIENT_KEYSTORE_CN));
+        conf.setAuthenticationEnabled(true);
+        conf.setAuthorizationEnabled(true);
+        Set<String> providers = new HashSet<>();
+        providers.add(AuthenticationProviderTls.class.getName());
+        conf.setAuthenticationProviders(providers);
+    }
+
+    protected void internalSetUpForClient(boolean addCertificates, String lookupUrl) throws Exception {
+        if (pulsarClient != null) {
+            pulsarClient.close();
+        }
+
+        Set<String> tlsProtocols = Sets.newConcurrentHashSet();
+        tlsProtocols.add("TLSv1.2");
+
+        ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(lookupUrl)
+                .enableTls(true)
+                .useKeyStoreTls(true)
+                .tlsTrustStorePath(BROKER_TRUSTSTORE_FILE_PATH)
+                .tlsTrustStorePassword(BROKER_TRUSTSTORE_PW)
+                .allowTlsInsecureConnection(false)
+                .tlsProtocols(tlsProtocols)
+                .operationTimeout(1000, TimeUnit.MILLISECONDS);
+        if (addCertificates) {
+            Map<String, String> authParams = new HashMap<>();
+            authParams.put(AuthenticationKeyStoreTls.KEYSTORE_TYPE, KEYSTORE_TYPE);
+            authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PATH, CLIENT_KEYSTORE_FILE_PATH);
+            authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PW, CLIENT_KEYSTORE_PW);
+            clientBuilder.authentication(AuthenticationKeyStoreTls.class.getName(), authParams);
+        }
+        pulsarClient = clientBuilder.build();
+    }
+
+    protected void internalSetUpForNamespace() throws Exception {
+        Map<String, String> authParams = new HashMap<>();
+        authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PATH, CLIENT_KEYSTORE_FILE_PATH);
+        authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PW, CLIENT_KEYSTORE_PW);
+
+        if (admin != null) {
+            admin.close();
+        }
+
+        admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrlTls.toString())
+                .useKeyStoreTls(true)
+                .tlsTrustStorePath(BROKER_TRUSTSTORE_FILE_PATH)
+                .tlsTrustStorePassword(BROKER_TRUSTSTORE_PW)
+                .allowTlsInsecureConnection(false)
+                .authentication(AuthenticationKeyStoreTls.class.getName(), authParams).build());
+        admin.clusters().createCluster(clusterName, new ClusterData(brokerUrl.toString(), brokerUrlTls.toString(),
+                pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()));
+        admin.tenants().createTenant("my-property",
+                new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use")));
+        admin.namespaces().createNamespace("my-property/my-ns");
+    }
+
+    /**
+     * verifies that messages whose size is larger than 2^14 bytes (max size of single TLS chunk) can be
+     * produced/consumed
+     *
+     * @throws Exception
+     */
+    @Test(timeOut = 30000)
+    public void testTlsLargeSizeMessage() throws Exception {
+        log.info("-- Starting {} test --", methodName);
+
+        final int MESSAGE_SIZE = 16 * 1024 + 1;
+        log.info("-- message size --", MESSAGE_SIZE);
+        String topicName = "persistent://my-property/use/my-ns/testTlsLargeSizeMessage"
+                           + System.currentTimeMillis();
+
+        internalSetUpForClient(true, pulsar.getBrokerServiceUrlTls());
+        internalSetUpForNamespace();
+
+        Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName)
+                .subscriptionName("my-subscriber-name").subscribe();
+
+        Producer<byte[]> producer = pulsarClient.newProducer().topic(topicName)
+                .create();
+        for (int i = 0; i < 10; i++) {
+            byte[] message = new byte[MESSAGE_SIZE];
+            Arrays.fill(message, (byte) i);
+            producer.send(message);
+        }
+
+        Message<byte[]> msg = null;
+        for (int i = 0; i < 10; i++) {
+            msg = consumer.receive(5, TimeUnit.SECONDS);
+            byte[] expected = new byte[MESSAGE_SIZE];
+            Arrays.fill(expected, (byte) i);
+            Assert.assertEquals(expected, msg.getData());
+        }
+        // Acknowledge the consumption of all messages at once
+        consumer.acknowledgeCumulative(msg);
+        consumer.close();
+        log.info("-- Exiting {} test --", methodName);
+    }
+
+    @Test(timeOut = 300000)
+    public void testTlsClientAuthOverBinaryProtocol() throws Exception {
+        log.info("-- Starting {} test --", methodName);
+
+        final int MESSAGE_SIZE = 16 * 1024 + 1;
+        log.info("-- message size --", MESSAGE_SIZE);
+        String topicName = "persistent://my-property/use/my-ns/testTlsClientAuthOverBinaryProtocol"
+                           + System.currentTimeMillis();
+
+        internalSetUpForNamespace();
+
+        // Test 1 - Using TLS on binary protocol without sending certs - expect failure
+        internalSetUpForClient(false, pulsar.getBrokerServiceUrlTls());
+
+        try {
+            pulsarClient.newConsumer().topic(topicName)
+                    .subscriptionName("my-subscriber-name").subscriptionType(SubscriptionType.Exclusive).subscribe();
+            Assert.fail("Server should have failed the TLS handshake since client didn't .");
+        } catch (Exception ex) {
+            // OK
+        }
+
+        // Using TLS on binary protocol - sending certs
+        internalSetUpForClient(true, pulsar.getBrokerServiceUrlTls());
+
+        try {
+            pulsarClient.newConsumer().topic(topicName)
+                    .subscriptionName("my-subscriber-name").subscriptionType(SubscriptionType.Exclusive).subscribe();
+        } catch (Exception ex) {
+            Assert.fail("Should not fail since certs are sent.");
+        }
+    }
+
+    @Test(timeOut = 30000)
+    public void testTlsClientAuthOverHTTPProtocol() throws Exception {
+        log.info("-- Starting {} test --", methodName);
+
+        final int MESSAGE_SIZE = 16 * 1024 + 1;
+        log.info("-- message size --", MESSAGE_SIZE);
+        String topicName = "persistent://my-property/use/my-ns/testTlsClientAuthOverHTTPProtocol"
+                           + System.currentTimeMillis();
+
+        internalSetUpForNamespace();
+
+        // Test 1 - Using TLS on https without sending certs - expect failure
+        internalSetUpForClient(false, pulsar.getWebServiceAddressTls());
+        try {
+            pulsarClient.newConsumer().topic(topicName)
+                    .subscriptionName("my-subscriber-name").subscriptionType(SubscriptionType.Exclusive).subscribe();
+            Assert.fail("Server should have failed the TLS handshake since client didn't .");
+        } catch (Exception ex) {
+            // OK
+        }
+
+        // Test 2 - Using TLS on https - sending certs
+        internalSetUpForClient(true, pulsar.getWebServiceAddressTls());
+        try {
+            pulsarClient.newConsumer().topic(topicName)
+                    .subscriptionName("my-subscriber-name").subscriptionType(SubscriptionType.Exclusive).subscribe();
+        } catch (Exception ex) {
+            Assert.fail("Should not fail since certs are sent.");
+        }
+    }
+
+}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithoutAuth.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithoutAuth.java
new file mode 100644
index 0000000..c95f3df
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithoutAuth.java
@@ -0,0 +1,255 @@
+/**
+ * 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.client.impl;
+
+import static org.mockito.Mockito.spy;
+
+import com.google.common.collect.Sets;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderTls;
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls;
+import org.apache.pulsar.common.policies.data.ClusterData;
+import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+// TLS test without authentication and authorization based on KeyStore type config.
+@Slf4j
+public class KeyStoreTlsProducerConsumerTestWithoutAuth extends ProducerConsumerBase {
+    protected final String BROKER_KEYSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/broker.keystore.jks";
+    protected final String BROKER_TRUSTSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/broker.truststore.jks";
+    protected final String BROKER_KEYSTORE_PW = "111111";
+    protected final String BROKER_TRUSTSTORE_PW = "111111";
+
+    protected final String CLIENT_KEYSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/client.keystore.jks";
+    protected final String CLIENT_TRUSTSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/client.truststore.jks";
+    protected final String CLIENT_KEYSTORE_PW = "111111";
+    protected final String CLIENT_TRUSTSTORE_PW = "111111";
+
+    protected final String KEYSTORE_TYPE = "JKS";
+
+    private final String clusterName = "use";
+    Set<String> tlsProtocols = Sets.newConcurrentHashSet();
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        // TLS configuration for Broker
+        internalSetUpForBroker();
+
+        // Start Broker
+        super.init();
+    }
+
+    @AfterMethod
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    protected void internalSetUpForBroker() throws Exception {
+        conf.setBrokerServicePortTls(Optional.of(0));
+        conf.setWebServicePortTls(Optional.of(0));
+        conf.setTlsEnabledWithKeyStore(true);
+
+        conf.setTlsKeyStoreType(KEYSTORE_TYPE);
+        conf.setTlsKeyStore(BROKER_KEYSTORE_FILE_PATH);
+        conf.setTlsKeyStorePassword(BROKER_KEYSTORE_PW);
+
+        conf.setTlsTrustStoreType(KEYSTORE_TYPE);
+        conf.setTlsTrustStore(CLIENT_TRUSTSTORE_FILE_PATH);
+        conf.setTlsTrustStorePassword(CLIENT_TRUSTSTORE_PW);
+
+        conf.setClusterName(clusterName);
+        conf.setTlsRequireTrustedClientCertOnConnect(true);
+        tlsProtocols.add("TLSv1.2");
+        conf.setTlsProtocols(tlsProtocols);
+    }
+
+    protected void internalSetUpForClient(boolean addCertificates, String lookupUrl) throws Exception {
+        if (pulsarClient != null) {
+            pulsarClient.close();
+        }
+
+        ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(lookupUrl)
+                .enableTls(true)
+                .useKeyStoreTls(true)
+                .tlsTrustStorePath(BROKER_TRUSTSTORE_FILE_PATH)
+                .tlsTrustStorePassword(BROKER_TRUSTSTORE_PW)
+                .allowTlsInsecureConnection(false)
+                .operationTimeout(1000, TimeUnit.MILLISECONDS);
+        if (addCertificates) {
+            Map<String, String> authParams = new HashMap<>();
+            authParams.put(AuthenticationKeyStoreTls.KEYSTORE_TYPE, KEYSTORE_TYPE);
+            authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PATH, CLIENT_KEYSTORE_FILE_PATH);
+            authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PW, CLIENT_KEYSTORE_PW);
+            clientBuilder.authentication(AuthenticationKeyStoreTls.class.getName(), authParams);
+        }
+        pulsarClient = clientBuilder.build();
+    }
+
+    protected void internalSetUpForNamespace() throws Exception {
+        Map<String, String> authParams = new HashMap<>();
+        authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PATH, CLIENT_KEYSTORE_FILE_PATH);
+        authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PW, CLIENT_KEYSTORE_PW);
+
+        if (admin != null) {
+            admin.close();
+        }
+
+        admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrlTls.toString())
+                .useKeyStoreTls(true)
+                .tlsTrustStorePath(BROKER_TRUSTSTORE_FILE_PATH)
+                .tlsTrustStorePassword(BROKER_TRUSTSTORE_PW)
+                .allowTlsInsecureConnection(true)
+                .authentication(AuthenticationKeyStoreTls.class.getName(), authParams).build());
+        admin.clusters().createCluster(clusterName, new ClusterData(brokerUrl.toString(), brokerUrlTls.toString(),
+                pulsar.getBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls()));
+        admin.tenants().createTenant("my-property",
+                new TenantInfo(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use")));
+        admin.namespaces().createNamespace("my-property/my-ns");
+    }
+
+    /**
+     * verifies that messages whose size is larger than 2^14 bytes (max size of single TLS chunk) can be
+     * produced/consumed
+     *
+     * @throws Exception
+     */
+    @Test(timeOut = 30000)
+    public void testTlsLargeSizeMessage() throws Exception {
+        log.info("-- Starting {} test --", methodName);
+
+        final int MESSAGE_SIZE = 16 * 1024 + 1;
+        log.info("-- message size --", MESSAGE_SIZE);
+        String topicName = "persistent://my-property/use/my-ns/testTlsLargeSizeMessage"
+                           + System.currentTimeMillis();
+
+        internalSetUpForClient(true, pulsar.getBrokerServiceUrlTls());
+        internalSetUpForNamespace();
+
+        Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName)
+                .subscriptionName("my-subscriber-name").subscribe();
+
+        Producer<byte[]> producer = pulsarClient.newProducer().topic(topicName)
+                .create();
+        for (int i = 0; i < 10; i++) {
+            byte[] message = new byte[MESSAGE_SIZE];
+            Arrays.fill(message, (byte) i);
+            producer.send(message);
+        }
+
+        Message<byte[]> msg = null;
+        for (int i = 0; i < 10; i++) {
+            msg = consumer.receive(5, TimeUnit.SECONDS);
+            byte[] expected = new byte[MESSAGE_SIZE];
+            Arrays.fill(expected, (byte) i);
+            Assert.assertEquals(expected, msg.getData());
+        }
+        // Acknowledge the consumption of all messages at once
+        consumer.acknowledgeCumulative(msg);
+        consumer.close();
+        log.info("-- Exiting {} test --", methodName);
+    }
+
+    @Test(timeOut = 300000)
+    public void testTlsClientAuthOverBinaryProtocol() throws Exception {
+        log.info("-- Starting {} test --", methodName);
+
+        final int MESSAGE_SIZE = 16 * 1024 + 1;
+        log.info("-- message size --", MESSAGE_SIZE);
+        String topicName = "persistent://my-property/use/my-ns/testTlsClientAuthOverBinaryProtocol"
+                           + System.currentTimeMillis();
+
+        internalSetUpForNamespace();
+
+        // Test 1 - Using TLS on binary protocol without sending certs - expect failure
+        internalSetUpForClient(false, pulsar.getBrokerServiceUrlTls());
+
+        try {
+            pulsarClient.newConsumer().topic(topicName)
+                    .subscriptionName("my-subscriber-name").subscriptionType(SubscriptionType.Exclusive).subscribe();
+            Assert.fail("Server should have failed the TLS handshake since client didn't .");
+        } catch (Exception ex) {
+            // OK
+        }
+
+        // Test 2 - Using TLS on binary protocol - sending certs
+        internalSetUpForClient(true, pulsar.getBrokerServiceUrlTls());
+        try {
+            pulsarClient.newConsumer().topic(topicName)
+                    .subscriptionName("my-subscriber-name").subscriptionType(SubscriptionType.Exclusive).subscribe();
+        } catch (Exception ex) {
+            Assert.fail("Should not fail since certs are sent.");
+        }
+    }
+
+    @Test(timeOut = 30000)
+    public void testTlsClientAuthOverHTTPProtocol() throws Exception {
+        log.info("-- Starting {} test --", methodName);
+
+        final int MESSAGE_SIZE = 16 * 1024 + 1;
+        log.info("-- message size --", MESSAGE_SIZE);
+        String topicName = "persistent://my-property/use/my-ns/testTlsClientAuthOverHTTPProtocol"
+                           + System.currentTimeMillis();
+
+        internalSetUpForNamespace();
+
+        // Test 1 - Using TLS on https without sending certs - expect failure
+        internalSetUpForClient(false, pulsar.getWebServiceAddressTls());
+        try {
+            pulsarClient.newConsumer().topic(topicName)
+                    .subscriptionName("my-subscriber-name").subscriptionType(SubscriptionType.Exclusive).subscribe();
+            Assert.fail("Server should have failed the TLS handshake since client didn't .");
+        } catch (Exception ex) {
+            // OK
+        }
+
+        // Test 2 - Using TLS on https - sending certs
+        internalSetUpForClient(true, pulsar.getWebServiceAddressTls());
+        try {
+            pulsarClient.newConsumer().topic(topicName)
+                    .subscriptionName("my-subscriber-name").subscriptionType(SubscriptionType.Exclusive).subscribe();
+        } catch (Exception ex) {
+            Assert.fail("Should not fail since certs are sent.");
+        }
+    }
+
+}
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsTest.java
new file mode 100644
index 0000000..0f9993d
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsTest.java
@@ -0,0 +1,80 @@
+/**
+ * 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.client.impl;
+
+import static org.apache.pulsar.common.util.SecurityUtility.getProvider;
+
+import java.security.Provider;
+import javax.net.ssl.SSLContext;
+import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext;
+import org.apache.pulsar.common.util.keystoretls.SSLContextValidatorEngine;
+import org.testng.annotations.Test;
+
+public class KeyStoreTlsTest {
+
+    protected final String BROKER_KEYSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/broker.keystore.jks";
+    protected final String BROKER_TRUSTSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/broker.truststore.jks";
+    protected final String BROKER_KEYSTORE_PW = "111111";
+    protected final String BROKER_TRUSTSTORE_PW = "111111";
+
+    protected final String CLIENT_KEYSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/client.keystore.jks";
+    protected final String CLIENT_TRUSTSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/client.truststore.jks";
+    protected final String CLIENT_KEYSTORE_PW = "111111";
+    protected final String CLIENT_TRUSTSTORE_PW = "111111";
+    protected final String KEYSTORE_TYPE = "JKS";
+
+    public static final Provider BC_PROVIDER = getProvider();
+
+    @Test(timeOut = 300000)
+    public void testValidate() throws Exception {
+        KeyStoreSSLContext serverSSLContext = new KeyStoreSSLContext(KeyStoreSSLContext.Mode.SERVER,
+                null,
+                KEYSTORE_TYPE,
+                BROKER_KEYSTORE_FILE_PATH,
+                BROKER_KEYSTORE_PW,
+                false,
+                KEYSTORE_TYPE,
+                BROKER_TRUSTSTORE_FILE_PATH,
+                BROKER_TRUSTSTORE_PW,
+                true,
+                null,
+                null);
+        SSLContext serverCnx = serverSSLContext.createSSLContext();
+
+        KeyStoreSSLContext clientSSLContext = new KeyStoreSSLContext(KeyStoreSSLContext.Mode.CLIENT,
+                null,
+                KEYSTORE_TYPE,
+                CLIENT_KEYSTORE_FILE_PATH,
+                CLIENT_KEYSTORE_PW,
+                false,
+                KEYSTORE_TYPE,
+                CLIENT_TRUSTSTORE_FILE_PATH,
+                CLIENT_TRUSTSTORE_PW,
+                false,
+                null,
+                null);
+        SSLContext clientCnx = clientSSLContext.createSSLContext();
+
+        SSLContextValidatorEngine.validate(clientCnx, serverCnx);
+    }
+}
diff --git a/pulsar-broker/src/test/resources/authentication/keystoretls/broker.keystore.jks b/pulsar-broker/src/test/resources/authentication/keystoretls/broker.keystore.jks
new file mode 100644
index 0000000..b4fec69
Binary files /dev/null and b/pulsar-broker/src/test/resources/authentication/keystoretls/broker.keystore.jks differ
diff --git a/pulsar-broker/src/test/resources/authentication/keystoretls/broker.truststore.jks b/pulsar-broker/src/test/resources/authentication/keystoretls/broker.truststore.jks
new file mode 100644
index 0000000..8ac03d8
Binary files /dev/null and b/pulsar-broker/src/test/resources/authentication/keystoretls/broker.truststore.jks differ
diff --git a/pulsar-broker/src/test/resources/authentication/keystoretls/brokerKeyStorePW.txt b/pulsar-broker/src/test/resources/authentication/keystoretls/brokerKeyStorePW.txt
new file mode 100644
index 0000000..90d2950
--- /dev/null
+++ b/pulsar-broker/src/test/resources/authentication/keystoretls/brokerKeyStorePW.txt
@@ -0,0 +1 @@
+111111
diff --git a/pulsar-broker/src/test/resources/authentication/keystoretls/brokerTrustStorePW.txt b/pulsar-broker/src/test/resources/authentication/keystoretls/brokerTrustStorePW.txt
new file mode 100644
index 0000000..90d2950
--- /dev/null
+++ b/pulsar-broker/src/test/resources/authentication/keystoretls/brokerTrustStorePW.txt
@@ -0,0 +1 @@
+111111
diff --git a/pulsar-broker/src/test/resources/authentication/keystoretls/client.keystore.jks b/pulsar-broker/src/test/resources/authentication/keystoretls/client.keystore.jks
new file mode 100644
index 0000000..499c8be
Binary files /dev/null and b/pulsar-broker/src/test/resources/authentication/keystoretls/client.keystore.jks differ
diff --git a/pulsar-broker/src/test/resources/authentication/keystoretls/client.truststore.jks b/pulsar-broker/src/test/resources/authentication/keystoretls/client.truststore.jks
new file mode 100644
index 0000000..8eaa06b
Binary files /dev/null and b/pulsar-broker/src/test/resources/authentication/keystoretls/client.truststore.jks differ
diff --git a/pulsar-broker/src/test/resources/authentication/keystoretls/clientKeyStorePW.txt b/pulsar-broker/src/test/resources/authentication/keystoretls/clientKeyStorePW.txt
new file mode 100644
index 0000000..90d2950
--- /dev/null
+++ b/pulsar-broker/src/test/resources/authentication/keystoretls/clientKeyStorePW.txt
@@ -0,0 +1 @@
+111111
diff --git a/pulsar-broker/src/test/resources/authentication/keystoretls/clientTrustStorePW.txt b/pulsar-broker/src/test/resources/authentication/keystoretls/clientTrustStorePW.txt
new file mode 100644
index 0000000..90d2950
--- /dev/null
+++ b/pulsar-broker/src/test/resources/authentication/keystoretls/clientTrustStorePW.txt
@@ -0,0 +1 @@
+111111
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java
index e4e04af..81b3ff6 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java
@@ -19,6 +19,7 @@
 package org.apache.pulsar.client.admin;
 
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.pulsar.client.api.Authentication;
@@ -172,7 +173,64 @@ public interface PulsarAdminBuilder {
     PulsarAdminBuilder enableTlsHostnameVerification(boolean enableTlsHostnameVerification);
 
     /**
-     * This sets the connection time out for the pulsar admin client
+     * If Tls is enabled, whether use KeyStore type as tls configuration parameter.
+     * False means use default pem type configuration.
+     *
+     * @param useKeyStoreTls
+     */
+    PulsarAdminBuilder useKeyStoreTls(boolean useKeyStoreTls);
+
+    /**
+     * The name of the security provider used for SSL connections.
+     * Default value is the default security provider of the JVM.
+     *
+     * @param sslProvider
+     */
+    PulsarAdminBuilder sslProvider(String sslProvider);
+
+    /**
+     * The file format of the trust store file.
+     *
+     * @param tlsTrustStoreType
+     */
+    PulsarAdminBuilder tlsTrustStoreType(String tlsTrustStoreType);
+
+    /**
+     * The location of the trust store file.
+     *
+     * @param tlsTrustStorePath
+     */
+    PulsarAdminBuilder tlsTrustStorePath(String tlsTrustStorePath);
+
+    /**
+     * The store password for the key store file.
+     *
+     * @param tlsTrustStorePassword
+     * @return the client builder instance
+     */
+    PulsarAdminBuilder tlsTrustStorePassword(String tlsTrustStorePassword);
+
+    /**
+     * A list of cipher suites.
+     * This is a named combination of authentication, encryption, MAC and key exchange algorithm
+     * used to negotiate the security settings for a network connection using TLS or SSL network protocol.
+     * By default all the available cipher suites are supported.
+     *
+     * @param tlsCiphers
+     */
+    PulsarAdminBuilder tlsCiphers(Set<String> tlsCiphers);
+
+    /**
+     * The SSL protocol used to generate the SSLContext.
+     * Default setting is TLS, which is fine for most cases.
+     * Allowed values in recent JVMs are TLS, TLSv1.1 and TLSv1.2. SSL, SSLv2.
+     *
+     * @param tlsProtocols
+     */
+    PulsarAdminBuilder tlsProtocols(Set<String> tlsProtocols);
+
+    /**
+     * This sets the connection time out for the pulsar admin client.
      *
      * @param connectionTimeout
      * @param connectionTimeoutUnit
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java
index bb5588b..d62ac33 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java
@@ -18,6 +18,10 @@
  */
 package org.apache.pulsar.client.admin.internal;
 
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.client.admin.PulsarAdminBuilder;
 import org.apache.pulsar.client.api.Authentication;
@@ -26,9 +30,6 @@ import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
 import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
 
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
 public class PulsarAdminBuilderImpl implements PulsarAdminBuilder {
 
     protected final ClientConfigurationData conf;
@@ -104,6 +105,48 @@ public class PulsarAdminBuilderImpl implements PulsarAdminBuilder {
     }
 
     @Override
+    public PulsarAdminBuilder useKeyStoreTls(boolean useKeyStoreTls) {
+        conf.setUseKeyStoreTls(useKeyStoreTls);
+        return this;
+    }
+
+    @Override
+    public PulsarAdminBuilder sslProvider(String sslProvider) {
+        conf.setSslProvider(sslProvider);
+        return this;
+    }
+
+    @Override
+    public PulsarAdminBuilder tlsTrustStoreType(String tlsTrustStoreType) {
+        conf.setTlsTrustStoreType(tlsTrustStoreType);
+        return this;
+    }
+
+    @Override
+    public PulsarAdminBuilder tlsTrustStorePath(String tlsTrustStorePath) {
+        conf.setTlsTrustStorePath(tlsTrustStorePath);
+        return this;
+    }
+
+    @Override
+    public PulsarAdminBuilder tlsTrustStorePassword(String tlsTrustStorePassword) {
+        conf.setTlsTrustStorePassword(tlsTrustStorePassword);
+        return this;
+    }
+
+    @Override
+    public PulsarAdminBuilder tlsCiphers(Set<String> tlsCiphers) {
+        conf.setTlsCiphers(tlsCiphers);
+        return this;
+    }
+
+    @Override
+    public PulsarAdminBuilder tlsProtocols(Set<String> tlsProtocols) {
+        conf.setTlsProtocols(tlsProtocols);
+        return this;
+    }
+
+    @Override
     public PulsarAdminBuilder connectionTimeout(int connectionTimeout, TimeUnit connectionTimeoutUnit) {
         this.connectTimeout = connectionTimeout;
         this.connectTimeoutUnit = connectionTimeoutUnit;
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 c90bf60..42e9c49 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
@@ -34,7 +34,9 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+
 import javax.ws.rs.ProcessingException;
+import javax.net.ssl.SSLContext;
 import javax.ws.rs.client.Client;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.Response.Status;
@@ -47,10 +49,12 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.pulsar.PulsarVersion;
 import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.client.api.AuthenticationDataProvider;
+import org.apache.pulsar.client.api.KeyStoreParams;
 import org.apache.pulsar.client.impl.PulsarServiceNameResolver;
 import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
 import org.apache.pulsar.common.util.SecurityUtility;
 import org.asynchttpclient.AsyncCompletionHandler;
+import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext;
 import org.asynchttpclient.AsyncHttpClient;
 import org.asynchttpclient.BoundRequestBuilder;
 import org.asynchttpclient.DefaultAsyncHttpClient;
@@ -58,6 +62,7 @@ import org.asynchttpclient.DefaultAsyncHttpClientConfig;
 import org.asynchttpclient.Request;
 import org.asynchttpclient.Response;
 import org.asynchttpclient.channel.DefaultKeepAliveStrategy;
+import org.asynchttpclient.netty.ssl.JsseSslEngineFactory;
 import org.glassfish.jersey.client.ClientProperties;
 import org.glassfish.jersey.client.ClientRequest;
 import org.glassfish.jersey.client.ClientResponse;
@@ -100,20 +105,41 @@ public class AsyncHttpConnector implements Connector {
         if (conf != null && StringUtils.isNotBlank(conf.getServiceUrl())) {
             serviceNameResolver.updateServiceUrl(conf.getServiceUrl());
             if (conf.getServiceUrl().startsWith("https://")) {
-
-                SslContext sslCtx = null;
-
                 // Set client key and certificate if available
                 AuthenticationDataProvider authData = conf.getAuthentication().getAuthData();
-                if (authData.hasDataForTls()) {
-                    sslCtx = SecurityUtility.createNettySslContextForClient(conf.isTlsAllowInsecureConnection() || !conf.isTlsHostnameVerificationEnable(),
-                                                                            conf.getTlsTrustCertsFilePath(), authData.getTlsCertificates(), authData.getTlsPrivateKey());
+
+                if (conf.isUseKeyStoreTls()) {
+                    KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : null;
+
+                    final SSLContext sslCtx = KeyStoreSSLContext.createClientSslContext(
+                            conf.getSslProvider(),
+                            params != null ? params.getKeyStoreType() : null,
+                            params != null ? params.getKeyStorePath() : null,
+                            params != null ? params.getKeyStorePassword() : null,
+                            conf.isTlsAllowInsecureConnection() || !conf.isTlsHostnameVerificationEnable(),
+                            conf.getTlsTrustStoreType(),
+                            conf.getTlsTrustStorePath(),
+                            conf.getTlsTrustStorePassword(),
+                            conf.getTlsCiphers(),
+                            conf.getTlsProtocols());
+
+                    JsseSslEngineFactory sslEngineFactory = new JsseSslEngineFactory(sslCtx);
+                    confBuilder.setSslEngineFactory(sslEngineFactory);
                 } else {
-                    sslCtx = SecurityUtility.createNettySslContextForClient(conf.isTlsAllowInsecureConnection() || !conf.isTlsHostnameVerificationEnable(),
-                                                                            conf.getTlsTrustCertsFilePath());
+                    SslContext sslCtx = null;
+                    if (authData.hasDataForTls()) {
+                        sslCtx = SecurityUtility.createNettySslContextForClient(
+                                conf.isTlsAllowInsecureConnection() || !conf.isTlsHostnameVerificationEnable(),
+                                conf.getTlsTrustCertsFilePath(),
+                                authData.getTlsCertificates(),
+                                authData.getTlsPrivateKey());
+                    } else {
+                        sslCtx = SecurityUtility.createNettySslContextForClient(
+                                conf.isTlsAllowInsecureConnection() || !conf.isTlsHostnameVerificationEnable(),
+                                conf.getTlsTrustCertsFilePath());
+                    }
+                    confBuilder.setSslContext(sslCtx);
                 }
-
-                confBuilder.setSslContext(sslCtx);
             }
         }
         httpClient = new DefaultAsyncHttpClient(confBuilder.build());
diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AuthenticationDataProvider.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AuthenticationDataProvider.java
index 122dd5b..77eafe5 100644
--- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AuthenticationDataProvider.java
+++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AuthenticationDataProvider.java
@@ -63,6 +63,15 @@ public interface AuthenticationDataProvider extends Serializable {
         return null;
     }
 
+    /**
+     * Used for TLS authentication with keystore type.
+     *
+     * @return a KeyStoreParams for the client certificate chain, or null if the data are not available
+     */
+    default KeyStoreParams getTlsKeyStoreParams() {
+        return null;
+    }
+
     /*
      * HTTP
      */
diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java
index addedaa..e84f8ba 100644
--- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java
+++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java
@@ -20,6 +20,7 @@ package org.apache.pulsar.client.api;
 
 import java.time.Clock;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import org.apache.pulsar.client.api.PulsarClientException.UnsupportedAuthenticationException;
 
@@ -290,6 +291,69 @@ public interface ClientBuilder extends Cloneable {
     ClientBuilder enableTlsHostnameVerification(boolean enableTlsHostnameVerification);
 
     /**
+     * If Tls is enabled, whether use KeyStore type as tls configuration parameter.
+     * False means use default pem type configuration.
+     *
+     * @param useKeyStoreTls
+     * @return the client builder instance
+     */
+    ClientBuilder useKeyStoreTls(boolean useKeyStoreTls);
+
+    /**
+     * The name of the security provider used for SSL connections.
+     * Default value is the default security provider of the JVM.
+     *
+     * @param sslProvider
+     * @return the client builder instance
+     */
+    ClientBuilder sslProvider(String sslProvider);
+
+    /**
+     * The file format of the trust store file.
+     *
+     * @param tlsTrustStoreType
+     * @return the client builder instance
+     */
+    ClientBuilder tlsTrustStoreType(String tlsTrustStoreType);
+
+    /**
+     * The location of the trust store file.
+     *
+     * @param tlsTrustStorePath
+     * @return the client builder instance
+     */
+    ClientBuilder tlsTrustStorePath(String tlsTrustStorePath);
+
+    /**
+     * The store password for the key store file.
+     *
+     * @param tlsTrustStorePassword
+     * @return the client builder instance
+     */
+    ClientBuilder tlsTrustStorePassword(String tlsTrustStorePassword);
+
+    /**
+     * A list of cipher suites.
+     * This is a named combination of authentication, encryption, MAC and key exchange algorithm
+     * used to negotiate the security settings for a network connection using TLS or SSL network protocol.
+     * By default all the available cipher suites are supported.
+     *
+     * @param tlsCiphers
+     * @return the client builder instance
+     */
+    ClientBuilder tlsCiphers(Set<String> tlsCiphers);
+
+    /**
+     * The SSL protocol used to generate the SSLContext.
+     * Default setting is TLS, which is fine for most cases.
+     * Allowed values in recent JVMs are TLS, TLSv1.1 and TLSv1.2. SSL, SSLv2.
+     *
+     * @param tlsProtocols
+     * @return the client builder instance
+     */
+    ClientBuilder tlsProtocols(Set<String> tlsProtocols);
+
+    /**
      * Set the interval between each stat info <i>(default: 60 seconds)</i> Stats will be activated with positive
      * statsInterval It should be set to at least 1 second.
      *
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/KeyStoreParams.java
similarity index 54%
copy from pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java
copy to pulsar-client-api/src/main/java/org/apache/pulsar/client/api/KeyStoreParams.java
index 03a9bd3..5759801 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java
+++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/KeyStoreParams.java
@@ -16,33 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pulsar.broker.authentication;
+package org.apache.pulsar.client.api;
 
-import java.security.cert.X509Certificate;
-
-import javax.servlet.http.HttpServletRequest;
-
-public class AuthenticationDataHttps extends AuthenticationDataHttp {
-
-    protected final X509Certificate[] certificates;
-
-    public AuthenticationDataHttps(HttpServletRequest request) {
-        super(request);
-        certificates = (X509Certificate[]) request.getAttribute("javax.servlet.request.X509Certificate");
-    }
-
-    /*
-     * TLS
-     */
-
-    @Override
-    public boolean hasDataFromTls() {
-        return (certificates != null);
-    }
-
-    @Override
-    public X509Certificate[] getTlsCertificates() {
-        return certificates;
-    }
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
 
+/**
+ * KeyStore parameters used for tls authentication.
+ */
+@Data
+@Builder
+@AllArgsConstructor
+public class KeyStoreParams{
+    private String keyStoreType;
+    private String keyStorePath;
+    private String keyStorePassword;
 }
diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/PulsarAdminTool.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/PulsarAdminTool.java
index 73f4c8d..6391697 100644
--- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/PulsarAdminTool.java
+++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/PulsarAdminTool.java
@@ -53,16 +53,22 @@ public class PulsarAdminTool {
 
     @Parameter(names = { "--tls-allow-insecure" }, description = "Allow TLS insecure connection")
     Boolean tlsAllowInsecureConnection;
-    
+
     @Parameter(names = { "--tls-trust-cert-path" }, description = "Allow TLS trust cert file path")
     String tlsTrustCertsFilePath;
-    
+
     @Parameter(names = { "--tls-enable-hostname-verification" }, description = "Enable TLS common name verification")
     Boolean tlsEnableHostnameVerification;
 
     @Parameter(names = { "-h", "--help", }, help = true, description = "Show this help.")
     boolean help;
 
+    // for tls with keystore type config
+    boolean useKeyStoreTls = false;
+    String tlsTrustStoreType = "JKS";
+    String tlsTrustStorePath = null;
+    String tlsTrustStorePassword = null;
+
     PulsarAdminTool(Properties properties) throws Exception {
         // fallback to previous-version serviceUrl property to maintain backward-compatibility
         serviceUrl = StringUtils.isNotBlank(properties.getProperty("webServiceUrl"))
@@ -80,9 +86,19 @@ public class PulsarAdminTool {
                 ? this.tlsTrustCertsFilePath
                 : properties.getProperty("tlsTrustCertsFilePath");
 
+        this.useKeyStoreTls = Boolean
+                .parseBoolean(properties.getProperty("useKeyStoreTls", "false"));
+        this.tlsTrustStoreType = properties.getProperty("tlsTrustStoreType", "JKS");
+        this.tlsTrustStorePath = properties.getProperty("tlsTrustStorePath");
+        this.tlsTrustStorePassword = properties.getProperty("tlsTrustStorePassword");
+
         adminBuilder = PulsarAdmin.builder().allowTlsInsecureConnection(tlsAllowInsecureConnection)
                 .enableTlsHostnameVerification(tlsEnableHostnameVerification)
-                .tlsTrustCertsFilePath(tlsTrustCertsFilePath);
+                .tlsTrustCertsFilePath(tlsTrustCertsFilePath)
+                .useKeyStoreTls(useKeyStoreTls)
+                .tlsTrustStoreType(tlsTrustStoreType)
+                .tlsTrustStorePath(tlsTrustStorePath)
+                .tlsTrustStorePassword(tlsTrustStorePassword);
 
         jcommander = new JCommander();
         jcommander.setProgramName("pulsar-admin");
diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/PulsarClientTool.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/PulsarClientTool.java
index 2c38b34..b86bc79 100644
--- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/PulsarClientTool.java
+++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/PulsarClientTool.java
@@ -60,6 +60,12 @@ public class PulsarClientTool {
     boolean tlsEnableHostnameVerification = false;
     String tlsTrustCertsFilePath = null;
 
+    // for tls with keystore type config
+    boolean useKeyStoreTls = false;
+    String tlsTrustStoreType = "JKS";
+    String tlsTrustStorePath = null;
+    String tlsTrustStorePassword = null;
+
     JCommander commandParser;
     CmdProduce produceCommand;
     CmdConsume consumeCommand;
@@ -79,6 +85,12 @@ public class PulsarClientTool {
                 .parseBoolean(properties.getProperty("tlsEnableHostnameVerification", "false"));
         this.tlsTrustCertsFilePath = properties.getProperty("tlsTrustCertsFilePath");
 
+        this.useKeyStoreTls = Boolean
+                .parseBoolean(properties.getProperty("useKeyStoreTls", "false"));
+        this.tlsTrustStoreType = properties.getProperty("tlsTrustStoreType", "JKS");
+        this.tlsTrustStorePath = properties.getProperty("tlsTrustStorePath");
+        this.tlsTrustStorePassword = properties.getProperty("tlsTrustStorePassword");
+
         produceCommand = new CmdProduce();
         consumeCommand = new CmdConsume();
 
@@ -99,6 +111,12 @@ public class PulsarClientTool {
         clientBuilder.allowTlsInsecureConnection(this.tlsAllowInsecureConnection);
         clientBuilder.tlsTrustCertsFilePath(this.tlsTrustCertsFilePath);
         clientBuilder.serviceUrl(serviceURL);
+
+        clientBuilder.useKeyStoreTls(useKeyStoreTls)
+                .tlsTrustStoreType(tlsTrustStoreType)
+                .tlsTrustStorePath(tlsTrustStorePath)
+                .tlsTrustStorePassword(tlsTrustStorePassword);
+
         this.produceCommand.updateConfig(clientBuilder, authentication, this.serviceURL);
         this.consumeCommand.updateConfig(clientBuilder, authentication, this.serviceURL);
     }
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java
index 16ea688..3283166 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java
@@ -20,6 +20,7 @@ package org.apache.pulsar.client.impl;
 
 import java.time.Clock;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.lang3.StringUtils;
@@ -174,6 +175,48 @@ public class ClientBuilderImpl implements ClientBuilder {
     }
 
     @Override
+    public ClientBuilder useKeyStoreTls(boolean useKeyStoreTls) {
+        conf.setUseKeyStoreTls(useKeyStoreTls);
+        return this;
+    }
+
+    @Override
+    public ClientBuilder sslProvider(String sslProvider) {
+        conf.setSslProvider(sslProvider);
+        return this;
+    }
+
+    @Override
+    public ClientBuilder tlsTrustStoreType(String tlsTrustStoreType) {
+        conf.setTlsTrustStoreType(tlsTrustStoreType);
+        return this;
+    }
+
+    @Override
+    public ClientBuilder tlsTrustStorePath(String tlsTrustStorePath) {
+        conf.setTlsTrustStorePath(tlsTrustStorePath);
+        return this;
+    }
+
+    @Override
+    public ClientBuilder tlsTrustStorePassword(String tlsTrustStorePassword) {
+        conf.setTlsTrustStorePassword(tlsTrustStorePassword);
+        return this;
+    }
+
+    @Override
+    public ClientBuilder tlsCiphers(Set<String> tlsCiphers) {
+        conf.setTlsCiphers(tlsCiphers);
+        return this;
+    }
+
+    @Override
+    public ClientBuilder tlsProtocols(Set<String> tlsProtocols) {
+        conf.setTlsProtocols(tlsProtocols);
+        return this;
+    }
+
+    @Override
     public ClientBuilder statsInterval(long statsInterval, TimeUnit unit) {
         conf.setStatsIntervalSeconds(unit.toSeconds(statsInterval));
         return this;
@@ -214,13 +257,13 @@ public class ClientBuilderImpl implements ClientBuilder {
     	conf.setInitialBackoffIntervalNanos(unit.toNanos(duration));
     	return this;
     }
-    
+
     @Override
     public ClientBuilder maxBackoffInterval(long duration, TimeUnit unit) {
     	conf.setMaxBackoffIntervalNanos(unit.toNanos(duration));
     	return this;
     }
-    
+
     public ClientConfigurationData getClientConfigurationData() {
         return conf;
     }
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 845c741..3e693ad 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
@@ -32,14 +32,18 @@ import io.netty.channel.EventLoopGroup;
 import io.netty.handler.codec.http.HttpRequest;
 import io.netty.handler.codec.http.HttpResponse;
 import io.netty.handler.ssl.SslContext;
+import javax.net.ssl.SSLContext;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.pulsar.PulsarVersion;
 import org.apache.pulsar.client.api.Authentication;
 import org.apache.pulsar.client.api.AuthenticationDataProvider;
+import org.apache.pulsar.client.api.KeyStoreParams;
 import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.client.api.PulsarClientException.NotFoundException;
+import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
 import org.apache.pulsar.common.util.ObjectMapperFactory;
 import org.apache.pulsar.common.util.SecurityUtility;
+import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext;
 import org.asynchttpclient.AsyncHttpClient;
 import org.asynchttpclient.AsyncHttpClientConfig;
 import org.asynchttpclient.BoundRequestBuilder;
@@ -47,6 +51,7 @@ import org.asynchttpclient.DefaultAsyncHttpClient;
 import org.asynchttpclient.DefaultAsyncHttpClientConfig;
 import org.asynchttpclient.Request;
 import org.asynchttpclient.channel.DefaultKeepAliveStrategy;
+import org.asynchttpclient.netty.ssl.JsseSslEngineFactory;
 
 
 @Slf4j
@@ -59,24 +64,15 @@ public class HttpClient implements Closeable {
     protected final ServiceNameResolver serviceNameResolver;
     protected final Authentication authentication;
 
-    protected HttpClient(String serviceUrl, Authentication authentication,
-            EventLoopGroup eventLoopGroup, boolean tlsAllowInsecureConnection, String tlsTrustCertsFilePath)
-            throws PulsarClientException {
-        this(serviceUrl, authentication, eventLoopGroup, tlsAllowInsecureConnection,
-                tlsTrustCertsFilePath, DEFAULT_CONNECT_TIMEOUT_IN_SECONDS, DEFAULT_READ_TIMEOUT_IN_SECONDS);
-    }
-
-    protected HttpClient(String serviceUrl, Authentication authentication,
-            EventLoopGroup eventLoopGroup, boolean tlsAllowInsecureConnection, String tlsTrustCertsFilePath,
-            int connectTimeoutInSeconds, int readTimeoutInSeconds) throws PulsarClientException {
-        this.authentication = authentication;
+    protected HttpClient(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) throws PulsarClientException {
+        this.authentication = conf.getAuthentication();
         this.serviceNameResolver = new PulsarServiceNameResolver();
-        this.serviceNameResolver.updateServiceUrl(serviceUrl);
+        this.serviceNameResolver.updateServiceUrl(conf.getServiceUrl());
 
         DefaultAsyncHttpClientConfig.Builder confBuilder = new DefaultAsyncHttpClientConfig.Builder();
         confBuilder.setFollowRedirect(true);
-        confBuilder.setConnectTimeout(connectTimeoutInSeconds * 1000);
-        confBuilder.setReadTimeout(readTimeoutInSeconds * 1000);
+        confBuilder.setConnectTimeout(DEFAULT_CONNECT_TIMEOUT_IN_SECONDS * 1000);
+        confBuilder.setReadTimeout(DEFAULT_READ_TIMEOUT_IN_SECONDS * 1000);
         confBuilder.setUserAgent(String.format("Pulsar-Java-v%s", PulsarVersion.getVersion()));
         confBuilder.setKeepAliveStrategy(new DefaultKeepAliveStrategy() {
             @Override
@@ -88,19 +84,45 @@ public class HttpClient implements Closeable {
 
         if ("https".equals(serviceNameResolver.getServiceUri().getServiceName())) {
             try {
-                SslContext sslCtx = null;
-
                 // Set client key and certificate if available
                 AuthenticationDataProvider authData = authentication.getAuthData();
-                if (authData.hasDataForTls()) {
-                    sslCtx = SecurityUtility.createNettySslContextForClient(tlsAllowInsecureConnection, tlsTrustCertsFilePath,
-                            authData.getTlsCertificates(), authData.getTlsPrivateKey());
+
+                if (conf.isUseKeyStoreTls()) {
+                    SSLContext sslCtx = null;
+                    KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : null;
+
+                    sslCtx = KeyStoreSSLContext.createClientSslContext(
+                            conf.getSslProvider(),
+                            params != null ? params.getKeyStoreType() : null,
+                            params != null ? params.getKeyStorePath() : null,
+                            params != null ? params.getKeyStorePassword() : null,
+                            conf.isTlsAllowInsecureConnection(),
+                            conf.getTlsTrustStoreType(),
+                            conf.getTlsTrustStorePath(),
+                            conf.getTlsTrustStorePassword(),
+                            conf.getTlsCiphers(),
+                            conf.getTlsProtocols());
+
+                    JsseSslEngineFactory sslEngineFactory = new JsseSslEngineFactory(sslCtx);
+                    confBuilder.setSslEngineFactory(sslEngineFactory);
                 } else {
-                    sslCtx = SecurityUtility.createNettySslContextForClient(tlsAllowInsecureConnection, tlsTrustCertsFilePath);
+                    SslContext sslCtx = null;
+                    if (authData.hasDataForTls()) {
+                        sslCtx = SecurityUtility.createNettySslContextForClient(
+                                conf.isTlsAllowInsecureConnection(),
+                                conf.getTlsTrustCertsFilePath(),
+                                authData.getTlsCertificates(),
+                                authData.getTlsPrivateKey());
+                    }
+                    else {
+                        sslCtx = SecurityUtility.createNettySslContextForClient(
+                                conf.isTlsAllowInsecureConnection(),
+                                conf.getTlsTrustCertsFilePath());
+                    }
+                    confBuilder.setSslContext(sslCtx);
                 }
 
-                confBuilder.setSslContext(sslCtx);
-                confBuilder.setUseInsecureTrustManager(tlsAllowInsecureConnection);
+                confBuilder.setUseInsecureTrustManager(conf.isTlsAllowInsecureConnection());
             } catch (Exception e) {
                 throw new PulsarClientException.InvalidConfigurationException(e);
             }
@@ -109,7 +131,7 @@ public class HttpClient implements Closeable {
         AsyncHttpClientConfig config = confBuilder.build();
         httpClient = new DefaultAsyncHttpClient(config);
 
-        log.debug("Using HTTP url: {}", serviceUrl);
+        log.debug("Using HTTP url: {}", conf.getServiceUrl());
     }
 
     String getServiceUrl() {
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java
index 5bc3bcd..3451ebc 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java
@@ -57,8 +57,7 @@ public class HttpLookupService implements LookupService {
 
     public HttpLookupService(ClientConfigurationData conf, EventLoopGroup eventLoopGroup)
             throws PulsarClientException {
-        this.httpClient = new HttpClient(conf.getServiceUrl(), conf.getAuthentication(),
-                eventLoopGroup, conf.isTlsAllowInsecureConnection(), conf.getTlsTrustCertsFilePath());
+        this.httpClient = new HttpClient(conf, eventLoopGroup);
         this.useTls = conf.isUseTls();
     }
 
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java
index a932253..4a145e7 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java
@@ -18,30 +18,34 @@
  */
 package org.apache.pulsar.client.impl;
 
-import java.security.cert.X509Certificate;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Supplier;
-
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
 import io.netty.handler.ssl.SslContext;
-
+import io.netty.handler.ssl.SslHandler;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
 import org.apache.pulsar.client.api.AuthenticationDataProvider;
 import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
 import org.apache.pulsar.client.util.ObjectCache;
 import org.apache.pulsar.common.protocol.ByteBufPair;
 import org.apache.pulsar.common.protocol.Commands;
 import org.apache.pulsar.common.util.SecurityUtility;
+import org.apache.pulsar.common.util.keystoretls.NettySSLContextAutoRefreshBuilder;
 
+@Slf4j
 public class PulsarChannelInitializer extends ChannelInitializer<SocketChannel> {
 
     public static final String TLS_HANDLER = "tls";
 
     private final Supplier<ClientCnx> clientCnxSupplier;
     private final boolean tlsEnabled;
+    private final boolean tlsEnabledWithKeyStore;
 
     private final Supplier<SslContext> sslContextSupplier;
+    private NettySSLContextAutoRefreshBuilder nettySSLContextAutoRefreshBuilder;
 
     private static final long TLS_CERTIFICATE_CACHE_MILLIS = TimeUnit.MINUTES.toMillis(1);
 
@@ -50,8 +54,24 @@ public class PulsarChannelInitializer extends ChannelInitializer<SocketChannel>
         super();
         this.clientCnxSupplier = clientCnxSupplier;
         this.tlsEnabled = conf.isUseTls();
+        this.tlsEnabledWithKeyStore = conf.isUseKeyStoreTls();
+
+        if (tlsEnabled) {
+            if (tlsEnabledWithKeyStore) {
+                AuthenticationDataProvider authData1 = conf.getAuthentication().getAuthData();
+
+                nettySSLContextAutoRefreshBuilder = new NettySSLContextAutoRefreshBuilder(
+                            conf.getSslProvider(),
+                            conf.isTlsAllowInsecureConnection(),
+                            conf.getTlsTrustStoreType(),
+                            conf.getTlsTrustStorePath(),
+                            conf.getTlsTrustStorePassword(),
+                            conf.getTlsCiphers(),
+                            conf.getTlsProtocols(),
+                            TLS_CERTIFICATE_CACHE_MILLIS,
+                            authData1);
+            }
 
-        if (conf.isUseTls()) {
             sslContextSupplier = new ObjectCache<SslContext>(() -> {
                 try {
                     // Set client certificate if available
@@ -76,7 +96,12 @@ public class PulsarChannelInitializer extends ChannelInitializer<SocketChannel>
     @Override
     public void initChannel(SocketChannel ch) throws Exception {
         if (tlsEnabled) {
-            ch.pipeline().addLast(TLS_HANDLER, sslContextSupplier.get().newHandler(ch.alloc()));
+            if (tlsEnabledWithKeyStore) {
+                ch.pipeline().addLast(TLS_HANDLER,
+                        new SslHandler(nettySSLContextAutoRefreshBuilder.get().createSSLEngine()));
+            } else {
+                ch.pipeline().addLast(TLS_HANDLER, sslContextSupplier.get().newHandler(ch.alloc()));
+            }
             ch.pipeline().addLast("ByteBufPairEncoder", ByteBufPair.COPYING_ENCODER);
         } else {
             ch.pipeline().addLast("ByteBufPairEncoder", ByteBufPair.ENCODER);
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationDataKeyStoreTls.java
similarity index 56%
copy from pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java
copy to pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationDataKeyStoreTls.java
index 03a9bd3..6d78004 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationDataKeyStoreTls.java
@@ -16,33 +16,35 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pulsar.broker.authentication;
+package org.apache.pulsar.client.impl.auth;
 
-import java.security.cert.X509Certificate;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.AuthenticationDataProvider;
+import org.apache.pulsar.client.api.KeyStoreParams;
 
-import javax.servlet.http.HttpServletRequest;
+@Slf4j
+public class AuthenticationDataKeyStoreTls implements AuthenticationDataProvider {
+    private final KeyStoreParams keyStoreParams;
 
-public class AuthenticationDataHttps extends AuthenticationDataHttp {
-
-    protected final X509Certificate[] certificates;
-
-    public AuthenticationDataHttps(HttpServletRequest request) {
-        super(request);
-        certificates = (X509Certificate[]) request.getAttribute("javax.servlet.request.X509Certificate");
+    public AuthenticationDataKeyStoreTls(KeyStoreParams keyStoreParams) throws Exception {
+        this.keyStoreParams = keyStoreParams;
     }
 
     /*
      * TLS
      */
-
     @Override
-    public boolean hasDataFromTls() {
-        return (certificates != null);
+    public boolean hasDataForTls() {
+        return true;
     }
 
     @Override
-    public X509Certificate[] getTlsCertificates() {
-        return certificates;
+    public KeyStoreParams getTlsKeyStoreParams() {
+        return this.keyStoreParams;
     }
 
+    @Override
+    public String getCommandData() {
+        return null;
+    }
 }
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationKeyStoreTls.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationKeyStoreTls.java
new file mode 100644
index 0000000..e8c7764
--- /dev/null
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationKeyStoreTls.java
@@ -0,0 +1,136 @@
+/**
+ * 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.client.impl.auth;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Strings;
+import java.io.IOException;
+import java.util.Map;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.Authentication;
+import org.apache.pulsar.client.api.AuthenticationDataProvider;
+import org.apache.pulsar.client.api.EncodedAuthenticationParameterSupport;
+import org.apache.pulsar.client.api.KeyStoreParams;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.impl.AuthenticationUtil;
+
+/**
+ * This plugin requires these parameters: keyStoreType, keyStorePath, and  keyStorePassword.
+ * This parameter will construct a AuthenticationDataProvider
+ */
+@Slf4j
+public class AuthenticationKeyStoreTls implements Authentication, EncodedAuthenticationParameterSupport {
+    private static final long serialVersionUID = 1L;
+
+    private final static String AUTH_NAME = "tls";
+
+    // parameter name
+    public final static String KEYSTORE_TYPE = "keyStoreType";
+    public final static String KEYSTORE_PATH= "keyStorePath";
+    public final static String KEYSTORE_PW = "keyStorePassword";
+    private final static String DEFAULT_KEYSTORE_TYPE = "JKS";
+
+    private KeyStoreParams keyStoreParams;
+
+    public AuthenticationKeyStoreTls() {
+    }
+
+    public AuthenticationKeyStoreTls(String keyStoreType, String keyStorePath, String keyStorePassword) {
+        this.keyStoreParams = KeyStoreParams.builder()
+                .keyStoreType(keyStoreType)
+                .keyStorePath(keyStorePath)
+                .keyStorePassword(keyStorePassword)
+                .build();
+    }
+
+    @Override
+    public void close() throws IOException {
+        // noop
+    }
+
+    @Override
+    public String getAuthMethodName() {
+        return AUTH_NAME;
+    }
+
+    @Override
+    public AuthenticationDataProvider getAuthData() throws PulsarClientException {
+        try {
+            return new AuthenticationDataKeyStoreTls(this.keyStoreParams);
+        } catch (Exception e) {
+            throw new PulsarClientException(e);
+        }
+    }
+
+    // passed in KEYSTORE_TYPE/KEYSTORE_PATH/KEYSTORE_PW to construct parameters.
+    // e.g. {"keyStoreType":"JKS","keyStorePath":"/path/to/keystorefile","keyStorePassword":"keystorepw"}
+    //  or: "keyStoreType":"JKS","keyStorePath":"/path/to/keystorefile","keyStorePassword":"keystorepw"
+    @Override
+    public void configure(String paramsString) {
+        Map<String, String> params = null;
+        try {
+            params = AuthenticationUtil.configureFromJsonString(paramsString);
+        } catch (Exception e) {
+            // auth-param is not in json format
+            log.info("parameter not in Json format: {}", paramsString);
+        }
+
+        // in ":" "," format.
+        params = (params == null || params.isEmpty())
+                ? AuthenticationUtil.configureFromPulsar1AuthParamString(paramsString)
+                : params;
+
+        configure(params);
+    }
+
+    @Override
+    public void configure(Map<String, String> params) {
+        String keyStoreType = params.get(KEYSTORE_TYPE);
+        String keyStorePath = params.get(KEYSTORE_PATH);
+        String keyStorePassword = params.get(KEYSTORE_PW);
+
+        if (Strings.isNullOrEmpty(keyStorePath)
+            || Strings.isNullOrEmpty(keyStorePassword)) {
+            throw new IllegalArgumentException("Passed in parameter empty. "
+                                               + KEYSTORE_PATH + ": " + keyStorePath
+                                               + " " + KEYSTORE_PW + ": " + keyStorePassword);
+        }
+
+        if (Strings.isNullOrEmpty(keyStoreType)) {
+            keyStoreType = DEFAULT_KEYSTORE_TYPE;
+        }
+
+        this.keyStoreParams = KeyStoreParams.builder()
+                .keyStoreType(keyStoreType)
+                .keyStorePath(keyStorePath)
+                .keyStorePassword(keyStorePassword)
+                .build();
+    }
+
+    @Override
+    public void start() throws PulsarClientException {
+        // noop
+    }
+
+    // return strings like : "key1":"value1", "key2":"value2", ...
+    public static String mapToString(Map<String, String> map) {
+        return Joiner.on(',').withKeyValueSeparator(':').join(map);
+    }
+}
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationTls.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationTls.java
index 22cd2f5..d899146 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationTls.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationTls.java
@@ -40,7 +40,7 @@ import com.google.common.annotations.VisibleForTesting;
  *
  */
 public class AuthenticationTls implements Authentication, EncodedAuthenticationParameterSupport {
-
+    private final static String AUTH_NAME = "tls";
     private static final long serialVersionUID = 1L;
 
     private String certFilePath;
@@ -67,7 +67,7 @@ public class AuthenticationTls implements Authentication, EncodedAuthenticationP
 
     @Override
     public String getAuthMethodName() {
-        return "tls";
+        return AUTH_NAME;
     }
 
     @Override
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java
index af6ad8d..0cf496a 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java
@@ -19,7 +19,9 @@
 package org.apache.pulsar.client.impl.conf;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.google.common.collect.Sets;
 import java.time.Clock;
+import java.util.Set;
 import lombok.AllArgsConstructor;
 import lombok.Data;
 import lombok.NoArgsConstructor;
@@ -70,6 +72,16 @@ public class ClientConfigurationData implements Serializable, Cloneable {
     private long initialBackoffIntervalNanos = TimeUnit.MILLISECONDS.toNanos(100);
     private long maxBackoffIntervalNanos = TimeUnit.SECONDS.toNanos(60);
 
+    // set TLS using KeyStore way.
+    private boolean useKeyStoreTls = false;
+    private String sslProvider = null;
+    // needed when client auth is required
+    private String tlsTrustStoreType = "JKS";
+    private String tlsTrustStorePath = null;
+    private String tlsTrustStorePassword = null;
+    private Set<String> tlsCiphers = Sets.newTreeSet();
+    private Set<String> tlsProtocols = Sets.newTreeSet();
+
     @JsonIgnore
     private Clock clock = Clock.systemDefaultZone();
 
diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml
index 3c74a4e..6974b0c 100644
--- a/pulsar-common/pom.xml
+++ b/pulsar-common/pom.xml
@@ -146,6 +146,10 @@
       <artifactId>jackson-dataformat-yaml</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
   </dependencies>
 
   <build>
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/ClientSslContextRefresher.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/ClientSslContextRefresher.java
deleted file mode 100644
index 48ac937..0000000
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/ClientSslContextRefresher.java
+++ /dev/null
@@ -1,67 +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.common.util;
-
-import io.netty.handler.ssl.SslContext;
-import java.io.IOException;
-import java.security.GeneralSecurityException;
-import java.security.cert.X509Certificate;
-import org.apache.pulsar.client.api.AuthenticationDataProvider;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@SuppressWarnings("checkstyle:JavadocType")
-public class ClientSslContextRefresher {
-    private volatile SslContext sslContext;
-    private boolean tlsAllowInsecureConnection;
-    private String tlsTrustCertsFilePath;
-    private AuthenticationDataProvider authData;
-
-    public ClientSslContextRefresher(boolean allowInsecure, String trustCertsFilePath,
-            AuthenticationDataProvider authData) throws IOException, GeneralSecurityException {
-        this.tlsAllowInsecureConnection = allowInsecure;
-        this.tlsTrustCertsFilePath = trustCertsFilePath;
-        this.authData = authData;
-
-        if (authData != null && authData.hasDataForTls()) {
-            this.sslContext = SecurityUtility.createNettySslContextForClient(this.tlsAllowInsecureConnection,
-                    this.tlsTrustCertsFilePath, (X509Certificate[]) authData.getTlsCertificates(),
-                    authData.getTlsPrivateKey());
-        } else {
-            this.sslContext = SecurityUtility.createNettySslContextForClient(this.tlsAllowInsecureConnection,
-                    this.tlsTrustCertsFilePath);
-        }
-    }
-
-    public SslContext get() {
-        if (authData != null && authData.hasDataForTls()) {
-            try {
-                this.sslContext = SecurityUtility.createNettySslContextForClient(this.tlsAllowInsecureConnection,
-                        this.tlsTrustCertsFilePath, (X509Certificate[]) authData.getTlsCertificates(),
-                        authData.getTlsPrivateKey());
-            } catch (GeneralSecurityException | IOException e) {
-                LOG.error("Exception occured while trying to refresh sslContext: ", e);
-            }
-
-        }
-        return sslContext;
-    }
-
-    private static final Logger LOG = LoggerFactory.getLogger(ClientSslContextRefresher.class);
-}
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultSslContextBuilder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultSslContextBuilder.java
index 3e888f4..c49bdd6 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultSslContextBuilder.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultSslContextBuilder.java
@@ -29,11 +29,19 @@ import javax.net.ssl.SSLException;
 public class DefaultSslContextBuilder extends SslContextAutoRefreshBuilder<SSLContext> {
     private volatile SSLContext sslContext;
 
+    protected final boolean tlsAllowInsecureConnection;
+    protected final FileModifiedTimeUpdater tlsTrustCertsFilePath, tlsCertificateFilePath, tlsKeyFilePath;
+    protected final boolean tlsRequireTrustedClientCertOnConnect;
+
     public DefaultSslContextBuilder(boolean allowInsecure, String trustCertsFilePath, String certificateFilePath,
             String keyFilePath, boolean requireTrustedClientCertOnConnect, long certRefreshInSec)
             throws SSLException, FileNotFoundException, GeneralSecurityException, IOException {
-        super(allowInsecure, trustCertsFilePath, certificateFilePath, keyFilePath, null, null,
-                requireTrustedClientCertOnConnect, certRefreshInSec);
+        super(certRefreshInSec);
+        this.tlsAllowInsecureConnection = allowInsecure;
+        this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(trustCertsFilePath);
+        this.tlsCertificateFilePath = new FileModifiedTimeUpdater(certificateFilePath);
+        this.tlsKeyFilePath = new FileModifiedTimeUpdater(keyFilePath);
+        this.tlsRequireTrustedClientCertOnConnect = requireTrustedClientCertOnConnect;
     }
 
     @Override
@@ -49,4 +57,10 @@ public class DefaultSslContextBuilder extends SslContextAutoRefreshBuilder<SSLCo
         return this.sslContext;
     }
 
+    @Override
+    public boolean needUpdate() {
+        return  tlsTrustCertsFilePath.checkAndRefresh()
+                || tlsCertificateFilePath.checkAndRefresh()
+                || tlsKeyFilePath.checkAndRefresh();
+    }
 }
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyClientSslContextRefresher.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyClientSslContextRefresher.java
new file mode 100644
index 0000000..48cf992
--- /dev/null
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyClientSslContextRefresher.java
@@ -0,0 +1,74 @@
+/**
+ * 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.util;
+
+import io.netty.handler.ssl.SslContext;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import java.security.cert.X509Certificate;
+import javax.net.ssl.SSLException;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.AuthenticationDataProvider;
+
+/**
+ * SSL context builder for Netty Client side.
+ */
+@Slf4j
+public class NettyClientSslContextRefresher extends SslContextAutoRefreshBuilder<SslContext> {
+    private volatile SslContext sslNettyContext;
+    private boolean tlsAllowInsecureConnection;
+    protected final FileModifiedTimeUpdater tlsTrustCertsFilePath;
+    private AuthenticationDataProvider authData;
+
+    public NettyClientSslContextRefresher(boolean allowInsecure,
+                                          String trustCertsFilePath,
+                                          AuthenticationDataProvider authData,
+                                          long delayInSeconds)
+            throws IOException, GeneralSecurityException {
+        super(delayInSeconds);
+        this.tlsAllowInsecureConnection = allowInsecure;
+        this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(trustCertsFilePath);
+        this.authData = authData;
+    }
+
+    @Override
+    public synchronized SslContext update()
+            throws SSLException, FileNotFoundException, GeneralSecurityException, IOException {
+        if (authData != null && authData.hasDataForTls()) {
+            this.sslNettyContext = SecurityUtility.createNettySslContextForClient(this.tlsAllowInsecureConnection,
+                    this.tlsTrustCertsFilePath.getFileName(), (X509Certificate[]) authData.getTlsCertificates(),
+                    authData.getTlsPrivateKey());
+        } else {
+            this.sslNettyContext = SecurityUtility.createNettySslContextForClient(this.tlsAllowInsecureConnection,
+                    this.tlsTrustCertsFilePath.getFileName());
+        }
+        return this.sslNettyContext;
+    }
+
+    @Override
+    public SslContext getSslContext() {
+        return this.sslNettyContext;
+    }
+
+    @Override
+    public boolean needUpdate() {
+        return  tlsTrustCertsFilePath.checkAndRefresh();
+    }
+}
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettySslContextBuilder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyServerSslContextBuilder.java
similarity index 52%
rename from pulsar-common/src/main/java/org/apache/pulsar/common/util/NettySslContextBuilder.java
rename to pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyServerSslContextBuilder.java
index 713c52d..250e628 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettySslContextBuilder.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyServerSslContextBuilder.java
@@ -26,16 +26,29 @@ import java.util.Set;
 import javax.net.ssl.SSLException;
 
 /**
- * SSL context builder for Netty.
+ * SSL context builder for Netty Server side.
  */
-public class NettySslContextBuilder extends SslContextAutoRefreshBuilder<SslContext> {
+public class NettyServerSslContextBuilder extends SslContextAutoRefreshBuilder<SslContext> {
     private volatile SslContext sslNettyContext;
 
-    public NettySslContextBuilder(boolean allowInsecure, String trustCertsFilePath, String certificateFilePath,
-            String keyFilePath, Set<String> ciphers, Set<String> protocols, boolean requireTrustedClientCertOnConnect,
-            long delayInSeconds) throws SSLException, FileNotFoundException, GeneralSecurityException, IOException {
-        super(allowInsecure, trustCertsFilePath, certificateFilePath, keyFilePath, ciphers, protocols,
-                requireTrustedClientCertOnConnect, delayInSeconds);
+    protected final boolean tlsAllowInsecureConnection;
+    protected final FileModifiedTimeUpdater tlsTrustCertsFilePath, tlsCertificateFilePath, tlsKeyFilePath;
+    protected final Set<String> tlsCiphers;
+    protected final Set<String> tlsProtocols;
+    protected final boolean tlsRequireTrustedClientCertOnConnect;
+
+    public NettyServerSslContextBuilder(boolean allowInsecure, String trustCertsFilePath, String certificateFilePath,
+                                        String keyFilePath, Set<String> ciphers, Set<String> protocols,
+                                        boolean requireTrustedClientCertOnConnect,
+                                        long delayInSeconds) {
+        super(delayInSeconds);
+        this.tlsAllowInsecureConnection = allowInsecure;
+        this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(trustCertsFilePath);
+        this.tlsCertificateFilePath = new FileModifiedTimeUpdater(certificateFilePath);
+        this.tlsKeyFilePath = new FileModifiedTimeUpdater(keyFilePath);
+        this.tlsCiphers = ciphers;
+        this.tlsProtocols = protocols;
+        this.tlsRequireTrustedClientCertOnConnect = requireTrustedClientCertOnConnect;
     }
 
     @Override
@@ -52,4 +65,10 @@ public class NettySslContextBuilder extends SslContextAutoRefreshBuilder<SslCont
         return this.sslNettyContext;
     }
 
+    @Override
+    public boolean needUpdate() {
+        return  tlsTrustCertsFilePath.checkAndRefresh()
+                || tlsCertificateFilePath.checkAndRefresh()
+                || tlsKeyFilePath.checkAndRefresh();
+    }
 }
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SslContextAutoRefreshBuilder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SslContextAutoRefreshBuilder.java
index 5fa9c1b..a29f051 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SslContextAutoRefreshBuilder.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SslContextAutoRefreshBuilder.java
@@ -18,16 +18,10 @@
  */
 package org.apache.pulsar.common.util;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.security.GeneralSecurityException;
-import java.util.Set;
 import java.util.concurrent.TimeUnit;
-
-import javax.net.ssl.SSLException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import lombok.extern.slf4j.Slf4j;
 
 /**
  * Auto refresher and builder of SSLContext.
@@ -35,30 +29,18 @@ import org.slf4j.LoggerFactory;
  * @param <T>
  *            type of SSLContext
  */
+@Slf4j
 public abstract class SslContextAutoRefreshBuilder<T> {
-    protected final boolean tlsAllowInsecureConnection;
-    protected final FileModifiedTimeUpdater tlsTrustCertsFilePath, tlsCertificateFilePath, tlsKeyFilePath;
-    protected final Set<String> tlsCiphers;
-    protected final Set<String> tlsProtocols;
-    protected final boolean tlsRequireTrustedClientCertOnConnect;
     protected final long refreshTime;
     protected long lastRefreshTime;
 
-    public SslContextAutoRefreshBuilder(boolean allowInsecure, String trustCertsFilePath, String certificateFilePath,
-            String keyFilePath, Set<String> ciphers, Set<String> protocols, boolean requireTrustedClientCertOnConnect,
-            long certRefreshInSec) throws SSLException, FileNotFoundException, GeneralSecurityException, IOException {
-        this.tlsAllowInsecureConnection = allowInsecure;
-        this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(trustCertsFilePath);
-        this.tlsCertificateFilePath = new FileModifiedTimeUpdater(certificateFilePath);
-        this.tlsKeyFilePath = new FileModifiedTimeUpdater(keyFilePath);
-        this.tlsCiphers = ciphers;
-        this.tlsProtocols = protocols;
-        this.tlsRequireTrustedClientCertOnConnect = requireTrustedClientCertOnConnect;
+    public SslContextAutoRefreshBuilder(
+            long certRefreshInSec) {
         this.refreshTime = TimeUnit.SECONDS.toMillis(certRefreshInSec);
         this.lastRefreshTime = -1;
 
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Certs will be refreshed every {} seconds", certRefreshInSec);
+        if (log.isDebugEnabled()) {
+            log.debug("Certs will be refreshed every {} seconds", certRefreshInSec);
         }
     }
 
@@ -79,6 +61,13 @@ public abstract class SslContextAutoRefreshBuilder<T> {
     protected abstract T getSslContext();
 
     /**
+     * Returns whether the key files modified after a refresh time, and context need update.
+     *
+     * @return true if files modified
+     */
+    protected abstract boolean needUpdate();
+
+    /**
      * It updates SSLContext at every configured refresh time and returns updated SSLContext.
      *
      * @return
@@ -91,24 +80,21 @@ public abstract class SslContextAutoRefreshBuilder<T> {
                 lastRefreshTime = System.currentTimeMillis();
                 return getSslContext();
             } catch (GeneralSecurityException | IOException e) {
-                LOG.error("Execption while trying to refresh ssl Context {}", e.getMessage(), e);
+                log.error("Exception while trying to refresh ssl Context {}", e.getMessage(), e);
             }
         } else {
             long now = System.currentTimeMillis();
             if (refreshTime <= 0 || now > (lastRefreshTime + refreshTime)) {
-                if (tlsTrustCertsFilePath.checkAndRefresh() || tlsCertificateFilePath.checkAndRefresh()
-                        || tlsKeyFilePath.checkAndRefresh()) {
+                if (needUpdate()) {
                     try {
                         ctx = update();
                         lastRefreshTime = now;
                     } catch (GeneralSecurityException | IOException e) {
-                        LOG.error("Execption while trying to refresh ssl Context {} ", e.getMessage(), e);
+                        log.error("Exception while trying to refresh ssl Context {} ", e.getMessage(), e);
                     }
                 }
             }
         }
         return ctx;
     }
-
-    private static final Logger LOG = LoggerFactory.getLogger(SslContextAutoRefreshBuilder.class);
 }
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/KeyStoreSSLContext.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/KeyStoreSSLContext.java
new file mode 100644
index 0000000..b9ad2e7
--- /dev/null
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/KeyStoreSSLContext.java
@@ -0,0 +1,355 @@
+/**
+ * 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.util.keystoretls;
+
+import static org.apache.pulsar.common.util.SecurityUtility.getProvider;
+
+import com.google.common.base.Strings;
+import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import java.security.KeyStore;
+import java.security.Provider;
+import java.security.SecureRandom;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.TrustManagerFactory;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+
+/**
+ * KeyStoreSSLContext that mainly wrap a SSLContext to provide SSL context for both webservice and netty.
+ */
+@Slf4j
+public class KeyStoreSSLContext {
+    public static final String DEFAULT_KEYSTORE_TYPE = "JKS";
+    public static final String DEFAULT_SSL_PROTOCOL = "TLS";
+    public static final String DEFAULT_SSL_ENABLED_PROTOCOLS = "TLSv1.2,TLSv1.1,TLSv1";
+    public static final String DEFAULT_SSL_KEYMANGER_ALGORITHM = KeyManagerFactory.getDefaultAlgorithm();
+    public static final String DEFAULT_SSL_TRUSTMANAGER_ALGORITHM = TrustManagerFactory.getDefaultAlgorithm();
+
+    public static final Provider BC_PROVIDER = getProvider();
+
+    /**
+     * Connection Mode for TLS.
+     */
+    public enum Mode {
+        CLIENT,
+        SERVER
+    }
+
+    @Getter
+    private final Mode mode;
+
+    private String sslProviderString;
+    private String keyStoreTypeString;
+    private String keyStorePath;
+    private String keyStorePassword;
+    private boolean allowInsecureConnection;
+    private String trustStoreTypeString;
+    private String trustStorePath;
+    private String trustStorePassword;
+    private boolean needClientAuth;
+    private Set<String> ciphers;
+    private Set<String> protocols;
+    @Getter
+    private SSLContext sslContext;
+
+    private String protocol = DEFAULT_SSL_PROTOCOL;
+    private String kmfAlgorithm = DEFAULT_SSL_KEYMANGER_ALGORITHM;
+    private String tmfAlgorithm = DEFAULT_SSL_TRUSTMANAGER_ALGORITHM;
+
+    // only init vars, before using it, need to call createSSLContext to create ssl context.
+    public KeyStoreSSLContext(Mode mode,
+                              String sslProviderString,
+                              String keyStoreTypeString,
+                              String keyStorePath,
+                              String keyStorePassword,
+                              boolean allowInsecureConnection,
+                              String trustStoreTypeString,
+                              String trustStorePath,
+                              String trustStorePassword,
+                              boolean requireTrustedClientCertOnConnect,
+                              Set<String> ciphers,
+                              Set<String> protocols) {
+        this.mode = mode;
+        this.sslProviderString = sslProviderString;
+        this.keyStoreTypeString = Strings.isNullOrEmpty(keyStoreTypeString)
+                ? DEFAULT_KEYSTORE_TYPE
+                : keyStoreTypeString;
+        this.keyStorePath = keyStorePath;
+        this.keyStorePassword = keyStorePassword;
+        this.trustStoreTypeString = Strings.isNullOrEmpty(trustStoreTypeString)
+                ? DEFAULT_KEYSTORE_TYPE
+                : trustStoreTypeString;
+        this.trustStorePath = trustStorePath;
+        this.trustStorePassword = trustStorePassword;
+        this.needClientAuth = requireTrustedClientCertOnConnect;
+        this.ciphers = ciphers;
+        this.protocols = protocols;
+
+        if (protocols != null && protocols.size() > 0) {
+            this.protocols = protocols;
+        } else {
+            this.protocols = new HashSet<>(Arrays.asList(DEFAULT_SSL_ENABLED_PROTOCOLS.split("\\s*,\\s*")));
+        }
+
+        if (ciphers != null && ciphers.size() > 0) {
+            this.ciphers = ciphers;
+        } else {
+            this.ciphers = null;
+        }
+
+        this.allowInsecureConnection = allowInsecureConnection;
+    }
+
+    public SSLContext createSSLContext() throws GeneralSecurityException, IOException {
+        SSLContext sslContext;
+        if (sslProviderString != null) {
+            sslContext = SSLContext.getInstance(protocol, sslProviderString);
+        } else {
+            sslContext = SSLContext.getInstance(protocol);
+        }
+
+        // key store
+        KeyManager[] keyManagers = null;
+        if (!Strings.isNullOrEmpty(keyStorePath)) {
+            KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(kmfAlgorithm);
+            KeyStore keyStore = KeyStore.getInstance(keyStoreTypeString);
+            char[] passwordChars = keyStorePassword.toCharArray();
+            keyStore.load(new FileInputStream(keyStorePath), passwordChars);
+            keyManagerFactory.init(keyStore, passwordChars);
+            keyManagers = keyManagerFactory.getKeyManagers();
+        }
+
+        // trust store
+        TrustManagerFactory trustManagerFactory;
+        if (this.allowInsecureConnection) {
+            trustManagerFactory = InsecureTrustManagerFactory.INSTANCE;
+        } else {
+            trustManagerFactory = TrustManagerFactory.getInstance(tmfAlgorithm);
+            KeyStore trustStore = KeyStore.getInstance(trustStoreTypeString);
+            char[] passwordChars = trustStorePassword.toCharArray();
+            trustStore.load(new FileInputStream(trustStorePath), passwordChars);
+            trustManagerFactory.init(trustStore);
+        }
+
+        // init
+        sslContext.init(keyManagers, trustManagerFactory.getTrustManagers(), new SecureRandom());
+        this.sslContext = sslContext;
+        return sslContext;
+    }
+
+    public SSLEngine createSSLEngine() {
+        SSLEngine sslEngine = sslContext.createSSLEngine();
+
+        sslEngine.setEnabledProtocols(sslEngine.getSupportedProtocols());
+        sslEngine.setEnabledCipherSuites(sslEngine.getSupportedCipherSuites());
+
+        if (this.mode == Mode.SERVER) {
+            sslEngine.setNeedClientAuth(this.needClientAuth);
+            sslEngine.setUseClientMode(false);
+        } else {
+            sslEngine.setUseClientMode(true);
+        }
+
+        return sslEngine;
+    }
+
+    public static KeyStoreSSLContext createClientKeyStoreSslContext(String sslProviderString,
+                                                            String keyStoreTypeString,
+                                                            String keyStorePath,
+                                                            String keyStorePassword,
+                                                            boolean allowInsecureConnection,
+                                                            String trustStoreTypeString,
+                                                            String trustStorePath,
+                                                            String trustStorePassword,
+                                                            Set<String> ciphers,
+                                                            Set<String> protocols)
+            throws GeneralSecurityException, SSLException, FileNotFoundException, IOException {
+        KeyStoreSSLContext keyStoreSSLContext = new KeyStoreSSLContext(Mode.CLIENT,
+                sslProviderString,
+                keyStoreTypeString,
+                keyStorePath,
+                keyStorePassword,
+                allowInsecureConnection,
+                trustStoreTypeString,
+                trustStorePath,
+                trustStorePassword,
+                false,
+                ciphers,
+                protocols);
+
+        keyStoreSSLContext.createSSLContext();
+        return keyStoreSSLContext;
+    }
+
+
+    public static KeyStoreSSLContext createServerKeyStoreSslContext(String sslProviderString,
+                                                    String keyStoreTypeString,
+                                                    String keyStorePath,
+                                                    String keyStorePassword,
+                                                    boolean allowInsecureConnection,
+                                                    String trustStoreTypeString,
+                                                    String trustStorePath,
+                                                    String trustStorePassword,
+                                                    boolean requireTrustedClientCertOnConnect,
+                                                    Set<String> ciphers,
+                                                    Set<String> protocols)
+            throws GeneralSecurityException, SSLException, FileNotFoundException, IOException {
+        KeyStoreSSLContext keyStoreSSLContext = new KeyStoreSSLContext(Mode.SERVER,
+                sslProviderString,
+                keyStoreTypeString,
+                keyStorePath,
+                keyStorePassword,
+                allowInsecureConnection,
+                trustStoreTypeString,
+                trustStorePath,
+                trustStorePassword,
+                requireTrustedClientCertOnConnect,
+                ciphers,
+                protocols);
+
+        keyStoreSSLContext.createSSLContext();
+        return keyStoreSSLContext;
+    }
+
+    // for web server use case, no need ciphers and protocols
+    public static SSLContext createServerSslContext(String sslProviderString,
+                                                    String keyStoreTypeString,
+                                                    String keyStorePath,
+                                                    String keyStorePassword,
+                                                    boolean allowInsecureConnection,
+                                                    String trustStoreTypeString,
+                                                    String trustStorePath,
+                                                    String trustStorePassword,
+                                                    boolean requireTrustedClientCertOnConnect)
+            throws GeneralSecurityException, SSLException, FileNotFoundException, IOException {
+
+        return createServerKeyStoreSslContext(
+                sslProviderString,
+                keyStoreTypeString,
+                keyStorePath,
+                keyStorePassword,
+                allowInsecureConnection,
+                trustStoreTypeString,
+                trustStorePath,
+                trustStorePassword,
+                requireTrustedClientCertOnConnect,
+                null,
+                null).getSslContext();
+    }
+
+    // for web client
+    public static SSLContext createClientSslContext(String sslProviderString,
+                                                    String keyStoreTypeString,
+                                                    String keyStorePath,
+                                                    String keyStorePassword,
+                                                    boolean allowInsecureConnection,
+                                                    String trustStoreTypeString,
+                                                    String trustStorePath,
+                                                    String trustStorePassword,
+                                                    Set<String> ciphers,
+                                                    Set<String> protocol)
+            throws GeneralSecurityException, SSLException, FileNotFoundException, IOException {
+        KeyStoreSSLContext keyStoreSSLContext = new KeyStoreSSLContext(Mode.CLIENT,
+                sslProviderString,
+                keyStoreTypeString,
+                keyStorePath,
+                keyStorePassword,
+                allowInsecureConnection,
+                trustStoreTypeString,
+                trustStorePath,
+                trustStorePassword,
+                false,
+                ciphers,
+                protocol);
+
+        return keyStoreSSLContext.createSSLContext();
+    }
+
+    // for web client
+    public static SSLContext createClientSslContext(String keyStoreTypeString,
+                                                    String keyStorePath,
+                                                    String keyStorePassword,
+                                                    String trustStoreTypeString,
+                                                    String trustStorePath,
+                                                    String trustStorePassword)
+            throws GeneralSecurityException, SSLException, FileNotFoundException, IOException {
+        KeyStoreSSLContext keyStoreSSLContext = new KeyStoreSSLContext(Mode.CLIENT,
+                null,
+                keyStoreTypeString,
+                keyStorePath,
+                keyStorePassword,
+                false,
+                trustStoreTypeString,
+                trustStorePath,
+                trustStorePassword,
+                false,
+                null,
+                null);
+
+        return keyStoreSSLContext.createSSLContext();
+    }
+
+    // for web server. autoRefresh is default true.
+    public static SslContextFactory createSslContextFactory(String sslProviderString,
+                                                            String keyStoreTypeString,
+                                                            String keyStore,
+                                                            String keyStorePassword,
+                                                            boolean allowInsecureConnection,
+                                                            String trustStoreTypeString,
+                                                            String trustStore,
+                                                            String trustStorePassword,
+                                                            boolean requireTrustedClientCertOnConnect,
+                                                            long certRefreshInSec)
+            throws GeneralSecurityException, SSLException, FileNotFoundException, IOException {
+        SslContextFactory sslCtxFactory;
+
+        sslCtxFactory = new SslContextFactoryWithAutoRefresh(
+                sslProviderString,
+                keyStoreTypeString,
+                keyStore,
+                keyStorePassword,
+                allowInsecureConnection,
+                trustStoreTypeString,
+                trustStore,
+                trustStorePassword,
+                requireTrustedClientCertOnConnect,
+                certRefreshInSec);
+
+        if (requireTrustedClientCertOnConnect) {
+            sslCtxFactory.setNeedClientAuth(true);
+        } else {
+            sslCtxFactory.setWantClientAuth(true);
+        }
+        sslCtxFactory.setTrustAll(true);
+
+        return sslCtxFactory;
+    }
+}
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NetSslContextBuilder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NetSslContextBuilder.java
new file mode 100644
index 0000000..38ebdb4
--- /dev/null
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NetSslContextBuilder.java
@@ -0,0 +1,92 @@
+/**
+ * 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.util.keystoretls;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLException;
+import org.apache.pulsar.common.util.FileModifiedTimeUpdater;
+import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder;
+
+/**
+ * Similar to `DefaultSslContextBuilder`, which build `javax.net.ssl.SSLContext` for web service.
+ */
+public class NetSslContextBuilder extends SslContextAutoRefreshBuilder<SSLContext> {
+    private volatile SSLContext sslContext;
+
+    protected final boolean tlsAllowInsecureConnection;
+    protected final boolean tlsRequireTrustedClientCertOnConnect;
+
+    protected final String tlsProvider;
+    protected final String tlsKeyStoreType;
+    protected final String tlsKeyStorePassword;
+    protected final FileModifiedTimeUpdater tlsKeyStore;
+    protected final String tlsTrustStoreType;
+    protected final String tlsTrustStorePassword;
+    protected final FileModifiedTimeUpdater tlsTrustStore;
+
+    public NetSslContextBuilder(String sslProviderString,
+                                String keyStoreTypeString,
+                                String keyStore,
+                                String keyStorePasswordPath,
+                                boolean allowInsecureConnection,
+                                String trustStoreTypeString,
+                                String trustStore,
+                                String trustStorePasswordPath,
+                                boolean requireTrustedClientCertOnConnect,
+                                long certRefreshInSec) {
+        super(certRefreshInSec);
+
+        this.tlsAllowInsecureConnection = allowInsecureConnection;
+        this.tlsProvider = sslProviderString;
+        this.tlsKeyStoreType = keyStoreTypeString;
+        this.tlsKeyStore = new FileModifiedTimeUpdater(keyStore);
+        this.tlsKeyStorePassword = keyStorePasswordPath;
+
+        this.tlsTrustStoreType = trustStoreTypeString;
+        this.tlsTrustStore = new FileModifiedTimeUpdater(trustStore);
+        this.tlsTrustStorePassword = trustStorePasswordPath;
+
+        this.tlsRequireTrustedClientCertOnConnect = requireTrustedClientCertOnConnect;
+    }
+
+    @Override
+    public synchronized SSLContext update()
+            throws SSLException, FileNotFoundException, GeneralSecurityException, IOException {
+        this.sslContext = KeyStoreSSLContext.createServerSslContext(tlsProvider,
+                tlsKeyStoreType, tlsKeyStore.getFileName(), tlsKeyStorePassword,
+                tlsAllowInsecureConnection,
+                tlsTrustStoreType, tlsTrustStore.getFileName(), tlsTrustStorePassword,
+                tlsRequireTrustedClientCertOnConnect);
+        return this.sslContext;
+    }
+
+    @Override
+    public SSLContext getSslContext() {
+        return this.sslContext;
+    }
+
+    @Override
+    public boolean needUpdate() {
+        return  tlsKeyStore.checkAndRefresh()
+                || tlsTrustStore.checkAndRefresh();
+    }
+}
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NettySSLContextAutoRefreshBuilder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NettySSLContextAutoRefreshBuilder.java
new file mode 100644
index 0000000..363fe1e
--- /dev/null
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NettySSLContextAutoRefreshBuilder.java
@@ -0,0 +1,144 @@
+/**
+ * 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.util.keystoretls;
+
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import java.util.Set;
+import org.apache.pulsar.client.api.AuthenticationDataProvider;
+import org.apache.pulsar.client.api.KeyStoreParams;
+import org.apache.pulsar.common.util.FileModifiedTimeUpdater;
+import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder;
+
+/**
+ * SSL context builder for Netty.
+ */
+public class NettySSLContextAutoRefreshBuilder extends SslContextAutoRefreshBuilder<KeyStoreSSLContext> {
+    private volatile KeyStoreSSLContext keyStoreSSLContext;
+
+    protected final boolean tlsAllowInsecureConnection;
+    protected final Set<String> tlsCiphers;
+    protected final Set<String> tlsProtocols;
+    protected boolean tlsRequireTrustedClientCertOnConnect;
+
+    protected final String tlsProvider;
+    protected final String tlsTrustStoreType;
+    protected final String tlsTrustStorePassword;
+    protected final FileModifiedTimeUpdater tlsTrustStore;
+
+    // client context not need keystore at start time, keyStore is passed in by authData.
+    protected String tlsKeyStoreType;
+    protected String tlsKeyStorePassword;
+    protected FileModifiedTimeUpdater tlsKeyStore;
+
+    protected AuthenticationDataProvider authData;
+    protected final boolean isServer;
+
+    // for server
+    public NettySSLContextAutoRefreshBuilder(String sslProviderString,
+                                             String keyStoreTypeString,
+                                             String keyStore,
+                                             String keyStorePassword,
+                                             boolean allowInsecureConnection,
+                                             String trustStoreTypeString,
+                                             String trustStore,
+                                             String trustStorePassword,
+                                             boolean requireTrustedClientCertOnConnect,
+                                             Set<String> ciphers,
+                                             Set<String> protocols,
+                                             long certRefreshInSec) {
+        super(certRefreshInSec);
+
+        this.tlsAllowInsecureConnection = allowInsecureConnection;
+        this.tlsProvider = sslProviderString;
+
+        this.tlsKeyStoreType = keyStoreTypeString;
+        this.tlsKeyStore = new FileModifiedTimeUpdater(keyStore);
+        this.tlsKeyStorePassword = keyStorePassword;
+
+        this.tlsTrustStoreType = trustStoreTypeString;
+        this.tlsTrustStore = new FileModifiedTimeUpdater(trustStore);
+        this.tlsTrustStorePassword = trustStorePassword;
+
+        this.tlsRequireTrustedClientCertOnConnect = requireTrustedClientCertOnConnect;
+        this.tlsCiphers = ciphers;
+        this.tlsProtocols = protocols;
+
+        this.isServer = true;
+    }
+
+    // for client
+    public NettySSLContextAutoRefreshBuilder(String sslProviderString,
+                                             boolean allowInsecureConnection,
+                                             String trustStoreTypeString,
+                                             String trustStore,
+                                             String trustStorePassword,
+                                             Set<String> ciphers,
+                                             Set<String> protocols,
+                                             long certRefreshInSec,
+                                             AuthenticationDataProvider authData) {
+        super(certRefreshInSec);
+
+        this.tlsAllowInsecureConnection = allowInsecureConnection;
+        this.tlsProvider = sslProviderString;
+
+        this.authData = authData;
+
+        this.tlsTrustStoreType = trustStoreTypeString;
+        this.tlsTrustStore = new FileModifiedTimeUpdater(trustStore);
+        this.tlsTrustStorePassword = trustStorePassword;
+
+        this.tlsCiphers = ciphers;
+        this.tlsProtocols = protocols;
+
+        this.isServer = false;
+    }
+
+    @Override
+    public synchronized KeyStoreSSLContext update() throws GeneralSecurityException, IOException {
+        if (isServer) {
+            this.keyStoreSSLContext = KeyStoreSSLContext.createServerKeyStoreSslContext(tlsProvider,
+                    tlsKeyStoreType, tlsKeyStore.getFileName(), tlsKeyStorePassword,
+                    tlsAllowInsecureConnection,
+                    tlsTrustStoreType, tlsTrustStore.getFileName(), tlsTrustStorePassword,
+                    tlsRequireTrustedClientCertOnConnect, tlsCiphers, tlsProtocols);
+        } else {
+            KeyStoreParams authParams = authData.getTlsKeyStoreParams();
+            this.keyStoreSSLContext = KeyStoreSSLContext.createClientKeyStoreSslContext(tlsProvider,
+                    authParams != null ? authParams.getKeyStoreType() : null,
+                    authParams != null ? authParams.getKeyStorePath() : null,
+                    authParams != null ? authParams.getKeyStorePassword() : null,
+                    tlsAllowInsecureConnection,
+                    tlsTrustStoreType, tlsTrustStore.getFileName(), tlsTrustStorePassword,
+                    tlsCiphers, tlsProtocols);
+        }
+        return this.keyStoreSSLContext;
+    }
+
+    @Override
+    public KeyStoreSSLContext getSslContext() {
+        return this.keyStoreSSLContext;
+    }
+
+    @Override
+    public boolean needUpdate() {
+        return  (tlsKeyStore != null && tlsKeyStore.checkAndRefresh())
+                || (tlsTrustStore != null && tlsTrustStore.checkAndRefresh());
+    }
+}
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/SSLContextValidatorEngine.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/SSLContextValidatorEngine.java
new file mode 100644
index 0000000..555d96e
--- /dev/null
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/SSLContextValidatorEngine.java
@@ -0,0 +1,176 @@
+/**
+ * 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.util.keystoretls;
+
+import java.nio.ByteBuffer;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+import javax.net.ssl.SSLEngineResult;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.SSLParameters;
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * SSLContextValidatorEngine to validate 2 SSlContext.
+ */
+@Slf4j
+public class SSLContextValidatorEngine {
+    /**
+     * Mode of peer.
+     */
+    public enum Mode {
+        CLIENT,
+        SERVER
+    }
+
+    private static final ByteBuffer EMPTY_BUF = ByteBuffer.allocate(0);
+    private final SSLEngine sslEngine;
+    private SSLEngineResult handshakeResult;
+    private ByteBuffer appBuffer;
+    private ByteBuffer netBuffer;
+    private Mode mode;
+
+    public static void validate(SSLContext clientSslContext, SSLContext serverSslContext) throws SSLException {
+        SSLContextValidatorEngine clientEngine = new SSLContextValidatorEngine(clientSslContext, Mode.CLIENT);
+        SSLContextValidatorEngine serverEngine = new SSLContextValidatorEngine(serverSslContext, Mode.SERVER);
+        try {
+            clientEngine.beginHandshake();
+            serverEngine.beginHandshake();
+            while (!serverEngine.complete() || !clientEngine.complete()) {
+                clientEngine.handshake(serverEngine);
+                serverEngine.handshake(clientEngine);
+            }
+        } finally {
+            clientEngine.close();
+            serverEngine.close();
+        }
+    }
+
+    private SSLContextValidatorEngine(SSLContext sslContext, Mode mode) {
+        this.mode = mode;
+        this.sslEngine = createSslEngine(sslContext, "localhost", 0); // these hints are not used for validation
+        sslEngine.setUseClientMode(mode == Mode.CLIENT);
+        appBuffer = ByteBuffer.allocate(sslEngine.getSession().getApplicationBufferSize());
+        netBuffer = ByteBuffer.allocate(sslEngine.getSession().getPacketBufferSize());
+    }
+
+    private SSLEngine createSslEngine(SSLContext sslContext, String peerHost, int peerPort) {
+        SSLEngine sslEngine = sslContext.createSSLEngine(peerHost, peerPort);
+
+        if (mode == Mode.SERVER) {
+            sslEngine.setNeedClientAuth(true);
+        } else {
+            sslEngine.setUseClientMode(true);
+            SSLParameters sslParams = sslEngine.getSSLParameters();
+            sslEngine.setSSLParameters(sslParams);
+        }
+        return sslEngine;
+    }
+
+    void beginHandshake() throws SSLException {
+        sslEngine.beginHandshake();
+    }
+
+    void handshake(SSLContextValidatorEngine peerEngine) throws SSLException {
+        SSLEngineResult.HandshakeStatus handshakeStatus = sslEngine.getHandshakeStatus();
+        while (true) {
+            switch (handshakeStatus) {
+                case NEED_WRAP:
+                    handshakeResult = sslEngine.wrap(EMPTY_BUF, netBuffer);
+                    switch (handshakeResult.getStatus()) {
+                        case OK: break;
+                        case BUFFER_OVERFLOW:
+                            netBuffer.compact();
+                            netBuffer = ensureCapacity(netBuffer, sslEngine.getSession().getPacketBufferSize());
+                            netBuffer.flip();
+                            break;
+                        case BUFFER_UNDERFLOW:
+                        case CLOSED:
+                        default:
+                            throw new SSLException("Unexpected handshake status: " + handshakeResult.getStatus());
+                    }
+                    return;
+                case NEED_UNWRAP:
+                    if (peerEngine.netBuffer.position() == 0) {
+                        return;
+                    }
+                    peerEngine.netBuffer.flip(); // unwrap the data from peer
+                    handshakeResult = sslEngine.unwrap(peerEngine.netBuffer, appBuffer);
+                    peerEngine.netBuffer.compact();
+                    handshakeStatus = handshakeResult.getHandshakeStatus();
+                    switch (handshakeResult.getStatus()) {
+                        case OK: break;
+                        case BUFFER_OVERFLOW:
+                            appBuffer = ensureCapacity(appBuffer, sslEngine.getSession().getApplicationBufferSize());
+                            break;
+                        case BUFFER_UNDERFLOW:
+                            netBuffer = ensureCapacity(netBuffer, sslEngine.getSession().getPacketBufferSize());
+                            break;
+                        case CLOSED:
+                        default:
+                            throw new SSLException("Unexpected handshake status: " + handshakeResult.getStatus());
+                    }
+                    break;
+                case NEED_TASK:
+                    sslEngine.getDelegatedTask().run();
+                    handshakeStatus = sslEngine.getHandshakeStatus();
+                    break;
+                case FINISHED:
+                    return;
+                case NOT_HANDSHAKING:
+                    if (handshakeResult.getHandshakeStatus() != SSLEngineResult.HandshakeStatus.FINISHED) {
+                        throw new SSLException("Did not finish handshake");
+                    }
+                    return;
+                default:
+                    throw new IllegalStateException("Unexpected handshake status " + handshakeStatus);
+            }
+        }
+    }
+
+    boolean complete() {
+        return sslEngine.getHandshakeStatus() == SSLEngineResult.HandshakeStatus.FINISHED
+               || sslEngine.getHandshakeStatus() == SSLEngineResult.HandshakeStatus.NOT_HANDSHAKING;
+    }
+
+    void close() {
+        sslEngine.closeOutbound();
+        try {
+            sslEngine.closeInbound();
+        } catch (Exception e) {
+            // ignore
+        }
+    }
+
+    /**
+     * Check if the given ByteBuffer capacity.
+     * @param existingBuffer ByteBuffer capacity to check
+     * @param newLength new length for the ByteBuffer.
+     * returns ByteBuffer
+     */
+    public static ByteBuffer ensureCapacity(ByteBuffer existingBuffer, int newLength) {
+        if (newLength > existingBuffer.capacity()) {
+            ByteBuffer newBuffer = ByteBuffer.allocate(newLength);
+            existingBuffer.flip();
+            newBuffer.put(existingBuffer);
+            return newBuffer;
+        }
+        return existingBuffer;
+    }
+}
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/SslContextFactoryWithAutoRefresh.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/SslContextFactoryWithAutoRefresh.java
new file mode 100644
index 0000000..e18e8c6
--- /dev/null
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/SslContextFactoryWithAutoRefresh.java
@@ -0,0 +1,63 @@
+/**
+ * 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.util.keystoretls;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.security.GeneralSecurityException;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLException;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+
+/**
+ * SslContextFactoryWithAutoRefresh that create SSLContext for web server, and refresh in time.
+ */
+public class SslContextFactoryWithAutoRefresh extends SslContextFactory {
+    private final NetSslContextBuilder sslCtxRefresher;
+
+    public SslContextFactoryWithAutoRefresh(String sslProviderString,
+                                            String keyStoreTypeString,
+                                            String keyStore,
+                                            String keyStorePassword,
+                                            boolean allowInsecureConnection,
+                                            String trustStoreTypeString,
+                                            String trustStore,
+                                            String trustStorePassword,
+                                            boolean requireTrustedClientCertOnConnect,
+                                            long certRefreshInSec)
+            throws SSLException, FileNotFoundException, GeneralSecurityException, IOException {
+        super();
+        sslCtxRefresher = new NetSslContextBuilder(
+                sslProviderString,
+                keyStoreTypeString,
+                keyStore,
+                keyStorePassword,
+                allowInsecureConnection,
+                trustStoreTypeString,
+                trustStore,
+                trustStorePassword,
+                requireTrustedClientCertOnConnect,
+                certRefreshInSec);
+    }
+
+    @Override
+    public SSLContext getSslContext() {
+        return sslCtxRefresher.get();
+    }
+}
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/package-info.java
similarity index 53%
copy from pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java
copy to pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/package-info.java
index 03a9bd3..11a8db4 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataHttps.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/package-info.java
@@ -16,33 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pulsar.broker.authentication;
-
-import java.security.cert.X509Certificate;
-
-import javax.servlet.http.HttpServletRequest;
-
-public class AuthenticationDataHttps extends AuthenticationDataHttp {
-
-    protected final X509Certificate[] certificates;
-
-    public AuthenticationDataHttps(HttpServletRequest request) {
-        super(request);
-        certificates = (X509Certificate[]) request.getAttribute("javax.servlet.request.X509Certificate");
-    }
-
-    /*
-     * TLS
-     */
-
-    @Override
-    public boolean hasDataFromTls() {
-        return (certificates != null);
-    }
-
-    @Override
-    public X509Certificate[] getTlsCertificates() {
-        return certificates;
-    }
-
-}
+/**
+ * Helpers to work with events from the non-blocking I/O client-server framework.
+ */
+package org.apache.pulsar.common.util.keystoretls;
diff --git a/pulsar-common/src/test/resources/broker.keystore.jks b/pulsar-common/src/test/resources/broker.keystore.jks
new file mode 100644
index 0000000..b4fec69
Binary files /dev/null and b/pulsar-common/src/test/resources/broker.keystore.jks differ
diff --git a/pulsar-common/src/test/resources/broker.truststore.jks b/pulsar-common/src/test/resources/broker.truststore.jks
new file mode 100644
index 0000000..8ac03d8
Binary files /dev/null and b/pulsar-common/src/test/resources/broker.truststore.jks differ
diff --git a/pulsar-common/src/test/resources/brokerKeyStorePW.txt b/pulsar-common/src/test/resources/brokerKeyStorePW.txt
new file mode 100644
index 0000000..90d2950
--- /dev/null
+++ b/pulsar-common/src/test/resources/brokerKeyStorePW.txt
@@ -0,0 +1 @@
+111111
diff --git a/pulsar-common/src/test/resources/brokerTrustStorePW.txt b/pulsar-common/src/test/resources/brokerTrustStorePW.txt
new file mode 100644
index 0000000..90d2950
--- /dev/null
+++ b/pulsar-common/src/test/resources/brokerTrustStorePW.txt
@@ -0,0 +1 @@
+111111
diff --git a/pulsar-common/src/test/resources/ca-cert b/pulsar-common/src/test/resources/ca-cert
new file mode 100644
index 0000000..32c8d92
--- /dev/null
+++ b/pulsar-common/src/test/resources/ca-cert
@@ -0,0 +1,16 @@
+-----BEGIN CERTIFICATE-----
+MIICmDCCAYACCQCYZHWHBQWWnTANBgkqhkiG9w0BAQsFADANMQswCQYDVQQGEwJj
+bjAgFw0yMDA0MjgxMzIxMDBaGA8yMTIwMDQwNDEzMjEwMFowDTELMAkGA1UEBhMC
+Y24wggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQC5CuNhU2OYdwlrd/04
+FOgXbzGLs/RoWToWq39Q4WSEVr7uaEBktFEyaoCcLYwsWm9ziF0ms5SLqNDptFiS
+c8Ftt1YJdTS+SfFRg+CWUyq8LmA9NsT/X6/Oy3Q/4398stzWdU1X6KXnWr4UIkDo
+wLDJEFCBkJCqK6w0mg92rzu8pPfIzQ8kvTy0ECK0DfRqe7+9YVPVCrR5ZItln/nu
+MnccQ9fQPL0pvTUXkWFAh/GupaUJkvA69SiAfzXJohtdCdrKrxV3m76kHHxoRlcf
+z1MRHq/r1DWBHLuV6c8p22TW4fAke0i/qwjEroiDRGX2MohCaHTJT3xxIbopznBF
+7GfJAgMBAAEwDQYJKoZIhvcNAQELBQADggEBAJOz1oyP6TmRc2t7LcUzfEBFdKGZ
+PRyF7cin8o+c/IBl8svViTFk4dmdGAoeGpRQsn1i+J+AOKNBSRdyydgEFvE8qopR
+h7c9DzfSssf00eRAgdg8oCz2fOXDtLPwBTMe52q8MdJRe4OelRjNFs4VRpyVgZVQ
+13+GMgcj1E8taGqqSBqLccujWNJW1bsoesLzb6bYQWe8WrCPTQxB1NLIoYTZvXoK
+AkHShUEJV502UOxeRhjDjYeearlILIoV82sczXmlFNrhuiYxgIYa9tCywsagenRe
+6/WANyQP5nmCky0odbK0Uh7XweppFdb76FrooWVcd94HZaJBV7PnNdLoj/8=
+-----END CERTIFICATE-----
diff --git a/pulsar-common/src/test/resources/ca-cert.srl b/pulsar-common/src/test/resources/ca-cert.srl
new file mode 100644
index 0000000..aee9981
--- /dev/null
+++ b/pulsar-common/src/test/resources/ca-cert.srl
@@ -0,0 +1 @@
+A30DEADB8FD23BED
diff --git a/pulsar-common/src/test/resources/ca-key b/pulsar-common/src/test/resources/ca-key
new file mode 100644
index 0000000..485d220
--- /dev/null
+++ b/pulsar-common/src/test/resources/ca-key
@@ -0,0 +1,30 @@
+-----BEGIN ENCRYPTED PRIVATE KEY-----
+MIIFHzBJBgkqhkiG9w0BBQ0wPDAbBgkqhkiG9w0BBQwwDgQII+lV8LqZ9n4CAggA
+MB0GCWCGSAFlAwQBKgQQOGncQpnXEogdhApthe09awSCBNCErYMzOQEblSjp2HUq
+whuE1l7EUp4Et3cCtMenXoGlNfzMG0llnCmbIJA4j13X2IfGGpYRKNEbkeUUAuX5
+Er8nwWBZw3ux3iD4zYUl2Q69tcnC62eQaA5+Zj5T58i0ptxYXNTZ0p+q9ytMSP30
+9gb8KwJQdXiKZU5UxflLb9TrU7OWi3Ucnjbw4YYmRkwRZGZ6P+fUSC2FpixzI0J6
+73yBI36zfTvFJR5bslIFw2CSHbIFZeJ535oiLVqzOzjJHEFZ+OTeuN+Vh8Cktz3Y
+KAOdD30knci64vugxo5iLLWc/IXQRcuTBNHskPsym5ahYVSM9/+JqMw4OL2xyDbR
++YCdx3iER4PD9ErCXSCdMLsx4izuuULQGAyONtnj9awUwmzKMqEbtr6lLFChgyEc
+TiWpm5sLZPP1SiyjGKnhesiWRyB25b6iC/fSktf10Nrl+Fb1YJeLLSufG+ZVpy5h
+sN2cPkAnymRw9WyEaitkUqNI52GfClOYPJ0H0bU1LssUs2+d8HRHz6GWb3oW2IX8
+046PE6y0kIuUYEryTQ1lzmLdREIOG2yfkcL7ywN1WBwtqPZBV69peA8P3M3T9VEz
+nnTtEL/5VU0M3Bsm6GM8j9fmJDBmBG+6E0hUD6JCuDeBBvTBuknZIbiBf704DjyI
+qPDZhAkkVfD7dylm96GJLn38PrPk8sQPa3IR4zAXga7YHkXvh7HuYc0V9tZNn7mI
+/f3XoDV2wk279TAr8NLDgLGQHK0K1tDTirJXf09KxWj4zZOC914SeASjEhzQZ6se
+K3PbG4ZQnJ4+dAsY9K6Qgc9BxjyeInCdsDoDROtjqNfKSkcSelKhkp8VEJBYinWT
+PcBHt6/1iQvB1fyp+OexBjq9CiUDg/Be2QTUZfVqCkyIVptPnSvPyTg67pvWl4M6
+uRHxsufQ35WsZAhqEr5eQ7mPAvem7XCUJ14hPz8/f5Qm2+llPUVIq9Rpa83GP/TI
+P9W9F6tRj0qpW+QpPlxmyISf6oHiE9IOGmRkdFV0m54JiSQsqGqXr/NCH423LzFd
+21TJVKH02+v0Tgu1A1+HT+dEKQOorBZ3/HQ2NuQuYi+rF3adNEcMVmfy6DlnHkYr
+lDIG0exVC6Bveuzs4BCupKz/g7CbNzAEOO9XwcD/crNRnjE+nzdu/SDNtw5vIlKA
+hSSzEcsgVkqGbeaV5fASgb3pAz50xJHwvX6O4cAOvbcemjUGyd16IxIdI5jPRVvh
+u1BiK3YwSsdtg8sQ54YVbirgQ6SWKIXdN+79luksimbUVnEu8VJS1fu9H5ojefAd
+J9hMeiGht+6LKvyPh6Sa++bCfYRjZmbkX4h6Afc3Wwibh7KnfpAUlt4QzqA7o+x4
+7rCaI/w/uK+EFaqtn67TowAg/iq6Lxd7i9l06JBSC/BA+Hsw6tS86f13qPg2OtTK
+GydNfxnGtfZIMsUUtfldp9mB+afRFqX49joEGGmb2vnm4Q09QaGP5tagpJboIAqO
+c/pxtGWzqokR8sdAwX0oAb1vsPrpY3sbUGqSYJfVR6s4SMXJdbSSiIKzuwrcO+HX
+TSiq2yGGfJBl5bh9E8cnH4NifAJC4kXsBERwy+Ahq/64MRps3EW2tFl6nPuA+HMl
+IXg4wepHtC7w7W9nJ5Tw3b6X4g==
+-----END ENCRYPTED PRIVATE KEY-----
diff --git a/pulsar-common/src/test/resources/cert-file b/pulsar-common/src/test/resources/cert-file
new file mode 100644
index 0000000..7d94877
--- /dev/null
+++ b/pulsar-common/src/test/resources/cert-file
@@ -0,0 +1,17 @@
+-----BEGIN NEW CERTIFICATE REQUEST-----
+MIICpTCCAmECAQAwbzEQMA4GA1UEBhMHVW5rbm93bjEQMA4GA1UECBMHVW5rbm93
+bjEQMA4GA1UEBxMHVW5rbm93bjEQMA4GA1UEChMHVW5rbm93bjEQMA4GA1UECxMH
+VW5rbm93bjETMBEGA1UEAxMKY2xpZW50dXNlcjCCAbcwggEsBgcqhkjOOAQBMIIB
+HwKBgQD9f1OBHXUSKVLfSpwu7OTn9hG3UjzvRADDHj+AtlEmaUVdQCJR+1k9jVj6
+v8X1ujD2y5tVbNeBO4AdNG/yZmC3a5lQpaSfn+gEexAiwk+7qdf+t8Yb+DtX58ao
+phUPBPuD9tPFHsMCNVQTWhaRMvZ1864rYdcq7/IiAxmd0UgBxwIVAJdgUI8VIwvM
+spK5gqLrhAvwWBz1AoGBAPfhoIXWmz3ey7yrXDa4V7l5lK+7+jrqgvlXTAs9B4Jn
+UVlXjrrUWU/mcQcQgYC0SRZxI+hMKBYTt88JMozIpuE8FnqLVHyNKOCjrh4rs6Z1
+kW6jfwv6ITVi8ftiegEkO8yk8b6oUZCJqIPf4VrlnwaSi2ZegHtVJWQBTDv+z0kq
+A4GEAAKBgBmF8WdZ9Yv1Sf2qjqF19DUSY3YB67B0azz+689y8lZw0tlnSuej0bBE
+NIP6lvgC/PIPFdxvkInZOgB3TsWwkpxHzKbFZTo2Yg2txZ1IH4KX1QggePeybi2m
+E2soysZ2/r3nX2ZSOTdzDLicVo3yyKAuM8u14N0zBeJR9NMdOG1NoDAwLgYJKoZI
+hvcNAQkOMSEwHzAdBgNVHQ4EFgQUXx44DNZ7cUAoduGpv/MC+d5noyIwCwYHKoZI
+zjgEAwUAAzEAMC4CFQCQ2BDtunGs9G0Ra+16OHPaWAI6+QIVAIrGtZWtGka43D+3
+GqOEI5+wGsbh
+-----END NEW CERTIFICATE REQUEST-----
diff --git a/pulsar-common/src/test/resources/cert-signed b/pulsar-common/src/test/resources/cert-signed
new file mode 100644
index 0000000..20db5c0
--- /dev/null
+++ b/pulsar-common/src/test/resources/cert-signed
@@ -0,0 +1,22 @@
+-----BEGIN CERTIFICATE-----
+MIIDjzCCAncCCQCjDerbj9I77TANBgkqhkiG9w0BAQUFADANMQswCQYDVQQGEwJj
+bjAgFw0yMDA0MjgxMzI4NDJaGA8yMTIwMDQwNDEzMjg0MlowbzEQMA4GA1UEBhMH
+VW5rbm93bjEQMA4GA1UECBMHVW5rbm93bjEQMA4GA1UEBxMHVW5rbm93bjEQMA4G
+A1UEChMHVW5rbm93bjEQMA4GA1UECxMHVW5rbm93bjETMBEGA1UEAxMKY2xpZW50
+dXNlcjCCAbcwggEsBgcqhkjOOAQBMIIBHwKBgQD9f1OBHXUSKVLfSpwu7OTn9hG3
+UjzvRADDHj+AtlEmaUVdQCJR+1k9jVj6v8X1ujD2y5tVbNeBO4AdNG/yZmC3a5lQ
+paSfn+gEexAiwk+7qdf+t8Yb+DtX58aophUPBPuD9tPFHsMCNVQTWhaRMvZ1864r
+Ydcq7/IiAxmd0UgBxwIVAJdgUI8VIwvMspK5gqLrhAvwWBz1AoGBAPfhoIXWmz3e
+y7yrXDa4V7l5lK+7+jrqgvlXTAs9B4JnUVlXjrrUWU/mcQcQgYC0SRZxI+hMKBYT
+t88JMozIpuE8FnqLVHyNKOCjrh4rs6Z1kW6jfwv6ITVi8ftiegEkO8yk8b6oUZCJ
+qIPf4VrlnwaSi2ZegHtVJWQBTDv+z0kqA4GEAAKBgBmF8WdZ9Yv1Sf2qjqF19DUS
+Y3YB67B0azz+689y8lZw0tlnSuej0bBENIP6lvgC/PIPFdxvkInZOgB3TsWwkpxH
+zKbFZTo2Yg2txZ1IH4KX1QggePeybi2mE2soysZ2/r3nX2ZSOTdzDLicVo3yyKAu
+M8u14N0zBeJR9NMdOG1NMA0GCSqGSIb3DQEBBQUAA4IBAQAlf/MlmkGXvOHi68LU
+FoRoDh0UMUVUMYcpf+LicZkOveD0r5J5z6igDQZ5qT7RMfkSkM8pSl7xcuPNzNkT
+teeH29QOaTiYax+T9yAT9p/i2/DwiLbrcSdPT8UKOy5CVPHEtlreHupiezaID0Op
+IFaeuvBaI/HSbRZQ2IdCXTnXSQ+8rkrcoxDyIi9wjaEnWKwAqphgq0C9icNVMleu
+Lz3Wz51Xn03DQTH9uOtZu6kQYzfAEi7Z0hKF98TQ3BmwEwCRf+h5kE3wFbuT9QFh
+uHLeCvNlJoaajT2Qud0YWkIN+z1yVKzT3NdndmNm5SuM2Mzec3b4PHSLpmW0Cnwv
+UW4t
+-----END CERTIFICATE-----
diff --git a/pulsar-common/src/test/resources/client.keystore.jks b/pulsar-common/src/test/resources/client.keystore.jks
new file mode 100644
index 0000000..499c8be
Binary files /dev/null and b/pulsar-common/src/test/resources/client.keystore.jks differ
diff --git a/pulsar-common/src/test/resources/client.truststore.jks b/pulsar-common/src/test/resources/client.truststore.jks
new file mode 100644
index 0000000..8eaa06b
Binary files /dev/null and b/pulsar-common/src/test/resources/client.truststore.jks differ
diff --git a/pulsar-common/src/test/resources/clientKeyStorePW.txt b/pulsar-common/src/test/resources/clientKeyStorePW.txt
new file mode 100644
index 0000000..90d2950
--- /dev/null
+++ b/pulsar-common/src/test/resources/clientKeyStorePW.txt
@@ -0,0 +1 @@
+111111
diff --git a/pulsar-common/src/test/resources/clientTrustStorePW.txt b/pulsar-common/src/test/resources/clientTrustStorePW.txt
new file mode 100644
index 0000000..90d2950
--- /dev/null
+++ b/pulsar-common/src/test/resources/clientTrustStorePW.txt
@@ -0,0 +1 @@
+111111
diff --git a/pulsar-common/src/test/resources/old/broker.keystore.jks b/pulsar-common/src/test/resources/old/broker.keystore.jks
new file mode 100644
index 0000000..d4526ac
Binary files /dev/null and b/pulsar-common/src/test/resources/old/broker.keystore.jks differ
diff --git a/pulsar-common/src/test/resources/old/broker.truststore.jks b/pulsar-common/src/test/resources/old/broker.truststore.jks
new file mode 100644
index 0000000..0c0e694
Binary files /dev/null and b/pulsar-common/src/test/resources/old/broker.truststore.jks differ
diff --git a/pulsar-common/src/test/resources/old/brokerKeyStorePW.txt b/pulsar-common/src/test/resources/old/brokerKeyStorePW.txt
new file mode 100644
index 0000000..bdc331e
--- /dev/null
+++ b/pulsar-common/src/test/resources/old/brokerKeyStorePW.txt
@@ -0,0 +1 @@
+broker
diff --git a/pulsar-common/src/test/resources/old/brokerTrustStorePW.txt b/pulsar-common/src/test/resources/old/brokerTrustStorePW.txt
new file mode 100644
index 0000000..bdc331e
--- /dev/null
+++ b/pulsar-common/src/test/resources/old/brokerTrustStorePW.txt
@@ -0,0 +1 @@
+broker
diff --git a/pulsar-common/src/test/resources/old/client.keystore.jks b/pulsar-common/src/test/resources/old/client.keystore.jks
new file mode 100644
index 0000000..e5d074e
Binary files /dev/null and b/pulsar-common/src/test/resources/old/client.keystore.jks differ
diff --git a/pulsar-common/src/test/resources/old/client.truststore.jks b/pulsar-common/src/test/resources/old/client.truststore.jks
new file mode 100644
index 0000000..36f9d72
Binary files /dev/null and b/pulsar-common/src/test/resources/old/client.truststore.jks differ
diff --git a/pulsar-common/src/test/resources/old/clientKeyStorePW.txt b/pulsar-common/src/test/resources/old/clientKeyStorePW.txt
new file mode 100644
index 0000000..b051c6c
--- /dev/null
+++ b/pulsar-common/src/test/resources/old/clientKeyStorePW.txt
@@ -0,0 +1 @@
+client
diff --git a/pulsar-common/src/test/resources/old/clientTrustStorePW.txt b/pulsar-common/src/test/resources/old/clientTrustStorePW.txt
new file mode 100644
index 0000000..b051c6c
--- /dev/null
+++ b/pulsar-common/src/test/resources/old/clientTrustStorePW.txt
@@ -0,0 +1 @@
+client
diff --git a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/ServiceChannelInitializer.java b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/ServiceChannelInitializer.java
index 6052f2b..250259b 100644
--- a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/ServiceChannelInitializer.java
+++ b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/ServiceChannelInitializer.java
@@ -18,8 +18,11 @@
  */
 package org.apache.pulsar.discovery.service;
 
+import io.netty.handler.ssl.SslHandler;
 import org.apache.pulsar.common.protocol.Commands;
-import org.apache.pulsar.common.util.NettySslContextBuilder;
+import org.apache.pulsar.common.util.NettyServerSslContextBuilder;
+import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder;
+import org.apache.pulsar.common.util.keystoretls.NettySSLContextAutoRefreshBuilder;
 import org.apache.pulsar.discovery.service.server.ServiceConfig;
 
 import io.netty.channel.ChannelInitializer;
@@ -36,19 +39,38 @@ public class ServiceChannelInitializer extends ChannelInitializer<SocketChannel>
     public static final String TLS_HANDLER = "tls";
     private final DiscoveryService discoveryService;
     private final boolean enableTls;
-    private final NettySslContextBuilder sslCtxRefresher;
+    private final boolean tlsEnabledWithKeyStore;
+    private SslContextAutoRefreshBuilder<SslContext> sslCtxRefresher;
+    private NettySSLContextAutoRefreshBuilder nettySSLContextAutoRefreshBuilder;
 
     public ServiceChannelInitializer(DiscoveryService discoveryService, ServiceConfig serviceConfig, boolean e)
             throws Exception {
         super();
         this.discoveryService = discoveryService;
         this.enableTls = e;
+        this.tlsEnabledWithKeyStore = serviceConfig.isTlsEnabledWithKeyStore();
         if (this.enableTls) {
-            sslCtxRefresher = new NettySslContextBuilder(serviceConfig.isTlsAllowInsecureConnection(),
-                    serviceConfig.getTlsTrustCertsFilePath(), serviceConfig.getTlsCertificateFilePath(),
-                    serviceConfig.getTlsKeyFilePath(), serviceConfig.getTlsCiphers(), serviceConfig.getTlsProtocols(),
-                    serviceConfig.getTlsRequireTrustedClientCertOnConnect(),
-                    serviceConfig.getTlsCertRefreshCheckDurationSec());
+            if (tlsEnabledWithKeyStore) {
+                nettySSLContextAutoRefreshBuilder = new NettySSLContextAutoRefreshBuilder(
+                        serviceConfig.getTlsProvider(),
+                        serviceConfig.getTlsKeyStoreType(),
+                        serviceConfig.getTlsKeyStore(),
+                        serviceConfig.getTlsKeyStorePassword(),
+                        serviceConfig.isTlsAllowInsecureConnection(),
+                        serviceConfig.getTlsTrustStoreType(),
+                        serviceConfig.getTlsTrustStore(),
+                        serviceConfig.getTlsTrustStorePassword(),
+                        serviceConfig.isTlsRequireTrustedClientCertOnConnect(),
+                        serviceConfig.getTlsCiphers(),
+                        serviceConfig.getTlsProtocols(),
+                        serviceConfig.getTlsCertRefreshCheckDurationSec());
+            } else {
+                sslCtxRefresher = new NettyServerSslContextBuilder(serviceConfig.isTlsAllowInsecureConnection(),
+                        serviceConfig.getTlsTrustCertsFilePath(), serviceConfig.getTlsCertificateFilePath(),
+                        serviceConfig.getTlsKeyFilePath(), serviceConfig.getTlsCiphers(), serviceConfig.getTlsProtocols(),
+                        serviceConfig.isTlsRequireTrustedClientCertOnConnect(),
+                        serviceConfig.getTlsCertRefreshCheckDurationSec());
+            }
         } else {
             this.sslCtxRefresher = null;
         }
@@ -57,9 +79,14 @@ public class ServiceChannelInitializer extends ChannelInitializer<SocketChannel>
     @Override
     protected void initChannel(SocketChannel ch) throws Exception {
         if (sslCtxRefresher != null && this.enableTls) {
-            SslContext sslContext = sslCtxRefresher.get();
-            if (sslContext != null) {
-                ch.pipeline().addLast(TLS_HANDLER, sslContext.newHandler(ch.alloc()));
+            if (this.tlsEnabledWithKeyStore) {
+                ch.pipeline().addLast(TLS_HANDLER,
+                        new SslHandler(nettySSLContextAutoRefreshBuilder.get().createSSLEngine()));
+            } else{
+                SslContext sslContext = sslCtxRefresher.get();
+                if (sslContext != null) {
+                    ch.pipeline().addLast(TLS_HANDLER, sslContext.newHandler(ch.alloc()));
+                }
             }
         }
         ch.pipeline().addLast("frameDecoder", new LengthFieldBasedFrameDecoder(
diff --git a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServerManager.java b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServerManager.java
index 945074c..bafc4ac 100644
--- a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServerManager.java
+++ b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServerManager.java
@@ -19,17 +19,16 @@
 package org.apache.pulsar.discovery.service.server;
 
 import com.google.common.collect.Lists;
-
 import java.net.URI;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.TimeZone;
-
 import javax.servlet.Servlet;
 
 import org.apache.pulsar.common.util.SecurityUtility;
 import org.apache.pulsar.discovery.service.web.RestException;
+import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
@@ -74,14 +73,30 @@ public class ServerManager {
 
         if (config.getWebServicePortTls().isPresent()) {
             try {
-                SslContextFactory sslCtxFactory = SecurityUtility.createSslContextFactory(
-                        config.isTlsAllowInsecureConnection(),
-                        config.getTlsTrustCertsFilePath(),
-                        config.getTlsCertificateFilePath(),
-                        config.getTlsKeyFilePath(),
-                        config.getTlsRequireTrustedClientCertOnConnect(),
-                        true,
-                        config.getTlsCertRefreshCheckDurationSec());
+                SslContextFactory sslCtxFactory;
+                if (config.isTlsEnabledWithKeyStore()) {
+                    sslCtxFactory = KeyStoreSSLContext.createSslContextFactory(
+                            config.getTlsProvider(),
+                            config.getTlsKeyStoreType(),
+                            config.getTlsKeyStore(),
+                            config.getTlsKeyStorePassword(),
+                            config.isTlsAllowInsecureConnection(),
+                            config.getTlsTrustStoreType(),
+                            config.getTlsTrustStore(),
+                            config.getTlsTrustStorePassword(),
+                            config.isTlsRequireTrustedClientCertOnConnect(),
+                            config.getTlsCertRefreshCheckDurationSec()
+                    );
+                } else {
+                    sslCtxFactory = SecurityUtility.createSslContextFactory(
+                            config.isTlsAllowInsecureConnection(),
+                            config.getTlsTrustCertsFilePath(),
+                            config.getTlsCertificateFilePath(),
+                            config.getTlsKeyFilePath(),
+                            config.isTlsRequireTrustedClientCertOnConnect(),
+                            true,
+                            config.getTlsCertRefreshCheckDurationSec());
+                }
                 connectorTls = new ServerConnector(server, 1, 1, sslCtxFactory);
                 connectorTls.setPort(config.getWebServicePortTls().get());
                 connectors.add(connectorTls);
diff --git a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServiceConfig.java b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServiceConfig.java
index fa750ff..d8c85df 100644
--- a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServiceConfig.java
+++ b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServiceConfig.java
@@ -22,6 +22,7 @@ import java.util.Optional;
 import java.util.Properties;
 import java.util.Set;
 
+import lombok.Data;
 import org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider;
 import org.apache.pulsar.common.configuration.PulsarConfiguration;
 import org.apache.pulsar.discovery.service.web.DiscoveryServiceServlet;
@@ -32,6 +33,7 @@ import com.google.common.collect.Sets;
  * Service Configuration to start :{@link DiscoveryServiceServlet}
  *
  */
+@Data
 public class ServiceConfig implements PulsarConfiguration {
 
     // Local-Zookeeper quorum connection string
@@ -78,7 +80,7 @@ public class ServiceConfig implements PulsarConfiguration {
     /***** --- TLS --- ****/
     @Deprecated
     private boolean tlsEnabled = false;
-    // Tls cert refresh duration in seconds (set 0 to check on every new connection) 
+    // Tls cert refresh duration in seconds (set 0 to check on every new connection)
     private long tlsCertRefreshCheckDurationSec = 300;
     // Path for the TLS certificate file
     private String tlsCertificateFilePath;
@@ -98,209 +100,27 @@ public class ServiceConfig implements PulsarConfiguration {
     // Reject the Connection if the Client Certificate is not trusted.
     private boolean tlsRequireTrustedClientCertOnConnect = false;
 
-    private Properties properties = new Properties();
-
-    public String getZookeeperServers() {
-        return zookeeperServers;
-    }
+    /***** --- TLS with KeyStore--- ****/
+    // Enable TLS with KeyStore type configuration in broker
+    private boolean tlsEnabledWithKeyStore = false;
+    // TLS Provider
+    private String tlsProvider = null;
+    // TLS KeyStore type configuration in broker: JKS, PKCS12
+    private String tlsKeyStoreType = "JKS";
+    // TLS KeyStore path in broker
+    private String tlsKeyStore = null;
+    // TLS KeyStore password in broker
+    private String tlsKeyStorePassword = null;
+    // TLS TrustStore type configuration in broker: JKS, PKCS12
+    private String tlsTrustStoreType = "JKS";
+    // TLS TrustStore path in broker
+    private String tlsTrustStore = null;
+    // TLS TrustStore password in broker"
+    private String tlsTrustStorePassword = null;
 
-    public void setZookeeperServers(String zookeeperServers) {
-        this.zookeeperServers = zookeeperServers;
-    }
-
-    @Deprecated
-    public String getGlobalZookeeperServers() {
-        return globalZookeeperServers;
-    }
-
-    @Deprecated
-    public void setGlobalZookeeperServers(String globalZookeeperServers) {
-        this.globalZookeeperServers = globalZookeeperServers;
-    }
+    private Properties properties = new Properties();
 
     public String getConfigurationStoreServers() {
         return null == configurationStoreServers ? getGlobalZookeeperServers() : configurationStoreServers;
     }
-
-    public void setConfigurationStoreServers(String configurationStoreServers) {
-        this.configurationStoreServers = configurationStoreServers;
-    }
-
-    public int getZookeeperSessionTimeoutMs() {
-        return zookeeperSessionTimeoutMs;
-    }
-
-    public void setZookeeperSessionTimeoutMs(int zookeeperSessionTimeoutMs) {
-        this.zookeeperSessionTimeoutMs = zookeeperSessionTimeoutMs;
-    }
-
-    public Optional<Integer> getServicePort() {
-        return servicePort;
-    }
-
-    public void setServicePort(Optional<Integer> servicePort) {
-        this.servicePort = servicePort;
-    }
-
-    public Optional<Integer> getServicePortTls() {
-        return servicePortTls;
-    }
-
-    public void setServicePortTls(Optional<Integer> servicePortTls) {
-        this.servicePortTls = servicePortTls;
-    }
-
-    public Optional<Integer> getWebServicePort() {
-        return webServicePort;
-    }
-
-    public void setWebServicePort(Optional<Integer> webServicePort) {
-        this.webServicePort = webServicePort;
-    }
-
-    public Optional<Integer> getWebServicePortTls() {
-        return webServicePortTls;
-    }
-
-    public void setWebServicePortTls(Optional<Integer> webServicePortTls) {
-        this.webServicePortTls = webServicePortTls;
-    }
-
-    @Deprecated
-    public boolean isTlsEnabled() {
-        return tlsEnabled || webServicePortTls != null || servicePortTls != null;
-    }
-
-    @Deprecated
-    public void setTlsEnabled(boolean tlsEnabled) {
-        this.tlsEnabled = tlsEnabled;
-    }
-
-    public String getTlsCertificateFilePath() {
-        return tlsCertificateFilePath;
-    }
-
-    public void setTlsCertificateFilePath(String tlsCertificateFilePath) {
-        this.tlsCertificateFilePath = tlsCertificateFilePath;
-    }
-
-    public String getTlsKeyFilePath() {
-        return tlsKeyFilePath;
-    }
-
-    public void setTlsKeyFilePath(String tlsKeyFilePath) {
-        this.tlsKeyFilePath = tlsKeyFilePath;
-    }
-
-    public String getTlsTrustCertsFilePath() {
-        return tlsTrustCertsFilePath;
-    }
-
-    public void setTlsTrustCertsFilePath(String tlsTrustCertsFilePath) {
-        this.tlsTrustCertsFilePath = tlsTrustCertsFilePath;
-    }
-
-    public boolean isTlsAllowInsecureConnection() {
-        return tlsAllowInsecureConnection;
-    }
-
-    public void setTlsAllowInsecureConnection(boolean tlsAllowInsecureConnection) {
-        this.tlsAllowInsecureConnection = tlsAllowInsecureConnection;
-    }
-
-    public boolean isBindOnLocalhost() {
-        return bindOnLocalhost;
-    }
-
-    public void setBindOnLocalhost(boolean bindOnLocalhost) {
-        this.bindOnLocalhost = bindOnLocalhost;
-    }
-
-    public boolean isAuthenticationEnabled() {
-        return authenticationEnabled;
-    }
-
-    public void setAuthenticationEnabled(boolean authenticationEnabled) {
-        this.authenticationEnabled = authenticationEnabled;
-    }
-
-    public Set<String> getAuthenticationProviders() {
-        return authenticationProviders;
-    }
-
-    public void setAuthenticationProviders(Set<String> authenticationProviders) {
-        this.authenticationProviders = authenticationProviders;
-    }
-
-    public boolean isAuthorizationEnabled() {
-        return authorizationEnabled;
-    }
-
-    public void setAuthorizationEnabled(boolean authorizationEnabled) {
-        this.authorizationEnabled = authorizationEnabled;
-    }
-
-    public String getAuthorizationProvider() {
-        return authorizationProvider;
-    }
-
-    public void setAuthorizationProvider(String authorizationProvider) {
-        this.authorizationProvider = authorizationProvider;
-    }
-
-    public Set<String> getSuperUserRoles() {
-        return superUserRoles;
-    }
-
-    public void setSuperUserRoles(Set<String> superUserRoles) {
-        this.superUserRoles = superUserRoles;
-    }
-
-    public boolean getAuthorizationAllowWildcardsMatching() {
-        return authorizationAllowWildcardsMatching;
-    }
-
-    public void setAuthorizationAllowWildcardsMatching(boolean authorizationAllowWildcardsMatching) {
-        this.authorizationAllowWildcardsMatching = authorizationAllowWildcardsMatching;
-    }
-
-    public Properties getProperties() {
-        return properties;
-    }
-
-    public void setProperties(Properties properties) {
-        this.properties = properties;
-    }
-
-    public Set<String> getTlsProtocols() {
-        return tlsProtocols;
-    }
-
-    public void setTlsProtocols(Set<String> tlsProtocols) {
-        this.tlsProtocols = tlsProtocols;
-    }
-
-    public long getTlsCertRefreshCheckDurationSec() {
-        return tlsCertRefreshCheckDurationSec;
-    }
-
-    public void setTlsCertRefreshCheckDurationSec(long tlsCertRefreshCheckDurationSec) {
-        this.tlsCertRefreshCheckDurationSec = tlsCertRefreshCheckDurationSec;
-    }
-
-    public Set<String> getTlsCiphers() {
-        return tlsCiphers;
-    }
-
-    public void setTlsCiphers(Set<String> tlsCiphers) {
-        this.tlsCiphers = tlsCiphers;
-    }
-
-    public boolean getTlsRequireTrustedClientCertOnConnect() {
-        return tlsRequireTrustedClientCertOnConnect;
-    }
-
-    public void setTlsRequireTrustedClientCertOnConnect(boolean tlsRequireTrustedClientCertOnConnect) {
-        this.tlsRequireTrustedClientCertOnConnect = tlsRequireTrustedClientCertOnConnect;
-    }
 }
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
index c2d9eca..10e96b8 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java
@@ -33,11 +33,13 @@ import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
-import io.netty.handler.ssl.SslContext;
 import io.netty.handler.ssl.SslHandler;
 import io.netty.util.concurrent.Future;
 import io.netty.util.concurrent.FutureListener;
 
+import java.util.function.Supplier;
+import lombok.Getter;
+
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.Map;
@@ -70,18 +72,18 @@ public class DirectProxyHandler {
     public static final String TLS_HANDLER = "tls";
 
     private final Authentication authentication;
-    private final SslContext sslCtx;
+    private final Supplier<SslHandler> sslHandlerSupplier;
     private AuthenticationDataProvider authenticationDataProvider;
 
     public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection, String targetBrokerUrl,
-            int protocolVersion, SslContext sslCtx) {
+            int protocolVersion, Supplier<SslHandler> sslHandlerSupplier) {
         this.authentication = proxyConnection.getClientAuthentication();
         this.inboundChannel = proxyConnection.ctx().channel();
         this.originalPrincipal = proxyConnection.clientAuthRole;
         this.clientAuthData = proxyConnection.clientAuthData;
         this.clientAuthMethod = proxyConnection.clientAuthMethod;
         this.protocolVersion = protocolVersion;
-        this.sslCtx = sslCtx;
+        this.sslHandlerSupplier = sslHandlerSupplier;
         ProxyConfiguration config = service.getConfiguration();
 
         // Start the connection attempt.
@@ -94,8 +96,8 @@ public class DirectProxyHandler {
         b.handler(new ChannelInitializer<SocketChannel>() {
             @Override
             protected void initChannel(SocketChannel ch) throws Exception {
-                if (sslCtx != null) {
-                    ch.pipeline().addLast(TLS_HANDLER, sslCtx.newHandler(ch.alloc()));
+                if (sslHandlerSupplier != null) {
+                    ch.pipeline().addLast(TLS_HANDLER, sslHandlerSupplier.get());
                 }
                 ch.pipeline().addLast("frameDecoder", new LengthFieldBasedFrameDecoder(
                     Commands.DEFAULT_MAX_MESSAGE_SIZE + Commands.MESSAGE_SIZE_FRAME_PADDING, 0, 4, 0, 4));
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 6a293a0..34beaa4 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
@@ -60,6 +60,8 @@ public class ProxyConfiguration implements PulsarConfiguration {
     @Category
     private static final String CATEGORY_TLS = "TLS";
     @Category
+    private static final String CATEGORY_KEYSTORE_TLS = "KeyStoreTLS";
+    @Category
     private static final String CATEGORY_TOKEN_AUTH = "Token Authentication Provider";
     @Category
     private static final String CATEGORY_HTTP = "HTTP";
@@ -314,7 +316,7 @@ public class ProxyConfiguration implements PulsarConfiguration {
     private Set<String> tlsProtocols = Sets.newTreeSet();
     @FieldContext(
         category = CATEGORY_TLS,
-        doc = "Specify the tls cipher the broker will use to negotiate during TLS Handshake"
+        doc = "Specify the tls cipher the proxy will use to negotiate during TLS Handshake"
             + " (a comma-separated list of ciphers).\n\n"
             + "Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256]"
     )
@@ -326,6 +328,106 @@ public class ProxyConfiguration implements PulsarConfiguration {
     )
     private boolean tlsRequireTrustedClientCertOnConnect = false;
 
+    /**** --- KeyStore TLS config variables --- ****/
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "Enable TLS with KeyStore type configuration for proxy"
+    )
+    private boolean tlsEnabledWithKeyStore = false;
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS Provider"
+    )
+    private String tlsProvider = null;
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS KeyStore type configuration for proxy: JKS, PKCS12"
+    )
+    private String tlsKeyStoreType = "JKS";
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS KeyStore path for proxy"
+    )
+    private String tlsKeyStore = null;
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS KeyStore password for proxy"
+    )
+    private String tlsKeyStorePassword = null;
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS TrustStore type configuration for proxy: JKS, PKCS12"
+    )
+    private String tlsTrustStoreType = "JKS";
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS TrustStore path for proxy"
+    )
+    private String tlsTrustStore = null;
+
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS TrustStore password for proxy"
+    )
+    private String tlsTrustStorePassword = null;
+
+    /**** --- KeyStore TLS config variables used for proxy to auth with broker--- ****/
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "Whether the Pulsar proxy use KeyStore type to authenticate with Pulsar brokers"
+    )
+    private boolean brokerClientTlsEnabledWithKeyStore = false;
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "The TLS Provider used by the Pulsar proxy to authenticate with Pulsar brokers"
+    )
+    private String brokerClientSslProvider = null;
+
+    // needed when client auth is required
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS TrustStore type configuration for proxy: JKS, PKCS12 "
+                  + " used by the Pulsar proxy to authenticate with Pulsar brokers"
+    )
+    private String brokerClientTlsTrustStoreType = "JKS";
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS TrustStore path for proxy, "
+                  + " used by the Pulsar proxy to authenticate with Pulsar brokers"
+    )
+    private String brokerClientTlsTrustStore = null;
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "TLS TrustStore password for proxy, "
+                  + " used by the Pulsar proxy to authenticate with Pulsar brokers"
+    )
+    private String brokerClientTlsTrustStorePassword = null;
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_TLS,
+            doc = "Specify the tls cipher the proxy will use to negotiate during TLS Handshake"
+                  + " (a comma-separated list of ciphers).\n\n"
+                  + "Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256].\n"
+                  + " used by the Pulsar proxy to authenticate with Pulsar brokers"
+    )
+    private Set<String> brokerClientTlsCiphers = Sets.newTreeSet();
+    @FieldContext(
+            category = CATEGORY_KEYSTORE_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] \n"
+                  + " used by the Pulsar proxy to authenticate with Pulsar brokers"
+    )
+    private Set<String> brokerClientTlsProtocols = Sets.newTreeSet();
+
+    /***** --- HTTP --- ****/
+
     @FieldContext(
         category = CATEGORY_HTTP,
         doc = "Http directs to redirect to non-pulsar services"
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
index 9b8b7fb..929b919 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java
@@ -23,6 +23,7 @@ import static com.google.common.base.Preconditions.checkArgument;
 import java.net.SocketAddress;
 import java.util.concurrent.TimeUnit;
 
+import java.util.function.Supplier;
 import javax.naming.AuthenticationException;
 import javax.net.ssl.SSLSession;
 
@@ -55,7 +56,6 @@ import org.slf4j.LoggerFactory;
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.ChannelHandler;
 import io.netty.channel.ChannelHandlerContext;
-import io.netty.handler.ssl.SslContext;
 import io.netty.handler.ssl.SslHandler;
 import io.netty.util.concurrent.Future;
 import io.netty.util.concurrent.FutureListener;
@@ -71,7 +71,7 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
     private Authentication clientAuthentication;
     AuthenticationDataSource authenticationData;
     private State state;
-    private final SslContext sslCtx;
+    private final Supplier<SslHandler> sslHandlerSupplier;
 
     private LookupProxyHandler lookupProxyHandler = null;
     private DirectProxyHandler directProxyHandler = null;
@@ -110,11 +110,11 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
         return client.getCnxPool();
     }
 
-    public ProxyConnection(ProxyService proxyService, SslContext sslCtx) {
+    public ProxyConnection(ProxyService proxyService, Supplier<SslHandler> sslHandlerSupplier) {
         super(30, TimeUnit.SECONDS);
         this.service = proxyService;
         this.state = State.Init;
-        this.sslCtx = sslCtx;
+        this.sslHandlerSupplier = sslHandlerSupplier;
     }
 
     @Override
@@ -209,7 +209,7 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
             // connection there and just pass bytes in both directions
             state = State.ProxyConnectionToBroker;
             directProxyHandler = new DirectProxyHandler(service, this, proxyToBrokerUrl,
-                protocolVersionToAdvertise, sslCtx);
+                protocolVersionToAdvertise, sslHandlerSupplier);
             cancelKeepAliveTask();
         } else {
             // Client is doing a lookup, we can consider the handshake complete
@@ -408,8 +408,15 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
         }
         if (proxyConfig.isTlsEnabledWithBroker()) {
             clientConf.setUseTls(true);
-            clientConf.setTlsTrustCertsFilePath(proxyConfig.getBrokerClientTrustCertsFilePath());
-            clientConf.setTlsAllowInsecureConnection(proxyConfig.isTlsAllowInsecureConnection());
+            if (proxyConfig.isBrokerClientTlsEnabledWithKeyStore()) {
+                clientConf.setUseKeyStoreTls(true);
+                clientConf.setTlsTrustStoreType(proxyConfig.getBrokerClientTlsTrustStoreType());
+                clientConf.setTlsTrustStorePath(proxyConfig.getBrokerClientTlsTrustStore());
+                clientConf.setTlsTrustStorePassword(proxyConfig.getBrokerClientTlsTrustStorePassword());
+            } else {
+                clientConf.setTlsTrustCertsFilePath(proxyConfig.getBrokerClientTrustCertsFilePath());
+                clientConf.setTlsAllowInsecureConnection(proxyConfig.isTlsAllowInsecureConnection());
+            }
         }
         return clientConf;
     }
diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java
index 156492a..42a5c07 100644
--- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java
+++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java
@@ -20,16 +20,20 @@ package org.apache.pulsar.proxy.server;
 
 import static org.apache.commons.lang3.StringUtils.isEmpty;
 
+import io.netty.handler.ssl.SslHandler;
+import java.util.function.Supplier;
 import org.apache.pulsar.client.api.AuthenticationDataProvider;
 import org.apache.pulsar.client.api.AuthenticationFactory;
 import org.apache.pulsar.common.protocol.Commands;
-import org.apache.pulsar.common.util.ClientSslContextRefresher;
-import org.apache.pulsar.common.util.NettySslContextBuilder;
+import org.apache.pulsar.common.util.NettyClientSslContextRefresher;
+import org.apache.pulsar.common.util.NettyServerSslContextBuilder;
 
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
 import io.netty.handler.ssl.SslContext;
+import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder;
+import org.apache.pulsar.common.util.keystoretls.NettySSLContextAutoRefreshBuilder;
 
 /**
  * Initialize service channel handlers.
@@ -39,22 +43,43 @@ public class ServiceChannelInitializer extends ChannelInitializer<SocketChannel>
 
     public static final String TLS_HANDLER = "tls";
     private final ProxyService proxyService;
-    private final NettySslContextBuilder serverSslCtxRefresher;
-    private final ClientSslContextRefresher clientSslCtxRefresher;
     private final boolean enableTls;
+    private final boolean tlsEnabledWithKeyStore;
+
+    private SslContextAutoRefreshBuilder<SslContext> serverSslCtxRefresher;
+    private SslContextAutoRefreshBuilder<SslContext> clientSslCtxRefresher;
+    private NettySSLContextAutoRefreshBuilder serverSSLContextAutoRefreshBuilder;
+    private NettySSLContextAutoRefreshBuilder clientSSLContextAutoRefreshBuilder;
 
     public ServiceChannelInitializer(ProxyService proxyService, ProxyConfiguration serviceConfig, boolean enableTls)
             throws Exception {
         super();
         this.proxyService = proxyService;
         this.enableTls = enableTls;
+        this.tlsEnabledWithKeyStore = serviceConfig.isTlsEnabledWithKeyStore();
 
         if (enableTls) {
-            serverSslCtxRefresher = new NettySslContextBuilder(serviceConfig.isTlsAllowInsecureConnection(),
-                    serviceConfig.getTlsTrustCertsFilePath(), serviceConfig.getTlsCertificateFilePath(),
-                    serviceConfig.getTlsKeyFilePath(), serviceConfig.getTlsCiphers(), serviceConfig.getTlsProtocols(),
-                    serviceConfig.isTlsRequireTrustedClientCertOnConnect(),
-                    serviceConfig.getTlsCertRefreshCheckDurationSec());
+            if (tlsEnabledWithKeyStore) {
+                serverSSLContextAutoRefreshBuilder = new NettySSLContextAutoRefreshBuilder(
+                        serviceConfig.getTlsProvider(),
+                        serviceConfig.getTlsKeyStoreType(),
+                        serviceConfig.getTlsKeyStore(),
+                        serviceConfig.getTlsKeyStorePassword(),
+                        serviceConfig.isTlsAllowInsecureConnection(),
+                        serviceConfig.getTlsTrustStoreType(),
+                        serviceConfig.getTlsTrustStore(),
+                        serviceConfig.getTlsTrustStorePassword(),
+                        serviceConfig.isTlsRequireTrustedClientCertOnConnect(),
+                        serviceConfig.getTlsCiphers(),
+                        serviceConfig.getTlsProtocols(),
+                        serviceConfig.getTlsCertRefreshCheckDurationSec());
+            } else {
+                serverSslCtxRefresher = new NettyServerSslContextBuilder(serviceConfig.isTlsAllowInsecureConnection(),
+                        serviceConfig.getTlsTrustCertsFilePath(), serviceConfig.getTlsCertificateFilePath(),
+                        serviceConfig.getTlsKeyFilePath(), serviceConfig.getTlsCiphers(), serviceConfig.getTlsProtocols(),
+                        serviceConfig.isTlsRequireTrustedClientCertOnConnect(),
+                        serviceConfig.getTlsCertRefreshCheckDurationSec());
+            }
         } else {
             this.serverSslCtxRefresher = null;
         }
@@ -67,9 +92,24 @@ public class ServiceChannelInitializer extends ChannelInitializer<SocketChannel>
                         serviceConfig.getBrokerClientAuthenticationParameters()).getAuthData();
             }
 
-            clientSslCtxRefresher = new ClientSslContextRefresher(serviceConfig.isTlsAllowInsecureConnection(),
-                    serviceConfig.getBrokerClientTrustCertsFilePath(), authData);
-
+            if (tlsEnabledWithKeyStore) {
+                clientSSLContextAutoRefreshBuilder = new NettySSLContextAutoRefreshBuilder(
+                        serviceConfig.getBrokerClientSslProvider(),
+                        serviceConfig.isTlsAllowInsecureConnection(),
+                        serviceConfig.getBrokerClientTlsTrustStoreType(),
+                        serviceConfig.getBrokerClientTlsTrustStore(),
+                        serviceConfig.getBrokerClientTlsTrustStorePassword(),
+                        serviceConfig.getBrokerClientTlsCiphers(),
+                        serviceConfig.getBrokerClientTlsProtocols(),
+                        serviceConfig.getTlsCertRefreshCheckDurationSec(),
+                        authData);
+            } else {
+                clientSslCtxRefresher = new NettyClientSslContextRefresher(
+                        serviceConfig.isTlsAllowInsecureConnection(),
+                        serviceConfig.getBrokerClientTrustCertsFilePath(),
+                        authData,
+                        serviceConfig.getTlsCertRefreshCheckDurationSec());
+            }
         } else {
             this.clientSslCtxRefresher = null;
         }
@@ -82,11 +122,33 @@ public class ServiceChannelInitializer extends ChannelInitializer<SocketChannel>
             if (sslContext != null) {
                 ch.pipeline().addLast(TLS_HANDLER, sslContext.newHandler(ch.alloc()));
             }
+        } else if (this.tlsEnabledWithKeyStore && serverSSLContextAutoRefreshBuilder != null) {
+            ch.pipeline().addLast(TLS_HANDLER,
+                    new SslHandler(serverSSLContextAutoRefreshBuilder.get().createSSLEngine()));
         }
 
         ch.pipeline().addLast("frameDecoder", new LengthFieldBasedFrameDecoder(
-            Commands.DEFAULT_MAX_MESSAGE_SIZE + Commands.MESSAGE_SIZE_FRAME_PADDING, 0, 4, 0, 4));
+                Commands.DEFAULT_MAX_MESSAGE_SIZE + Commands.MESSAGE_SIZE_FRAME_PADDING, 0, 4, 0, 4));
+
+        Supplier<SslHandler> sslHandlerSupplier = null;
+        if (clientSslCtxRefresher != null) {
+            sslHandlerSupplier = new Supplier<SslHandler>() {
+                @Override
+                public SslHandler get() {
+                    return clientSslCtxRefresher.get().newHandler(ch.alloc());
+                }
+            };
+        } else if (clientSSLContextAutoRefreshBuilder != null) {
+            sslHandlerSupplier = new Supplier<SslHandler>() {
+                @Override
+                public SslHandler get() {
+                    return new SslHandler(clientSSLContextAutoRefreshBuilder.get().createSSLEngine());
+                }
+            };
+        }
+
         ch.pipeline().addLast("handler",
-                new ProxyConnection(proxyService, clientSslCtxRefresher == null ? null : clientSslCtxRefresher.get()));
+                new ProxyConnection(proxyService, sslHandlerSupplier));
+
     }
 }
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 0b8dca2..3a45399 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
@@ -19,9 +19,7 @@
 package org.apache.pulsar.proxy.server;
 
 import com.google.common.collect.Lists;
-
 import io.prometheus.client.jetty.JettyStatisticsCollector;
-
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
@@ -31,15 +29,14 @@ import java.util.EnumSet;
 import java.util.List;
 import java.util.Optional;
 import java.util.TimeZone;
-
 import javax.servlet.DispatcherType;
-
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.pulsar.broker.authentication.AuthenticationService;
 import org.apache.pulsar.broker.web.AuthenticationFilter;
 import org.apache.pulsar.broker.web.JsonMapperProvider;
 import org.apache.pulsar.broker.web.WebExecutorThreadPool;
 import org.apache.pulsar.common.util.SecurityUtility;
+import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext;
 import org.eclipse.jetty.server.Connector;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.HttpConfiguration;
@@ -99,14 +96,30 @@ public class WebServer {
         }
         if (config.getWebServicePortTls().isPresent()) {
             try {
-                SslContextFactory sslCtxFactory = SecurityUtility.createSslContextFactory(
-                        config.isTlsAllowInsecureConnection(),
-                        config.getTlsTrustCertsFilePath(),
-                        config.getTlsCertificateFilePath(),
-                        config.getTlsKeyFilePath(),
-                        config.isTlsRequireTrustedClientCertOnConnect(),
-                        true,
-                        config.getTlsCertRefreshCheckDurationSec());
+                SslContextFactory sslCtxFactory;
+                if (config.isTlsEnabledWithKeyStore()) {
+                    sslCtxFactory = KeyStoreSSLContext.createSslContextFactory(
+                            config.getTlsProvider(),
+                            config.getTlsKeyStoreType(),
+                            config.getTlsKeyStore(),
+                            config.getTlsKeyStorePassword(),
+                            config.isTlsAllowInsecureConnection(),
+                            config.getTlsTrustStoreType(),
+                            config.getTlsTrustStore(),
+                            config.getTlsTrustStorePassword(),
+                            config.isTlsRequireTrustedClientCertOnConnect(),
+                            config.getTlsCertRefreshCheckDurationSec()
+                    );
+                } else {
+                    sslCtxFactory = SecurityUtility.createSslContextFactory(
+                            config.isTlsAllowInsecureConnection(),
+                            config.getTlsTrustCertsFilePath(),
+                            config.getTlsCertificateFilePath(),
+                            config.getTlsKeyFilePath(),
+                            config.isTlsRequireTrustedClientCertOnConnect(),
+                            true,
+                            config.getTlsCertRefreshCheckDurationSec());
+                }
                 connectorTls = new ServerConnector(server, 1, 1, sslCtxFactory);
                 connectorTls.setPort(config.getWebServicePortTls().get());
                 connectors.add(connectorTls);
diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithAuth.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithAuth.java
new file mode 100644
index 0000000..51354ed
--- /dev/null
+++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithAuth.java
@@ -0,0 +1,202 @@
+/**
+ * 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 com.google.common.base.Preconditions.checkNotNull;
+import static org.mockito.Mockito.doReturn;
+
+import com.google.common.collect.Sets;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import lombok.Cleanup;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.authentication.AuthenticationProviderTls;
+import org.apache.pulsar.broker.authentication.AuthenticationService;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageRoutingMode;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls;
+import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
+import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class ProxyKeyStoreTlsTestWithAuth extends MockedPulsarServiceBaseTest {
+    protected final String BROKER_KEYSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/broker.keystore.jks";
+    protected final String BROKER_TRUSTSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/broker.truststore.jks";
+    protected final String BROKER_KEYSTORE_PW = "111111";
+    protected final String BROKER_TRUSTSTORE_PW = "111111";
+
+    protected final String CLIENT_KEYSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/client.keystore.jks";
+    protected final String CLIENT_TRUSTSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/client.truststore.jks";
+    protected final String CLIENT_KEYSTORE_PW = "111111";
+    protected final String CLIENT_TRUSTSTORE_PW = "111111";
+
+    protected final String CLIENT_KEYSTORE_CN = "clientuser";
+    protected final String KEYSTORE_TYPE = "JKS";
+
+    private final String DUMMY_VALUE = "DUMMY_VALUE";
+
+    private ProxyService proxyService;
+    private ProxyConfiguration proxyConfig = new ProxyConfiguration();
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        internalSetup();
+
+        proxyConfig.setServicePort(Optional.of(0));
+        proxyConfig.setServicePortTls(Optional.of(0));
+        proxyConfig.setWebServicePort(Optional.of(0));
+        proxyConfig.setWebServicePortTls(Optional.of(0));
+        proxyConfig.setTlsEnabledWithBroker(false);
+
+        proxyConfig.setTlsEnabledWithKeyStore(true);
+        proxyConfig.setTlsKeyStoreType(KEYSTORE_TYPE);
+        proxyConfig.setTlsKeyStore(BROKER_KEYSTORE_FILE_PATH);
+        proxyConfig.setTlsKeyStorePassword(BROKER_KEYSTORE_PW);
+        proxyConfig.setTlsTrustStoreType(KEYSTORE_TYPE);
+        proxyConfig.setTlsTrustStore(CLIENT_TRUSTSTORE_FILE_PATH);
+        proxyConfig.setTlsTrustStorePassword(CLIENT_TRUSTSTORE_PW);
+
+        proxyConfig.setZookeeperServers(DUMMY_VALUE);
+        proxyConfig.setConfigurationStoreServers(DUMMY_VALUE);
+
+
+        // config for authentication and authorization.
+        proxyConfig.setTlsRequireTrustedClientCertOnConnect(true);
+        proxyConfig.setSuperUserRoles(Sets.newHashSet(CLIENT_KEYSTORE_CN));
+        proxyConfig.setAuthenticationEnabled(true);
+        proxyConfig.setAuthorizationEnabled(true);
+        Set<String> providers = new HashSet<>();
+        providers.add(AuthenticationProviderTls.class.getName());
+        proxyConfig.setAuthenticationProviders(providers);
+
+        proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService(
+                                                            PulsarConfigurationLoader.convertFrom(proxyConfig))));
+        doReturn(mockZooKeeperClientFactory).when(proxyService).getZooKeeperClientFactory();
+
+        proxyService.start();
+    }
+
+    @Override
+    @AfterMethod
+    protected void cleanup() throws Exception {
+        internalCleanup();
+
+        proxyService.close();
+    }
+
+    protected PulsarClient internalSetUpForClient(boolean addCertificates, String lookupUrl) throws Exception {
+        ClientBuilder clientBuilder = PulsarClient.builder()
+                .serviceUrl(lookupUrl)
+                .enableTls(true)
+                .useKeyStoreTls(true)
+                .tlsTrustStorePath(BROKER_TRUSTSTORE_FILE_PATH)
+                .tlsTrustStorePassword(BROKER_TRUSTSTORE_PW)
+                .allowTlsInsecureConnection(false)
+                .operationTimeout(1000, TimeUnit.MILLISECONDS);
+        if (addCertificates) {
+            Map<String, String> authParams = new HashMap<>();
+            authParams.put(AuthenticationKeyStoreTls.KEYSTORE_TYPE, KEYSTORE_TYPE);
+            authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PATH, CLIENT_KEYSTORE_FILE_PATH);
+            authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PW, CLIENT_KEYSTORE_PW);
+            clientBuilder.authentication(AuthenticationKeyStoreTls.class.getName(), authParams);
+        }
+        return clientBuilder.build();
+    }
+
+    @Test
+    public void testProducer() throws Exception {
+        @Cleanup
+        PulsarClient client = internalSetUpForClient(true, proxyService.getServiceUrlTls());
+        @Cleanup
+        Producer<byte[]> producer = client.newProducer(Schema.BYTES)
+                .topic("persistent://sample/test/local/topic" + System.currentTimeMillis())
+                .create();
+
+        for (int i = 0; i < 10; i++) {
+            producer.send("test".getBytes());
+        }
+    }
+
+    @Test
+    public void testProducerFailed() throws Exception {
+        @Cleanup
+        PulsarClient client = internalSetUpForClient(false, proxyService.getServiceUrlTls());
+        try {
+            @Cleanup
+            Producer<byte[]> producer = client.newProducer(Schema.BYTES)
+                    .topic("persistent://sample/test/local/topic" + System.currentTimeMillis())
+                    .create();
+            Assert.fail("Should failed since broker setTlsRequireTrustedClientCertOnConnect, "
+                        + "while client not set keystore");
+        } catch (Exception e) {
+            // expected
+            log.info("Expected failed since broker setTlsRequireTrustedClientCertOnConnect,"
+                     + " while client not set keystore");
+        }
+    }
+
+    @Test
+    public void testPartitions() throws Exception {
+        @Cleanup
+        PulsarClient client = internalSetUpForClient(true, proxyService.getServiceUrlTls());
+        String topicName = "persistent://sample/test/local/partitioned-topic" + System.currentTimeMillis();
+        admin.tenants().createTenant("sample", new TenantInfo());
+        admin.topics().createPartitionedTopic(topicName, 2);
+
+        @Cleanup
+        Producer<byte[]> producer = client.newProducer(Schema.BYTES).topic(topicName)
+                .messageRoutingMode(MessageRoutingMode.RoundRobinPartition).create();
+
+        // Create a consumer directly attached to broker
+        @Cleanup
+        Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName)
+                .subscriptionName("my-sub").subscribe();
+
+        for (int i = 0; i < 10; i++) {
+            producer.send("test".getBytes());
+        }
+
+        for (int i = 0; i < 10; i++) {
+            Message<byte[]> msg = consumer.receive(1, TimeUnit.SECONDS);
+            checkNotNull(msg);
+        }
+    }
+
+
+}
diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithoutAuth.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithoutAuth.java
new file mode 100644
index 0000000..e6ce925
--- /dev/null
+++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithoutAuth.java
@@ -0,0 +1,186 @@
+/**
+ * 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 com.google.common.base.Preconditions.checkNotNull;
+import static org.mockito.Mockito.doReturn;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import lombok.Cleanup;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
+import org.apache.pulsar.broker.authentication.AuthenticationService;
+import org.apache.pulsar.client.api.ClientBuilder;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageRoutingMode;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls;
+import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
+import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class ProxyKeyStoreTlsTestWithoutAuth extends MockedPulsarServiceBaseTest {
+    protected final String BROKER_KEYSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/broker.keystore.jks";
+    protected final String BROKER_TRUSTSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/broker.truststore.jks";
+    protected final String BROKER_KEYSTORE_PW = "111111";
+    protected final String BROKER_TRUSTSTORE_PW = "111111";
+
+    protected final String CLIENT_KEYSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/client.keystore.jks";
+    protected final String CLIENT_TRUSTSTORE_FILE_PATH =
+            "./src/test/resources/authentication/keystoretls/client.truststore.jks";
+    protected final String CLIENT_KEYSTORE_PW = "111111";
+    protected final String CLIENT_TRUSTSTORE_PW = "111111";
+
+    protected final String KEYSTORE_TYPE = "JKS";
+
+    private final String DUMMY_VALUE = "DUMMY_VALUE";
+
+    private ProxyService proxyService;
+    private ProxyConfiguration proxyConfig = new ProxyConfiguration();
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        internalSetup();
+
+        proxyConfig.setServicePort(Optional.of(0));
+        proxyConfig.setServicePortTls(Optional.of(0));
+        proxyConfig.setWebServicePort(Optional.of(0));
+        proxyConfig.setWebServicePortTls(Optional.of(0));
+        proxyConfig.setTlsEnabledWithBroker(false);
+
+        proxyConfig.setTlsEnabledWithKeyStore(true);
+        proxyConfig.setTlsKeyStoreType(KEYSTORE_TYPE);
+        proxyConfig.setTlsKeyStore(BROKER_KEYSTORE_FILE_PATH);
+        proxyConfig.setTlsKeyStorePassword(BROKER_KEYSTORE_PW);
+        proxyConfig.setTlsTrustStoreType(KEYSTORE_TYPE);
+        proxyConfig.setTlsTrustStore(CLIENT_TRUSTSTORE_FILE_PATH);
+        proxyConfig.setTlsTrustStorePassword(CLIENT_TRUSTSTORE_PW);
+        proxyConfig.setTlsRequireTrustedClientCertOnConnect(true);
+
+        proxyConfig.setZookeeperServers(DUMMY_VALUE);
+        proxyConfig.setConfigurationStoreServers(DUMMY_VALUE);
+
+        proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService(
+                                                            PulsarConfigurationLoader.convertFrom(proxyConfig))));
+        doReturn(mockZooKeeperClientFactory).when(proxyService).getZooKeeperClientFactory();
+
+        proxyService.start();
+    }
+
+    protected PulsarClient internalSetUpForClient(boolean addCertificates, String lookupUrl) throws Exception {
+        ClientBuilder clientBuilder = PulsarClient.builder()
+                .serviceUrl(lookupUrl)
+                .enableTls(true)
+                .useKeyStoreTls(true)
+                .tlsTrustStorePath(BROKER_TRUSTSTORE_FILE_PATH)
+                .tlsTrustStorePassword(BROKER_TRUSTSTORE_PW)
+                .allowTlsInsecureConnection(false)
+                .operationTimeout(1000, TimeUnit.MILLISECONDS);
+        if (addCertificates) {
+            Map<String, String> authParams = new HashMap<>();
+            authParams.put(AuthenticationKeyStoreTls.KEYSTORE_TYPE, KEYSTORE_TYPE);
+            authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PATH, CLIENT_KEYSTORE_FILE_PATH);
+            authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PW, CLIENT_KEYSTORE_PW);
+            clientBuilder.authentication(AuthenticationKeyStoreTls.class.getName(), authParams);
+        }
+        return clientBuilder.build();
+    }
+
+    @Override
+    @AfterMethod
+    protected void cleanup() throws Exception {
+        internalCleanup();
+        proxyService.close();
+    }
+
+    @Test
+    public void testProducer() throws Exception {
+        @Cleanup
+        PulsarClient client = internalSetUpForClient(true, proxyService.getServiceUrlTls());
+        @Cleanup
+        Producer<byte[]> producer = client.newProducer(Schema.BYTES)
+                .topic("persistent://sample/test/local/topic" + System.currentTimeMillis())
+                .create();
+
+        for (int i = 0; i < 10; i++) {
+            producer.send("test".getBytes());
+        }
+    }
+
+    @Test
+    public void testProducerFailed() throws Exception {
+        @Cleanup
+        PulsarClient client = internalSetUpForClient(false, proxyService.getServiceUrlTls());
+        try {
+            @Cleanup
+            Producer<byte[]> producer = client.newProducer(Schema.BYTES)
+                    .topic("persistent://sample/test/local/topic" + System.currentTimeMillis())
+                    .create();
+            Assert.fail("Should failed since broker setTlsRequireTrustedClientCertOnConnect, "
+                        + "while client not set keystore");
+        } catch (Exception e) {
+            // expected
+            log.info("Expected failed since broker setTlsRequireTrustedClientCertOnConnect,"
+                     + " while client not set keystore");
+        }
+    }
+
+    @Test
+    public void testPartitions() throws Exception {
+        @Cleanup
+        PulsarClient client = internalSetUpForClient(true, proxyService.getServiceUrlTls());
+        String topicName = "persistent://sample/test/local/partitioned-topic" + System.currentTimeMillis();
+        admin.tenants().createTenant("sample", new TenantInfo());
+        admin.topics().createPartitionedTopic(topicName, 2);
+
+        @Cleanup
+        Producer<byte[]> producer = client.newProducer(Schema.BYTES).topic(topicName)
+                .messageRoutingMode(MessageRoutingMode.RoundRobinPartition).create();
+
+        // Create a consumer directly attached to broker
+        @Cleanup
+        Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName)
+                .subscriptionName("my-sub").subscribe();
+
+        for (int i = 0; i < 10; i++) {
+            producer.send("test".getBytes());
+        }
+
+        for (int i = 0; i < 10; i++) {
+            Message<byte[]> msg = consumer.receive(1, TimeUnit.SECONDS);
+            checkNotNull(msg);
+        }
+    }
+
+}
diff --git a/pulsar-proxy/src/test/resources/authentication/keystoretls/broker.keystore.jks b/pulsar-proxy/src/test/resources/authentication/keystoretls/broker.keystore.jks
new file mode 100644
index 0000000..b4fec69
Binary files /dev/null and b/pulsar-proxy/src/test/resources/authentication/keystoretls/broker.keystore.jks differ
diff --git a/pulsar-proxy/src/test/resources/authentication/keystoretls/broker.truststore.jks b/pulsar-proxy/src/test/resources/authentication/keystoretls/broker.truststore.jks
new file mode 100644
index 0000000..8ac03d8
Binary files /dev/null and b/pulsar-proxy/src/test/resources/authentication/keystoretls/broker.truststore.jks differ
diff --git a/pulsar-proxy/src/test/resources/authentication/keystoretls/brokerKeyStorePW.txt b/pulsar-proxy/src/test/resources/authentication/keystoretls/brokerKeyStorePW.txt
new file mode 100644
index 0000000..90d2950
--- /dev/null
+++ b/pulsar-proxy/src/test/resources/authentication/keystoretls/brokerKeyStorePW.txt
@@ -0,0 +1 @@
+111111
diff --git a/pulsar-proxy/src/test/resources/authentication/keystoretls/brokerTrustStorePW.txt b/pulsar-proxy/src/test/resources/authentication/keystoretls/brokerTrustStorePW.txt
new file mode 100644
index 0000000..90d2950
--- /dev/null
+++ b/pulsar-proxy/src/test/resources/authentication/keystoretls/brokerTrustStorePW.txt
@@ -0,0 +1 @@
+111111
diff --git a/pulsar-proxy/src/test/resources/authentication/keystoretls/client.keystore.jks b/pulsar-proxy/src/test/resources/authentication/keystoretls/client.keystore.jks
new file mode 100644
index 0000000..499c8be
Binary files /dev/null and b/pulsar-proxy/src/test/resources/authentication/keystoretls/client.keystore.jks differ
diff --git a/pulsar-proxy/src/test/resources/authentication/keystoretls/client.truststore.jks b/pulsar-proxy/src/test/resources/authentication/keystoretls/client.truststore.jks
new file mode 100644
index 0000000..8eaa06b
Binary files /dev/null and b/pulsar-proxy/src/test/resources/authentication/keystoretls/client.truststore.jks differ
diff --git a/pulsar-proxy/src/test/resources/authentication/keystoretls/clientKeyStorePW.txt b/pulsar-proxy/src/test/resources/authentication/keystoretls/clientKeyStorePW.txt
new file mode 100644
index 0000000..90d2950
--- /dev/null
+++ b/pulsar-proxy/src/test/resources/authentication/keystoretls/clientKeyStorePW.txt
@@ -0,0 +1 @@
+111111
diff --git a/pulsar-proxy/src/test/resources/authentication/keystoretls/clientTrustStorePW.txt b/pulsar-proxy/src/test/resources/authentication/keystoretls/clientTrustStorePW.txt
new file mode 100644
index 0000000..90d2950
--- /dev/null
+++ b/pulsar-proxy/src/test/resources/authentication/keystoretls/clientTrustStorePW.txt
@@ -0,0 +1 @@
+111111
diff --git a/site2/docs/reference-configuration.md b/site2/docs/reference-configuration.md
index f08656f..642a7c0 100644
--- a/site2/docs/reference-configuration.md
+++ b/site2/docs/reference-configuration.md
@@ -146,6 +146,18 @@ Pulsar brokers are responsible for handling incoming messages from producers, di
 |tlsAllowInsecureConnection|  Accept untrusted TLS certificate from client  |false|
 |tlsProtocols|Specify the tls protocols the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLSv1.2```, ```TLSv1.1```, ```TLSv1``` ||
 |tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256```||
+|tlsEnabledWithKeyStore| Enable TLS with KeyStore type configuration in broker |false|
+|tlsProvider| TLS Provider for KeyStore type ||
+|tlsKeyStoreType| LS KeyStore type configuration in broker: JKS, PKCS12 |JKS|
+|tlsKeyStore| TLS KeyStore path in broker ||
+|tlsKeyStorePassword| TLS KeyStore password for broker ||
+|brokerClientTlsEnabledWithKeyStore| Whether internal client use KeyStore type to authenticate with Pulsar brokers |false|
+|brokerClientSslProvider| The TLS Provider used by internal client to authenticate with other Pulsar brokers ||
+|brokerClientTlsTrustStoreType| TLS TrustStore type configuration for internal client: JKS, PKCS12, used by the internal client to authenticate with Pulsar brokers |JKS|
+|brokerClientTlsTrustStore| TLS TrustStore path for internal client, used by the internal client to authenticate with Pulsar brokers ||
+|brokerClientTlsTrustStorePassword| TLS TrustStore password for internal client, used by the internal client to authenticate with Pulsar brokers ||
+|brokerClientTlsCiphers| Specify the tls cipher the internal client will use to negotiate during TLS Handshake. (a comma-separated list of ciphers) e.g.  [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256]||
+|brokerClientTlsProtocols|Specify the tls protocols the broker will use to negotiate during TLS handshake. (a comma-separated list of protocol names). e.g.  [TLSv1.2, TLSv1.1, TLSv1] ||
 |tokenSecretKey| Configure the secret key to be used to validate auth tokens. The key can be specified like: `tokenSecretKey=data:base64,xxxxxxxxx` or `tokenSecretKey=file:///my/secret.key`||
 |tokenPublicKey| Configure the public key to be used to validate auth tokens. The key can be specified like: `tokenPublicKey=data:base64,xxxxxxxxx` or `tokenPublicKey=file:///my/secret.key`||
 |tokenPublicAlg| Configure the algorithm to be used to validate auth tokens. This can be any of the asymettric algorithms supported by Java JWT (https://github.com/jwtk/jjwt#signature-algorithms-keys) |RS256|


[pulsar] 02/38: Retention policy should be respected when there is no traffic (#6676)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 7f0cd1af01d74074a60ec9c6478adf9d59e63bca
Author: feynmanlin <31...@qq.com>
AuthorDate: Wed Apr 8 10:29:53 2020 +0800

    Retention policy should be respected when there is no traffic (#6676)
    
    Add a new feature:
    Retention policy should be respected when there is no traffic #6655
    
    * Retention policy should be respected when there is no traffic(#6655)
    
    * change parameter name and add annotations
    
    * Reduce the scheduling interval
    
    Co-authored-by: feynmanlin <fe...@tencent.com>
    Co-authored-by: Sijie Guo <si...@apache.org>(cherry picked from commit a658791b4dc696b8ada1fde1a7aa239b251dfda6)
---
 conf/broker.conf                                   |  4 +
 .../apache/bookkeeper/mledger/ManagedLedger.java   |  7 ++
 .../bookkeeper/mledger/impl/ManagedLedgerImpl.java |  7 +-
 .../apache/pulsar/broker/ServiceConfiguration.java |  6 ++
 .../pulsar/broker/service/BrokerService.java       | 25 ++++++
 .../broker/service/ConsumedLedgersTrimTest.java    | 93 ++++++++++++++++++++++
 6 files changed, 138 insertions(+), 4 deletions(-)

diff --git a/conf/broker.conf b/conf/broker.conf
index 212051a..83d009d 100644
--- a/conf/broker.conf
+++ b/conf/broker.conf
@@ -336,6 +336,10 @@ maxMessagePublishBufferSizeInMB=
 # Use 0 or negative number to disable the max publish buffer limiting.
 messagePublishBufferCheckIntervalInMillis=100
 
+# Check between intervals to see if consumed ledgers need to be trimmed
+# Use 0 or negative number to disable the check
+retentionCheckIntervalInSeconds=120
+
 ### --- Authentication --- ###
 # Role names that are treated as "proxy roles". If the broker sees a request with
 #role as proxyRoles - it will demand to see a valid original principal.
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java
index bedb81d..50439f5 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java
@@ -22,6 +22,7 @@ import com.google.common.annotations.Beta;
 import io.netty.buffer.ByteBuf;
 
 import java.util.Map;
+import java.util.concurrent.CompletableFuture;
 
 import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback;
 import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback;
@@ -449,4 +450,10 @@ public interface ManagedLedger {
      * Signaling managed ledger that we can resume after BK write failure
      */
     void readyToCreateNewLedger();
+
+    /**
+     * Trim consumed ledgers in background
+     * @param promise
+     */
+    void trimConsumedLedgersInBackground(CompletableFuture<?> promise);
 }
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
index 644ae34..08b5e5e 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
@@ -1841,10 +1841,9 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
         trimConsumedLedgersInBackground(Futures.NULL_PROMISE);
     }
 
-    private void trimConsumedLedgersInBackground(CompletableFuture<?> promise) {
-        executor.executeOrdered(name, safeRun(() -> {
-            internalTrimConsumedLedgers(promise);
-        }));
+    @Override
+    public void trimConsumedLedgersInBackground(CompletableFuture<?> promise) {
+        executor.executeOrdered(name, safeRun(() -> internalTrimConsumedLedgers(promise)));
     }
 
     private void scheduleDeferredTrimming(CompletableFuture<?> promise) {
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
index 19a4337..d7d0d63 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
@@ -621,6 +621,12 @@ public class ServiceConfiguration implements PulsarConfiguration {
     )
     private int messagePublishBufferCheckIntervalInMillis = 100;
 
+    @FieldContext(
+        category = CATEGORY_SERVER,
+        doc = "Check between intervals to see if consumed ledgers need to be trimmed"
+    )
+    private int retentionCheckIntervalInSeconds = 120;
+
     /**** --- Messaging Protocols --- ****/
 
     @FieldContext(
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
index b1ee242..085bae4 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
@@ -78,6 +78,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
 import org.apache.bookkeeper.mledger.ManagedLedgerException;
 import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerNotFoundException;
 import org.apache.bookkeeper.mledger.ManagedLedgerFactory;
+import org.apache.bookkeeper.mledger.util.Futures;
 import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
@@ -189,6 +190,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
     private final ScheduledExecutorService messageExpiryMonitor;
     private final ScheduledExecutorService compactionMonitor;
     private final ScheduledExecutorService messagePublishBufferMonitor;
+    private final ScheduledExecutorService consumedLedgersMonitor;
     private ScheduledExecutorService topicPublishRateLimiterMonitor;
     private ScheduledExecutorService brokerPublishRateLimiterMonitor;
     protected volatile PublishRateLimiter brokerPublishRateLimiter = PublishRateLimiter.DISABLED_RATE_LIMITER;
@@ -270,6 +272,8 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
             Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-compaction-monitor"));
         this.messagePublishBufferMonitor =
             Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-publish-buffer-monitor"));
+        this.consumedLedgersMonitor = Executors
+                .newSingleThreadScheduledExecutor(new DefaultThreadFactory("consumed-Ledgers-monitor"));
 
         this.backlogQuotaManager = new BacklogQuotaManager(pulsar);
         this.backlogQuotaChecker = Executors
@@ -400,6 +404,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
         this.startMessageExpiryMonitor();
         this.startCompactionMonitor();
         this.startMessagePublishBufferMonitor();
+        this.startConsumedLedgersMonitor();
         this.startBacklogQuotaChecker();
         this.updateBrokerPublisherThrottlingMaxRate();
         // register listener to capture zk-latency
@@ -460,6 +465,14 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
         }
     }
 
+    protected void startConsumedLedgersMonitor() {
+        int interval = pulsar().getConfiguration().getRetentionCheckIntervalInSeconds();
+        if (interval > 0) {
+            consumedLedgersMonitor.scheduleAtFixedRate(safeRun(this::checkConsumedLedgers),
+                                                            interval, interval, TimeUnit.SECONDS);
+        }
+    }
+
     protected void startBacklogQuotaChecker() {
         if (pulsar().getConfiguration().isBacklogQuotaCheckEnabled()) {
             final int interval = pulsar().getConfiguration().getBacklogQuotaCheckIntervalInSeconds();
@@ -1150,6 +1163,18 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
             });
     }
 
+    private void checkConsumedLedgers() {
+        forEachTopic((t) -> {
+            if (t instanceof PersistentTopic) {
+                Optional.ofNullable(((PersistentTopic) t).getManagedLedger()).ifPresent(
+                        managedLedger -> {
+                            managedLedger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE);
+                        }
+                );
+            }
+        });
+    }
+
     public void checkMessageDeduplicationInfo() {
         forEachTopic(Topic::checkMessageDeduplicationInfo);
     }
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumedLedgersTrimTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumedLedgersTrimTest.java
new file mode 100644
index 0000000..72c2eff
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumedLedgersTrimTest.java
@@ -0,0 +1,93 @@
+/**
+ * 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.service;
+
+
+import lombok.Cleanup;
+import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.junit.Test;
+import org.testng.Assert;
+
+import java.util.concurrent.TimeUnit;
+
+public class ConsumedLedgersTrimTest extends BrokerTestBase {
+    @Override
+    protected void setup() throws Exception {
+        //No-op
+    }
+
+    @Override
+    protected void cleanup() throws Exception {
+        //No-op
+    }
+
+    @Test
+    public void TestConsumedLedgersTrim() throws Exception {
+        conf.setRetentionCheckIntervalInSeconds(1);
+        super.baseSetup();
+        final String topicName = "persistent://prop/ns-abc/TestConsumedLedgersTrim";
+        final String subscriptionName = "my-subscriber-name";
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topicName)
+                .producerName("producer-name")
+                .create();
+        @Cleanup
+        Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName)
+                .subscribe();
+        Topic topicRef = pulsar.getBrokerService().getTopicReference(topicName).get();
+        Assert.assertNotNull(topicRef);
+        PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get();
+
+        ManagedLedgerConfig managedLedgerConfig = persistentTopic.getManagedLedger().getConfig();
+        managedLedgerConfig.setRetentionSizeInMB(1L);
+        managedLedgerConfig.setRetentionTime(1, TimeUnit.SECONDS);
+        managedLedgerConfig.setMaxEntriesPerLedger(2);
+        managedLedgerConfig.setMinimumRolloverTime(1, TimeUnit.MILLISECONDS);
+
+        int msgNum = 10;
+        for (int i = 0; i < msgNum; i++) {
+            producer.send(new byte[1024 * 1024]);
+        }
+
+        ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger();
+        Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), msgNum / 2);
+
+        //no traffic, unconsumed ledger will be retained
+        Thread.sleep(1200);
+        Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), msgNum / 2);
+
+        for (int i = 0; i < msgNum; i++) {
+            Message<byte[]> msg = consumer.receive(2, TimeUnit.SECONDS);
+            Assert.assertTrue(msg != null);
+            consumer.acknowledge(msg);
+        }
+        Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), msgNum / 2);
+
+        //no traffic, but consumed ledger will be cleaned
+        Thread.sleep(1500);
+        Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), 1);
+    }
+}


[pulsar] 24/38: [Issue #5395][broker] Implement AutoTopicCreation by namespace override (#6471)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 1c029455b6c6d32f305901d19db76a98b777a031
Author: Kai <kl...@toasttab.com>
AuthorDate: Wed Mar 25 23:59:43 2020 -0700

    [Issue #5395][broker] Implement AutoTopicCreation by namespace override (#6471)
    
    Fixes #5395
    
    This change introduces a new namespace policy `autoTopicCreationOverride`, which will enable an override of broker `autoTopicCreation` settings on the namespace level. You may keep `autoTopicCreation` disabled for the broker and allow it on a specific namespace using this feature.
    
    - Add new namespace policy: `autoTopicCreationOverride` and associated API / CLI interface for setting and removing. Defaults to non-partitioned type, but also allows partitioned topics.
    - Modifies BrokerService: when checking `autoTopicCreation` configuration, the broker first retrieves namespace policies from zookeeper. If the `autoTopicCreationOverride` policy exists for that namespace then it uses those settings. If not, falls back to broker configuration.
    - Slight refactor to move `TopicType` enum to pulsar-common and add `autoTopicCreationOverride` to pulsar-common.
    (cherry picked from commit fdc3a9bc8f04c4e424fec90a636a4aa25b35dcd8)
---
 .../apache/pulsar/broker/ServiceConfiguration.java |  15 +-
 .../pulsar/broker/admin/impl/NamespacesBase.java   | 106 ++++++++++-
 .../apache/pulsar/broker/admin/v2/Namespaces.java  |  38 ++++
 .../pulsar/broker/service/BrokerService.java       |  59 +++++-
 .../apache/pulsar/broker/service/ServerCnx.java    |   2 +-
 .../broker/auth/MockedPulsarServiceBaseTest.java   |   2 +-
 .../BrokerServiceAutoTopicCreationTest.java        | 199 ++++++++++++++++++---
 .../org/apache/pulsar/client/admin/Namespaces.java |  59 +++++-
 .../client/admin/internal/NamespacesImpl.java      |  29 ++-
 .../pulsar/admin/cli/PulsarAdminToolTest.java      |   9 +
 .../org/apache/pulsar/admin/cli/CmdNamespaces.java |  59 ++++++
 .../policies/data/AutoTopicCreationOverride.java   |  89 +++++++++
 .../pulsar/common/policies/data/Policies.java      |   6 +-
 .../pulsar/common/policies/data/TopicType.java     |  45 +++++
 .../data/AutoTopicCreationOverrideTest.java        |  63 +++++++
 15 files changed, 729 insertions(+), 51 deletions(-)

diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
index 6ad8b46..8d9a2b7 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
@@ -35,6 +35,7 @@ import lombok.Setter;
 import org.apache.bookkeeper.client.api.DigestType;
 import org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider;
 import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode;
+import org.apache.pulsar.common.policies.data.TopicType;
 import org.apache.pulsar.common.protocol.Commands;
 import org.apache.pulsar.common.configuration.Category;
 import org.apache.pulsar.common.configuration.FieldContext;
@@ -1533,18 +1534,4 @@ public class ServiceConfiguration implements PulsarConfiguration {
             return brokerDeleteInactiveTopicsMaxInactiveDurationSeconds;
         }
     }
-
-    enum TopicType {
-        PARTITIONED("partitioned"),
-        NON_PARTITIONED("non-partitioned");
-        private String type;
-
-        TopicType(String type) {
-            this.type = type;
-        }
-
-        public String toString() {
-            return type;
-        }
-    }
 }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
index 01927f2..535361e 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
@@ -69,6 +69,7 @@ import org.apache.pulsar.common.naming.NamespaceBundles;
 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.AutoTopicCreationOverride;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.policies.data.BacklogQuota;
 import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType;
@@ -560,6 +561,105 @@ public abstract class NamespacesBase extends AdminResource {
         }
     }
 
+    protected void internalSetAutoTopicCreation(AsyncResponse asyncResponse, AutoTopicCreationOverride autoTopicCreationOverride) {
+        validateAdminAccessForTenant(namespaceName.getTenant());
+        validatePoliciesReadOnlyAccess();
+
+        if (!AutoTopicCreationOverride.isValidOverride(autoTopicCreationOverride)) {
+            throw new RestException(Status.PRECONDITION_FAILED, "Invalid configuration for autoTopicCreationOverride");
+        }
+
+        // Force to read the data s.t. the watch to the cache content is setup.
+        policiesCache().getWithStatAsync(path(POLICIES, namespaceName.toString())).thenApply(
+                policies -> {
+                    if (policies.isPresent()) {
+                        Entry<Policies, Stat> policiesNode = policies.get();
+                        policiesNode.getKey().autoTopicCreationOverride = autoTopicCreationOverride;
+                        try {
+                            // Write back the new policies into zookeeper
+                            globalZk().setData(path(POLICIES, namespaceName.toString()),
+                                    jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion());
+                            policiesCache().invalidate(path(POLICIES, namespaceName.toString()));
+                            asyncResponse.resume(Response.noContent().build());
+                            log.info("[{}] Successfully {} on namespace {}", clientAppId(),
+                                    autoTopicCreationOverride.allowAutoTopicCreation ? "enabled" : "disabled", namespaceName);
+                            return null;
+                        } catch (KeeperException.NoNodeException e) {
+                            log.error("[{}] Failed to modify autoTopicCreation status for namespace {}: does not exist", clientAppId(),
+                                    namespaceName);
+                            asyncResponse.resume(new RestException(Status.NOT_FOUND, "Namespace does not exist"));
+                            return null;
+                        } catch (KeeperException.BadVersionException e) {
+                            log.error(
+                                    "[{}] Failed to modify autoTopicCreation status on namespace {} expected policy node version={} : concurrent modification",
+                                    clientAppId(), namespaceName, policiesNode.getValue().getVersion());
+
+                            asyncResponse.resume(new RestException(Status.CONFLICT, "Concurrent modification"));
+                            return null;
+                        } catch (Exception e) {
+                            log.error("[{}] Failed to modify autoTopicCreation status on namespace {}", clientAppId(), namespaceName, e);
+                            asyncResponse.resume(new RestException(e));
+                            return null;
+                        }
+                    } else {
+                        asyncResponse.resume(new RestException(Status.NOT_FOUND, "Namespace " + namespaceName + " does not exist"));
+                        return null;
+                    }
+                }
+        ).exceptionally(e -> {
+            log.error("[{}] Failed to modify autoTopicCreation status on namespace {}", clientAppId(), namespaceName, e);
+            asyncResponse.resume(new RestException(e));
+            return null;
+        });
+    }
+
+    protected void internalRemoveAutoTopicCreation(AsyncResponse asyncResponse) {
+        validateAdminAccessForTenant(namespaceName.getTenant());
+        validatePoliciesReadOnlyAccess();
+
+        // Force to read the data s.t. the watch to the cache content is setup.
+        policiesCache().getWithStatAsync(path(POLICIES, namespaceName.toString())).thenApply(
+                policies -> {
+                    if (policies.isPresent()) {
+                        Entry<Policies, Stat> policiesNode = policies.get();
+                        policiesNode.getKey().autoTopicCreationOverride = null;
+                        try {
+                            // Write back the new policies into zookeeper
+                            globalZk().setData(path(POLICIES, namespaceName.toString()),
+                                    jsonMapper().writeValueAsBytes(policiesNode.getKey()), policiesNode.getValue().getVersion());
+                            policiesCache().invalidate(path(POLICIES, namespaceName.toString()));
+                            asyncResponse.resume(Response.noContent().build());
+                            log.info("[{}] Successfully removed override on namespace {}", clientAppId(), namespaceName);
+                            return null;
+                        } catch (KeeperException.NoNodeException e) {
+                            log.error("[{}] Failed to modify autoTopicCreation status for namespace {}: does not exist", clientAppId(),
+                                    namespaceName);
+                            asyncResponse.resume(new RestException(Status.NOT_FOUND, "Namespace does not exist"));
+                            return null;
+                        } catch (KeeperException.BadVersionException e) {
+                            log.error(
+                                    "[{}] Failed to modify autoTopicCreation status on namespace {} expected policy node version={} : concurrent modification",
+                                    clientAppId(), namespaceName, policiesNode.getValue().getVersion());
+
+                            asyncResponse.resume(new RestException(Status.CONFLICT, "Concurrent modification"));
+                            return null;
+                        } catch (Exception e) {
+                            log.error("[{}] Failed to modify autoTopicCreation status on namespace {}", clientAppId(), namespaceName, e);
+                            asyncResponse.resume(new RestException(e));
+                            return null;
+                        }
+                    } else {
+                        asyncResponse.resume(new RestException(Status.NOT_FOUND, "Namespace " + namespaceName + " does not exist"));
+                        return null;
+                    }
+                }
+        ).exceptionally(e -> {
+            log.error("[{}] Failed to modify autoTopicCreation status on namespace {}", clientAppId(), namespaceName, e);
+            asyncResponse.resume(new RestException(e));
+            return null;
+        });
+    }
+
     protected void internalModifyDeduplication(boolean enableDeduplication) {
         validateAdminAccessForTenant(namespaceName.getTenant());
         validatePoliciesReadOnlyAccess();
@@ -580,17 +680,17 @@ public abstract class NamespacesBase extends AdminResource {
             log.info("[{}] Successfully {} on namespace {}", clientAppId(),
                     enableDeduplication ? "enabled" : "disabled", namespaceName);
         } catch (KeeperException.NoNodeException e) {
-            log.warn("[{}] Failed to modify deplication status for namespace {}: does not exist", clientAppId(),
+            log.warn("[{}] Failed to modify deduplication status for namespace {}: does not exist", clientAppId(),
                     namespaceName);
             throw new RestException(Status.NOT_FOUND, "Namespace does not exist");
         } catch (KeeperException.BadVersionException e) {
             log.warn(
-                    "[{}] Failed to modify deplication status on namespace {} expected policy node version={} : concurrent modification",
+                    "[{}] Failed to modify deduplication status on namespace {} expected policy node version={} : concurrent modification",
                     clientAppId(), namespaceName, policiesNode.getValue().getVersion());
 
             throw new RestException(Status.CONFLICT, "Concurrent modification");
         } catch (Exception e) {
-            log.error("[{}] Failed to modify deplication status on namespace {}", clientAppId(), namespaceName, e);
+            log.error("[{}] Failed to modify deduplication status on namespace {}", clientAppId(), namespaceName, e);
             throw new RestException(e);
         }
     }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
index 3dedcf6..26867e7 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java
@@ -45,6 +45,7 @@ import javax.ws.rs.core.MediaType;
 import org.apache.pulsar.broker.admin.impl.NamespacesBase;
 import org.apache.pulsar.broker.web.RestException;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode;
+import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride;
 import org.apache.pulsar.common.policies.data.AuthAction;
 import org.apache.pulsar.common.policies.data.BacklogQuota;
 import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType;
@@ -295,6 +296,43 @@ public class Namespaces extends NamespacesBase {
         internalModifyDeduplication(enableDeduplication);
     }
 
+    @POST
+    @Path("/{tenant}/{namespace}/autoTopicCreation")
+    @ApiOperation(value = "Override broker's allowAutoTopicCreation setting for a namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"),
+            @ApiResponse(code = 400, message = "Invalid autoTopicCreation override") })
+    public void setAutoTopicCreation(
+            @Suspended final AsyncResponse asyncResponse,
+                                                  @PathParam("tenant") String tenant, @PathParam("namespace") String namespace,
+                                                  AutoTopicCreationOverride autoTopicCreationOverride) {
+        try {
+            validateNamespaceName(tenant, namespace);
+            internalSetAutoTopicCreation(asyncResponse, autoTopicCreationOverride);
+        } catch (RestException e) {
+            asyncResponse.resume(e);
+        } catch (Exception e ) {
+            asyncResponse.resume(new RestException(e));
+        }
+    }
+
+    @DELETE
+    @Path("/{tenant}/{namespace}/autoTopicCreation")
+    @ApiOperation(value = "Remove override of broker's allowAutoTopicCreation in a namespace")
+    @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"),
+            @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") })
+    public void removeAutoTopicCreation(@Suspended final AsyncResponse asyncResponse,
+                                        @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) {
+        try {
+            validateNamespaceName(tenant, namespace);
+            internalRemoveAutoTopicCreation(asyncResponse);
+        } catch (RestException e) {
+            asyncResponse.resume(e);
+        } catch (Exception e) {
+            asyncResponse.resume(new RestException(e));
+        }
+    }
+
     @GET
     @Path("/{tenant}/{namespace}/bundles")
     @ApiOperation(value = "Get the bundles split data.")
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
index 3f0f9cf..ffd75b6 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
@@ -123,6 +123,7 @@ 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.partition.PartitionedTopicMetadata;
+import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride;
 import org.apache.pulsar.common.policies.data.ClusterData;
 import org.apache.pulsar.common.policies.data.LocalPolicies;
 import org.apache.pulsar.common.policies.data.OffloadPolicies;
@@ -132,6 +133,7 @@ import org.apache.pulsar.common.policies.data.Policies;
 import org.apache.pulsar.common.policies.data.PublishRate;
 import org.apache.pulsar.common.policies.data.RetentionPolicies;
 import org.apache.pulsar.common.policies.data.TopicStats;
+import org.apache.pulsar.common.policies.data.TopicType;
 import org.apache.pulsar.common.stats.Metrics;
 import org.apache.pulsar.common.util.FieldParser;
 import org.apache.pulsar.common.util.FutureUtil;
@@ -671,7 +673,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
     }
 
     public CompletableFuture<Topic> getOrCreateTopic(final String topic) {
-        return getTopic(topic, pulsar.getConfiguration().isAllowAutoTopicCreation()).thenApply(Optional::get);
+        return getTopic(topic, isAllowAutoTopicCreation(topic)).thenApply(Optional::get);
     }
 
     public CompletableFuture<Optional<Topic>> getTopic(final String topic, boolean createIfMissing) {
@@ -1861,8 +1863,8 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
                                 // If topic is already exist, creating partitioned topic is not allowed.
                                 if (metadata.partitions == 0
                                         && !topicExists
-                                        && pulsar.getConfiguration().isAllowAutoTopicCreation()
-                                        && pulsar.getConfiguration().isDefaultTopicTypePartitioned()) {
+                                        && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
+                                        && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
                                     return pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName);
                                 } else {
                                     return CompletableFuture.completedFuture(metadata);
@@ -1873,7 +1875,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
 
     @SuppressWarnings("deprecation")
     private CompletableFuture<PartitionedTopicMetadata> createDefaultPartitionedTopicAsync(TopicName topicName) {
-        int defaultNumPartitions = pulsar.getConfiguration().getDefaultNumPartitions();
+        int defaultNumPartitions = pulsar.getBrokerService().getDefaultNumPartitions(topicName);
         checkArgument(defaultNumPartitions > 0, "Default number of partitions should be more than 0");
 
         PartitionedTopicMetadata configMetadata = new PartitionedTopicMetadata(defaultNumPartitions);
@@ -2106,4 +2108,53 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies
         foreachProducer(producer -> currentMessagePublishBufferBytes.addAndGet(producer.getCnx().getMessagePublishBufferSize()));
         return currentMessagePublishBufferBytes.get();
     }
+
+    public boolean isAllowAutoTopicCreation(final String topic) {
+        TopicName topicName = TopicName.get(topic);
+        return isAllowAutoTopicCreation(topicName);
+    }
+
+    public boolean isAllowAutoTopicCreation(final TopicName topicName) {
+        AutoTopicCreationOverride autoTopicCreationOverride = getAutoTopicCreationOverride(topicName);
+        if (autoTopicCreationOverride != null) {
+            return autoTopicCreationOverride.allowAutoTopicCreation;
+        } else {
+            return pulsar.getConfiguration().isAllowAutoTopicCreation();
+        }
+    }
+
+    public boolean isDefaultTopicTypePartitioned(final TopicName topicName) {
+        AutoTopicCreationOverride autoTopicCreationOverride = getAutoTopicCreationOverride(topicName);
+        if (autoTopicCreationOverride != null) {
+            return TopicType.PARTITIONED.toString().equals(autoTopicCreationOverride.topicType);
+        } else {
+            return pulsar.getConfiguration().isDefaultTopicTypePartitioned();
+        }
+    }
+
+    public int getDefaultNumPartitions(final TopicName topicName) {
+        AutoTopicCreationOverride autoTopicCreationOverride = getAutoTopicCreationOverride(topicName);
+        if (autoTopicCreationOverride != null) {
+            return autoTopicCreationOverride.defaultNumPartitions;
+        } else {
+            return pulsar.getConfiguration().getDefaultNumPartitions();
+        }
+    }
+
+    private AutoTopicCreationOverride getAutoTopicCreationOverride(final TopicName topicName) {
+        try {
+            Optional<Policies> policies = pulsar.getConfigurationCache().policiesCache()
+                            .get(AdminResource.path(POLICIES, topicName.getNamespace()));
+            // If namespace policies have the field set, it will override the broker-level setting
+            if (policies.isPresent() && policies.get().autoTopicCreationOverride != null) {
+                return policies.get().autoTopicCreationOverride;
+            }
+        } catch (Throwable t) {
+            // Ignoring since if we don't have policies, we fallback on the default
+            log.warn("Got exception when reading autoTopicCreateOverride policy for {}: {};", topicName, t.getMessage(), t);
+            return null;
+        }
+        log.warn("No autoTopicCreateOverride policy found for {}", topicName);
+        return null;
+    }
 }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
index 0e99529..ce8411e 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
@@ -803,7 +803,7 @@ public class ServerCnx extends PulsarHandler {
                         }
 
                         boolean createTopicIfDoesNotExist = forceTopicCreation
-                                && service.pulsar().getConfig().isAllowAutoTopicCreation();
+                                && service.isAllowAutoTopicCreation(topicName.toString());
 
                         service.getTopic(topicName.toString(), createTopicIfDoesNotExist)
                                 .thenCompose(optTopic -> {
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java
index a47e3eb..7b1d8d3 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java
@@ -96,7 +96,7 @@ public abstract class MockedPulsarServiceBaseTest {
         this.conf.setDefaultNumberOfNamespaceBundles(1);
         this.conf.setZookeeperServers("localhost:2181");
         this.conf.setConfigurationStoreServers("localhost:3181");
-        this.conf.setAllowAutoTopicCreationType("non-persistent");
+        this.conf.setAllowAutoTopicCreationType("non-partitioned");
         this.conf.setBrokerServicePort(Optional.of(0));
         this.conf.setBrokerServicePortTls(Optional.of(0));
         this.conf.setWebServicePort(Optional.of(0));
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java
index 3f404af..275ec3f 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java
@@ -24,8 +24,11 @@ import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertFalse;
 import static org.testng.Assert.assertTrue;
 import static org.testng.Assert.fail;
-
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride;
+import org.apache.pulsar.common.policies.data.TopicType;
 import org.testng.annotations.AfterClass;
+import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
@@ -43,17 +46,35 @@ public class BrokerServiceAutoTopicCreationTest extends BrokerTestBase{
         super.internalCleanup();
     }
 
+    @AfterMethod
+    protected void cleanupTest() throws Exception {
+        pulsar.getAdminClient().namespaces().removeAutoTopicCreation("prop/ns-abc");
+    }
+
+
     @Test
     public void testAutoNonPartitionedTopicCreation() throws Exception{
         pulsar.getConfiguration().setAllowAutoTopicCreation(true);
         pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned");
 
-        final String topicName = "persistent://prop/ns-abc/non-partitioned-topic";
+        final String topicString = "persistent://prop/ns-abc/non-partitioned-topic";
         final String subscriptionName = "non-partitioned-topic-sub";
-        pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe();
+        pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe();
 
-        assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicName));
-        assertFalse(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicName));
+        assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString));
+        assertFalse(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicString));
+    }
+
+    @Test
+    public void testAutoNonPartitionedTopicCreationOnProduce() throws Exception{
+        pulsar.getConfiguration().setAllowAutoTopicCreation(true);
+        pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned");
+
+        final String topicString = "persistent://prop/ns-abc/non-partitioned-topic-2";
+        pulsarClient.newProducer().topic(topicString).create();
+
+        assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString));
+        assertFalse(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicString));
     }
 
     @Test
@@ -62,13 +83,28 @@ public class BrokerServiceAutoTopicCreationTest extends BrokerTestBase{
         pulsar.getConfiguration().setAllowAutoTopicCreationType("partitioned");
         pulsar.getConfiguration().setDefaultNumPartitions(3);
 
-        final String topicName = "persistent://prop/ns-abc/partitioned-topic";
+        final String topicString = "persistent://prop/ns-abc/partitioned-topic";
         final String subscriptionName = "partitioned-topic-sub";
-        pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe();
+        pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe();
 
-        assertTrue(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicName));
+        assertTrue(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicString));
         for (int i = 0; i < 3; i++) {
-            assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicName + "-partition-" + i));
+            assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString + "-partition-" + i));
+        }
+    }
+
+    @Test
+    public void testAutoPartitionedTopicCreationOnProduce() throws Exception{
+        pulsar.getConfiguration().setAllowAutoTopicCreation(true);
+        pulsar.getConfiguration().setAllowAutoTopicCreationType("partitioned");
+        pulsar.getConfiguration().setDefaultNumPartitions(3);
+
+        final String topicString = "persistent://prop/ns-abc/partitioned-topic-1";
+        pulsarClient.newProducer().topic(topicString).create();
+
+        assertTrue(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicString));
+        for (int i = 0; i < 3; i++) {
+            assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString + "-partition-" + i));
         }
     }
 
@@ -76,15 +112,15 @@ public class BrokerServiceAutoTopicCreationTest extends BrokerTestBase{
     public void testAutoTopicCreationDisable() throws Exception{
         pulsar.getConfiguration().setAllowAutoTopicCreation(false);
 
-        final String topicName = "persistent://prop/ns-abc/test-topic";
+        final String topicString = "persistent://prop/ns-abc/test-topic";
         final String subscriptionName = "test-topic-sub";
         try {
-            pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe();
+            pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe();
             fail("Subscribe operation should have failed");
         } catch (Exception e) {
             assertTrue(e instanceof PulsarClientException);
         }
-        assertFalse(admin.namespaces().getTopics("prop/ns-abc").contains(topicName));
+        assertFalse(admin.namespaces().getTopics("prop/ns-abc").contains(topicString));
     }
 
     @Test
@@ -93,16 +129,16 @@ public class BrokerServiceAutoTopicCreationTest extends BrokerTestBase{
         pulsar.getConfiguration().setAllowAutoTopicCreationType("partitioned");
         pulsar.getConfiguration().setDefaultNumPartitions(3);
 
-        final String topicName = "persistent://prop/ns-abc/test-topic-2";
+        final String topicString = "persistent://prop/ns-abc/test-topic-2";
         final String subscriptionName = "partitioned-topic-sub";
-        admin.topics().createNonPartitionedTopic(topicName);
-        pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe();
+        admin.topics().createNonPartitionedTopic(topicString);
+        pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe();
 
-        assertFalse(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicName));
+        assertFalse(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicString));
         for (int i = 0; i < 3; i++) {
-            assertFalse(admin.namespaces().getTopics("prop/ns-abc").contains(topicName + "-partition-" + i));
+            assertFalse(admin.namespaces().getTopics("prop/ns-abc").contains(topicString + "-partition-" + i));
         }
-        assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicName));
+        assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString));
     }
 
     /**
@@ -116,10 +152,131 @@ public class BrokerServiceAutoTopicCreationTest extends BrokerTestBase{
         pulsar.getConfiguration().setAllowAutoTopicCreationType("partitioned");
         pulsar.getConfiguration().setDefaultNumPartitions(3);
 
-        final String topicName = "persistent://prop/ns-abc/test-topic-3";
-        int partitions = admin.topics().getPartitionedTopicMetadata(topicName).partitions;
+        final String topicString = "persistent://prop/ns-abc/test-topic-3";
+        int partitions = admin.topics().getPartitionedTopicMetadata(topicString).partitions;
         assertEquals(partitions, 0);
-        assertFalse(admin.namespaces().getTopics("prop/ns-abc").contains(topicName));
+        assertFalse(admin.namespaces().getTopics("prop/ns-abc").contains(topicString));
+    }
+
+    @Test
+    public void testAutoCreationNamespaceAllowOverridesBroker() throws Exception {
+        final String topicString = "persistent://prop/ns-abc/test-topic-4";
+        final String subscriptionName = "test-topic-sub-4";
+        final TopicName topicName = TopicName.get(topicString);
+        pulsar.getConfiguration().setAllowAutoTopicCreation(false);
+        pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(),
+                new AutoTopicCreationOverride(true, TopicType.NON_PARTITIONED.toString(), null));
+
+        pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe();
+        assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString));
+        assertFalse(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicString));
+    }
+
+    @Test
+    public void testAutoCreationNamespaceDisallowOverridesBroker() throws Exception {
+        final String topicString = "persistent://prop/ns-abc/test-topic-5";
+        final String subscriptionName = "test-topic-sub-5";
+        final TopicName topicName = TopicName.get(topicString);
+        pulsar.getConfiguration().setAllowAutoTopicCreation(true);
+        pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(),
+                new AutoTopicCreationOverride(false, null, null));
+
+        try {
+            pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe();
+            fail("Subscribe operation should have failed");
+        } catch (Exception e) {
+            assertTrue(e instanceof PulsarClientException);
+        }
+        assertFalse(admin.namespaces().getTopics("prop/ns-abc").contains(topicString));
+    }
+
+    @Test
+    public void testAutoCreationNamespaceOverrideAllowsPartitionedTopics() throws Exception {
+        final String topicString = "persistent://prop/ns-abc/partitioned-test-topic-6";
+        final TopicName topicName = TopicName.get(topicString);
+
+        pulsar.getConfiguration().setAllowAutoTopicCreation(false);
+        pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(),
+                new AutoTopicCreationOverride(true, TopicType.PARTITIONED.toString(), 4));
+
+        final String subscriptionName = "test-topic-sub-6";
+        pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe();
+
+        assertTrue(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicString));
+        for (int i = 0; i < 4; i++) {
+            assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString + "-partition-" + i));
+        }
+    }
+
+    @Test
+    public void testAutoCreationNamespaceOverridesTopicTypePartitioned() throws Exception {
+        final String topicString = "persistent://prop/ns-abc/partitioned-test-topic-7";
+        final TopicName topicName = TopicName.get(topicString);
+
+        pulsar.getConfiguration().setAllowAutoTopicCreation(true);
+        pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned");
+        pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(),
+                new AutoTopicCreationOverride(true, TopicType.PARTITIONED.toString(), 3));
+
+        final String subscriptionName = "test-topic-sub-7";
+        pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe();
+
+        assertTrue(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicString));
+        for (int i = 0; i < 3; i++) {
+            assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString + "-partition-" + i));
+        }
+    }
+
+    @Test
+    public void testAutoCreationNamespaceOverridesTopicTypeNonPartitioned() throws Exception {
+        final String topicString = "persistent://prop/ns-abc/partitioned-test-topic-8";
+        final TopicName topicName = TopicName.get(topicString);
+
+        pulsar.getConfiguration().setAllowAutoTopicCreation(true);
+        pulsar.getConfiguration().setAllowAutoTopicCreationType("partitioned");
+        pulsar.getConfiguration().setDefaultNumPartitions(2);
+        pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(),
+                new AutoTopicCreationOverride(true, TopicType.NON_PARTITIONED.toString(), null));
+
+        final String subscriptionName = "test-topic-sub-8";
+        pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe();
+
+        assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString));
+        assertFalse(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicString));
+    }
+
+    @Test
+    public void testAutoCreationNamespaceOverridesDefaultNumPartitions() throws Exception {
+        final String topicString = "persistent://prop/ns-abc/partitioned-test-topic-9";
+        final TopicName topicName = TopicName.get(topicString);
+
+        pulsar.getConfiguration().setAllowAutoTopicCreation(true);
+        pulsar.getConfiguration().setAllowAutoTopicCreationType("partitioned");
+        pulsar.getConfiguration().setDefaultNumPartitions(2);
+        pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(),
+                new AutoTopicCreationOverride(true, TopicType.PARTITIONED.toString(), 4));
+
+        final String subscriptionName = "test-topic-sub-9";
+
+        pulsarClient.newConsumer().topic(topicString).subscriptionName(subscriptionName).subscribe();
+
+        assertTrue(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicString));
+        for (int i = 0; i < 4; i++) {
+            assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString + "-partition-" + i));
+        }
+    }
+
+    @Test
+    public void testAutoCreationNamespaceAllowOverridesBrokerOnProduce() throws Exception {
+        final String topicString = "persistent://prop/ns-abc/test-topic-10";
+        final TopicName topicName = TopicName.get(topicString);
+        pulsar.getConfiguration().setAllowAutoTopicCreation(false);
+        pulsar.getAdminClient().namespaces().setAutoTopicCreation(topicName.getNamespace(),
+                new AutoTopicCreationOverride(true, TopicType.NON_PARTITIONED.toString(), null));
+
+        pulsarClient.newProducer().topic(topicString).create();
+        assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicString));
+        assertFalse(admin.topics().getPartitionedTopicList("prop/ns-abc").contains(topicString));
     }
 
 
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java
index bce5aba5..a1f7ec1 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Namespaces.java
@@ -29,6 +29,7 @@ import org.apache.pulsar.client.admin.PulsarAdminException.NotAuthorizedExceptio
 import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException;
 import org.apache.pulsar.client.admin.PulsarAdminException.PreconditionFailedException;
 import org.apache.pulsar.common.policies.data.AuthAction;
+import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride;
 import org.apache.pulsar.common.policies.data.BacklogQuota;
 import org.apache.pulsar.common.policies.data.BookieAffinityGroupData;
 import org.apache.pulsar.common.policies.data.BundlesData;
@@ -606,6 +607,56 @@ public interface Namespaces {
     void setDeduplicationStatus(String namespace, boolean enableDeduplication) throws PulsarAdminException;
 
     /**
+     * Sets the autoTopicCreation policy for a given namespace, overriding broker settings
+     * <p/>
+     * When autoTopicCreationOverride is enabled, new topics will be created upon connection,
+     * regardless of the broker level configuration.
+     * <p/>
+     * Request example:
+     *
+     * <pre>
+     * <code>
+     *  {
+     *      "allowAutoTopicCreation" : true,
+     *      "topicType" : "partitioned",
+     *      "defaultNumPartitions": 2
+     *  }
+     * </code>
+     * </pre>
+     *
+     * @param namespace
+     *            Namespace name
+     * @param autoTopicCreationOverride
+     *            Override policies for auto topic creation
+     *
+     * @throws NotAuthorizedException
+     *             Don't have admin permission
+     * @throws NotFoundException
+     *             Namespace does not exist
+     * @throws PulsarAdminException
+     *             Unexpected error
+     */
+    void setAutoTopicCreation(String namespace, AutoTopicCreationOverride autoTopicCreationOverride)
+            throws PulsarAdminException;
+
+    /**
+     * Removes the autoTopicCreation policy for a given namespace,
+     * allowing the broker to dictate the auto-creation policy.
+     * <p/>
+     *
+     * @param namespace
+     *            Namespace name
+     *
+     * @throws NotAuthorizedException
+     *             Don't have admin permission
+     * @throws NotFoundException
+     *             Namespace does not exist
+     * @throws PulsarAdminException
+     *             Unexpected error
+     */
+    void removeAutoTopicCreation(String namespace) throws PulsarAdminException;
+
+    /**
      * Get the bundles split data.
      *
      * @param namespace
@@ -784,7 +835,7 @@ public interface Namespaces {
     /**
      * Set bookie affinity group for a namespace to isolate namespace write to bookies that are part of given affinity
      * group.
-     * 
+     *
      * @param namespace
      *            namespace name
      * @param bookieAffinityGroup
@@ -793,10 +844,10 @@ public interface Namespaces {
      */
     void setBookieAffinityGroup(String namespace, BookieAffinityGroupData bookieAffinityGroup)
             throws PulsarAdminException;
-    
+
     /**
      * Delete bookie affinity group configured for a namespace.
-     * 
+     *
      * @param namespace
      * @throws PulsarAdminException
      */
@@ -804,7 +855,7 @@ public interface Namespaces {
 
     /**
      * Get bookie affinity group configured for a namespace.
-     * 
+     *
      * @param namespace
      * @return
      * @throws PulsarAdminException
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java
index c21b601..a7d7b47 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java
@@ -37,6 +37,7 @@ import org.apache.pulsar.client.admin.PulsarAdminException;
 import org.apache.pulsar.client.api.Authentication;
 import org.apache.pulsar.common.naming.NamespaceName;
 import org.apache.pulsar.common.policies.data.AuthAction;
+import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride;
 import org.apache.pulsar.common.policies.data.BacklogQuota;
 import org.apache.pulsar.common.policies.data.BookieAffinityGroupData;
 import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType;
@@ -373,6 +374,30 @@ public class NamespacesImpl extends BaseResource implements Namespaces {
     }
 
     @Override
+    public void setAutoTopicCreation(String namespace,
+                                     AutoTopicCreationOverride autoTopicCreationOverride) throws PulsarAdminException {
+        try {
+            NamespaceName ns = NamespaceName.get(namespace);
+            WebTarget path = namespacePath(ns, "autoTopicCreation");
+            request(path).post(Entity.entity(autoTopicCreationOverride,
+                    MediaType.APPLICATION_JSON), ErrorData.class);
+        } catch (Exception e) {
+            throw getApiException(e);
+        }
+    }
+
+    @Override
+    public void removeAutoTopicCreation(String namespace) throws PulsarAdminException {
+        try {
+            NamespaceName ns = NamespaceName.get(namespace);
+            WebTarget path = namespacePath(ns, "autoTopicCreation");
+            request(path).delete(ErrorData.class);
+        } catch (Exception e) {
+            throw getApiException(e);
+        }
+    }
+
+    @Override
     public Map<BacklogQuotaType, BacklogQuota> getBacklogQuotaMap(String namespace) throws PulsarAdminException {
         try {
             NamespaceName ns = NamespaceName.get(namespace);
@@ -550,7 +575,7 @@ public class NamespacesImpl extends BaseResource implements Namespaces {
         } catch (Exception e) {
             throw getApiException(e);
         }
-    
+
     }
 
     @Override
@@ -563,7 +588,7 @@ public class NamespacesImpl extends BaseResource implements Namespaces {
             throw getApiException(e);
         }
     }
-    
+
     @Override
     public void setDispatchRate(String namespace, DispatchRate dispatchRate) throws PulsarAdminException {
         try {
diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java
index 9318f15..0a9d119 100644
--- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java
+++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java
@@ -55,6 +55,7 @@ import org.apache.pulsar.client.api.MessageId;
 import org.apache.pulsar.client.impl.auth.AuthenticationTls;
 import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
 import org.apache.pulsar.common.policies.data.AuthAction;
+import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride;
 import org.apache.pulsar.common.policies.data.BacklogQuota;
 import org.apache.pulsar.common.policies.data.BacklogQuota.RetentionPolicy;
 import org.apache.pulsar.common.policies.data.BookieAffinityGroupData;
@@ -72,6 +73,7 @@ import org.apache.pulsar.common.policies.data.ResourceQuota;
 import org.apache.pulsar.common.policies.data.RetentionPolicies;
 import org.apache.pulsar.common.policies.data.SubscribeRate;
 import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.apache.pulsar.common.policies.data.TopicType;
 import org.apache.pulsar.common.util.ObjectMapperFactory;
 import org.mockito.ArgumentMatcher;
 import org.mockito.Mockito;
@@ -355,6 +357,13 @@ public class PulsarAdminToolTest {
         namespaces.run(split("set-deduplication myprop/clust/ns1 --enable"));
         verify(mockNamespaces).setDeduplicationStatus("myprop/clust/ns1", true);
 
+        namespaces.run(split("set-auto-topic-creation myprop/clust/ns1 -e -t non-partitioned"));
+        verify(mockNamespaces).setAutoTopicCreation("myprop/clust/ns1",
+                new AutoTopicCreationOverride(true, TopicType.NON_PARTITIONED.toString(), null));
+
+        namespaces.run(split("remove-auto-topic-creation myprop/clust/ns1"));
+        verify(mockNamespaces).removeAutoTopicCreation("myprop/clust/ns1");
+
         namespaces.run(split("get-message-ttl myprop/clust/ns1"));
         verify(mockNamespaces).getNamespaceMessageTTL("myprop/clust/ns1");
 
diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java
index 3dd4a6f..0de2f74 100644
--- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java
+++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java
@@ -37,6 +37,7 @@ import org.apache.pulsar.admin.cli.utils.IOUtils;
 import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.client.admin.PulsarAdminException;
 import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride;
 import org.apache.pulsar.common.policies.data.BacklogQuota;
 import org.apache.pulsar.common.policies.data.BookieAffinityGroupData;
 import org.apache.pulsar.common.policies.data.BundlesData;
@@ -51,6 +52,7 @@ import org.apache.pulsar.common.policies.data.SchemaAutoUpdateCompatibilityStrat
 import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy;
 import org.apache.pulsar.common.policies.data.SubscribeRate;
 import org.apache.pulsar.common.policies.data.SubscriptionAuthMode;
+import org.apache.pulsar.common.policies.data.TopicType;
 import org.apache.pulsar.common.util.RelativeTimeUtil;
 
 @Parameters(commandDescription = "Operations about namespaces")
@@ -378,6 +380,60 @@ public class CmdNamespaces extends CmdBase {
         }
     }
 
+    @Parameters(commandDescription = "Enable or disable autoTopicCreation for a namespace, overriding broker settings")
+    private class SetAutoTopicCreation extends CliCommand {
+        @Parameter(description = "tenant/namespace", required = true)
+        private java.util.List<String> params;
+
+        @Parameter(names = { "--enable", "-e" }, description = "Enable allowAutoTopicCreation on namespace")
+        private boolean enable = false;
+
+        @Parameter(names = { "--disable", "-d" }, description = "Disable allowAutoTopicCreation on namespace")
+        private boolean disable = false;
+
+        @Parameter(names = { "--type", "-t" }, description = "Type of topic to be auto-created. " +
+                "Possible values: (partitioned, non-partitioned). Default value: non-partitioned")
+        private String type = "non-partitioned";
+
+        @Parameter(names = { "--num-partitions", "-n" }, description = "Default number of partitions of topic to be auto-created," +
+                " applicable to partitioned topics only", required = false)
+        private Integer defaultNumPartitions = null;
+
+        @Override
+        void run() throws PulsarAdminException {
+            String namespace = validateNamespace(params);
+            type = type.toLowerCase().trim();
+
+            if (enable == disable) {
+                throw new ParameterException("Need to specify either --enable or --disable");
+            }
+            if (enable) {
+                if (!TopicType.isValidTopicType(type)) {
+                    throw new ParameterException("Must specify type of topic to be created. " +
+                            "Possible values: (partitioned, non-partitioned)");
+                }
+
+                if (TopicType.PARTITIONED.toString().equals(type) && !(defaultNumPartitions > 0)) {
+                    throw new ParameterException("Must specify num-partitions > 0 for partitioned topic type.");
+                }
+            }
+            admin.namespaces().setAutoTopicCreation(namespace, new AutoTopicCreationOverride(enable, type, defaultNumPartitions));
+        }
+    }
+
+    @Parameters(commandDescription = "Remove override of autoTopicCreation for a namespace")
+    private class RemoveAutoTopicCreation extends CliCommand {
+        @Parameter(description = "tenant/namespace", required = true)
+        private java.util.List<String> params;
+
+        @Override
+        void run() throws PulsarAdminException {
+            String namespace = validateNamespace(params);
+
+            admin.namespaces().removeAutoTopicCreation(namespace);
+        }
+    }
+
     @Parameters(commandDescription = "Set the retention policy for a namespace")
     private class SetRetention extends CliCommand {
         @Parameter(description = "tenant/namespace", required = true)
@@ -1486,6 +1542,9 @@ public class CmdNamespaces extends CmdBase {
 
         jcommander.addCommand("set-deduplication", new SetDeduplication());
 
+        jcommander.addCommand("set-auto-topic-creation", new SetAutoTopicCreation());
+        jcommander.addCommand("remove-auto-topic-creation", new RemoveAutoTopicCreation());
+
         jcommander.addCommand("get-retention", new GetRetention());
         jcommander.addCommand("set-retention", new SetRetention());
         
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverride.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverride.java
new file mode 100644
index 0000000..3eb7a49
--- /dev/null
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverride.java
@@ -0,0 +1,89 @@
+/**
+ * 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.policies.data;
+
+import com.google.common.base.MoreObjects;
+
+import java.util.Objects;
+
+/**
+ * Override of autoTopicCreation settings on a namespace level.
+ */
+public class AutoTopicCreationOverride {
+    public boolean allowAutoTopicCreation;
+    public String topicType;
+    public Integer defaultNumPartitions;
+
+    public AutoTopicCreationOverride() {
+    }
+
+    public AutoTopicCreationOverride(boolean allowAutoTopicCreation, String topicType,
+                                     Integer defaultNumPartitions) {
+        this.allowAutoTopicCreation = allowAutoTopicCreation;
+        this.topicType = topicType;
+        this.defaultNumPartitions = defaultNumPartitions;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(allowAutoTopicCreation, topicType, defaultNumPartitions);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj instanceof AutoTopicCreationOverride) {
+            AutoTopicCreationOverride other = (AutoTopicCreationOverride) obj;
+            return Objects.equals(this.allowAutoTopicCreation, other.allowAutoTopicCreation)
+                    && Objects.equals(this.topicType, other.topicType)
+                    && Objects.equals(this.defaultNumPartitions, other.defaultNumPartitions);
+        }
+        return false;
+    }
+
+    @Override
+    public String toString() {
+        return MoreObjects.toStringHelper(this).add("allowAutoTopicCreation", allowAutoTopicCreation)
+                .add("topicType", topicType).add("defaultNumPartitions", defaultNumPartitions).toString();
+    }
+
+    public static boolean isValidOverride(AutoTopicCreationOverride override) {
+        if (override == null) {
+            return false;
+        }
+        if (override.allowAutoTopicCreation) {
+            if (!TopicType.isValidTopicType(override.topicType)) {
+                return false;
+            }
+            if (TopicType.PARTITIONED.toString().equals(override.topicType)) {
+                if (override.defaultNumPartitions == null) {
+                    return false;
+                }
+                if (!(override.defaultNumPartitions > 0)) {
+                    return false;
+                }
+            } else if (TopicType.NON_PARTITIONED.toString().equals(override.topicType)) {
+                if (override.defaultNumPartitions != null) {
+                    return false;
+                }
+            }
+        }
+        return true;
+    }
+
+}
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java
index fae2ae2..7e56666 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java
@@ -49,6 +49,8 @@ public class Policies {
 
     // If set, it will override the broker settings for enabling deduplication
     public Boolean deduplicationEnabled = null;
+    // If set, it will override the broker settings for allowing auto topic creation
+    public AutoTopicCreationOverride autoTopicCreationOverride = null;
     public Map<String, PublishRate> publishMaxMessageRate = Maps.newHashMap();
 
     @SuppressWarnings("checkstyle:MemberName")
@@ -106,7 +108,7 @@ public class Policies {
         return Objects.hash(auth_policies, replication_clusters,
                 backlog_quota_map, publishMaxMessageRate, clusterDispatchRate,
                 topicDispatchRate, subscriptionDispatchRate, replicatorDispatchRate,
-                clusterSubscribeRate, deduplicationEnabled, persistence,
+                clusterSubscribeRate, deduplicationEnabled, autoTopicCreationOverride, persistence,
                 bundles, latency_stats_sample_rate,
                 message_ttl_in_seconds, retention_policies,
                 encryption_required, delayed_delivery_policies,
@@ -136,6 +138,7 @@ public class Policies {
                     && Objects.equals(clusterSubscribeRate, other.clusterSubscribeRate)
                     && Objects.equals(publishMaxMessageRate, other.publishMaxMessageRate)
                     && Objects.equals(deduplicationEnabled, other.deduplicationEnabled)
+                    && Objects.equals(autoTopicCreationOverride, other.autoTopicCreationOverride)
                     && Objects.equals(persistence, other.persistence) && Objects.equals(bundles, other.bundles)
                     && Objects.equals(latency_stats_sample_rate, other.latency_stats_sample_rate)
                     && Objects.equals(message_ttl_in_seconds,
@@ -183,6 +186,7 @@ public class Policies {
                 .add("replication_clusters", replication_clusters).add("bundles", bundles)
                 .add("backlog_quota_map", backlog_quota_map).add("persistence", persistence)
                 .add("deduplicationEnabled", deduplicationEnabled)
+                .add("autoTopicCreationOverride", autoTopicCreationOverride)
                 .add("clusterDispatchRate", clusterDispatchRate)
                 .add("topicDispatchRate", topicDispatchRate)
                 .add("subscriptionDispatchRate", subscriptionDispatchRate)
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicType.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicType.java
new file mode 100644
index 0000000..b184bf6
--- /dev/null
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicType.java
@@ -0,0 +1,45 @@
+/**
+ * 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.policies.data;
+
+/**
+ * Topic types -- partitioned or non-partitioned.
+ */
+public enum TopicType {
+    PARTITIONED("partitioned"),
+    NON_PARTITIONED("non-partitioned");
+    private String type;
+
+    TopicType(String type) {
+        this.type = type;
+    }
+
+    public String toString() {
+        return type;
+    }
+
+    public static boolean isValidTopicType(String type) {
+        for (TopicType topicType : TopicType.values()) {
+            if (topicType.toString().equalsIgnoreCase(type)) {
+                return true;
+            }
+        }
+        return false;
+    }
+}
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverrideTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverrideTest.java
new file mode 100644
index 0000000..75bc110
--- /dev/null
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/AutoTopicCreationOverrideTest.java
@@ -0,0 +1,63 @@
+/**
+ * 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.policies.data;
+
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+
+public class AutoTopicCreationOverrideTest {
+
+    @Test
+    public void testValidOverrideNonPartitioned() {
+        AutoTopicCreationOverride override = new AutoTopicCreationOverride(true, TopicType.NON_PARTITIONED.toString(), null);
+        assertTrue(AutoTopicCreationOverride.isValidOverride(override));
+    }
+
+    @Test
+    public void testValidOverridePartitioned() {
+        AutoTopicCreationOverride override = new AutoTopicCreationOverride(true, TopicType.PARTITIONED.toString(), 2);
+        assertTrue(AutoTopicCreationOverride.isValidOverride(override));
+    }
+
+    @Test
+    public void testInvalidTopicType() {
+        AutoTopicCreationOverride override = new AutoTopicCreationOverride(true, "aaa", null);
+        assertFalse(AutoTopicCreationOverride.isValidOverride(override));
+    }
+
+    @Test
+    public void testNumPartitionsTooLow() {
+        AutoTopicCreationOverride override = new AutoTopicCreationOverride(true, TopicType.PARTITIONED.toString(), 0);
+        assertFalse(AutoTopicCreationOverride.isValidOverride(override));
+    }
+
+    @Test
+    public void testNumPartitionsNotSet() {
+        AutoTopicCreationOverride override = new AutoTopicCreationOverride(true, TopicType.PARTITIONED.toString(), null);
+        assertFalse(AutoTopicCreationOverride.isValidOverride(override));
+    }
+
+    @Test
+    public void testNumPartitionsOnNonPartitioned() {
+        AutoTopicCreationOverride override = new AutoTopicCreationOverride(true, TopicType.NON_PARTITIONED.toString(), 2);
+        assertFalse(AutoTopicCreationOverride.isValidOverride(override));
+    }
+}


[pulsar] 07/38: fix_admin_getIndividualMsgsFromBatch_bug (#6715)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit e75e0a06cc0ab3af38f88ba5544202b624e4d69d
Author: liudezhi <33...@users.noreply.github.com>
AuthorDate: Tue Apr 14 12:10:28 2020 +0800

    fix_admin_getIndividualMsgsFromBatch_bug (#6715)
    
    **Motivation**
    fix when get batch message from http response, only get the first message. #6714
    ```javascript
    for (int i = 0; i < batchSize; i++) {
                String batchMsgId = msgId + ":" + i;
                PulsarApi.SingleMessageMetadata.Builder singleMessageMetadataBuilder = PulsarApi.SingleMessageMetadata
                        .newBuilder();
                ByteBuf buf = Unpooled.wrappedBuffer(data); // here you need to move out of the loop
                try {
                    ByteBuf singleMessagePayload = Commands.deSerializeSingleMessageInBatch(buf, singleMessageMetadataBuilder, i,
                            batchSize);
                    SingleMessageMetadata singleMessageMetadata = singleMessageMetadataBuilder.build();
                    if (singleMessageMetadata.getPropertiesCount() > 0) {
                        for (KeyValue entry : singleMessageMetadata.getPropertiesList()) {
                            properties.put(entry.getKey(), entry.getValue());
                        }
                    }
                    ret.add(new MessageImpl<>(topic, batchMsgId, properties, singleMessagePayload, Schema.BYTES));
                } catch (Exception ex) {
                    log.error("Exception occured while trying to get BatchMsgId: {}", batchMsgId, ex);
                }
                buf.release();
                singleMessageMetadataBuilder.recycle();
            }
    ```
    ByteBuf buf  need to move out of the loop
    
    **Changes**
    Replace old value with new value
    ```javascript
     ByteBuf buf = Unpooled.wrappedBuffer(data);
            for (int i = 0; i < batchSize; i++) {
                String batchMsgId = msgId + ":" + i;
                PulsarApi.SingleMessageMetadata.Builder singleMessageMetadataBuilder = PulsarApi.SingleMessageMetadata
                        .newBuilder();
                try {
                    ByteBuf singleMessagePayload =
                            Commands.deSerializeSingleMessageInBatch(buf, singleMessageMetadataBuilder, i, batchSize);
                    SingleMessageMetadata singleMessageMetadata = singleMessageMetadataBuilder.build();
                    if (singleMessageMetadata.getPropertiesCount() > 0) {
                        for (KeyValue entry : singleMessageMetadata.getPropertiesList()) {
                            properties.put(entry.getKey(), entry.getValue());
                        }
                    }
                    ret.add(new MessageImpl<>(topic, batchMsgId, properties, singleMessagePayload, Schema.BYTES));
                } catch (Exception ex) {
                    log.error("Exception occured while trying to get BatchMsgId: {}", batchMsgId, ex);
                }
                singleMessageMetadataBuilder.recycle();
            }
            buf.release();
    ```
    (cherry picked from commit 565e3196e0be0b52f78137fddcb0cfde655ee010)
---
 .../pulsar/broker/admin/AdminTopicApiTest.java     | 103 +++++++++++++++++++++
 .../pulsar/client/admin/internal/TopicsImpl.java   |   4 +-
 2 files changed, 105 insertions(+), 2 deletions(-)

diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTopicApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTopicApiTest.java
new file mode 100644
index 0000000..2c4b7fc
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTopicApiTest.java
@@ -0,0 +1,103 @@
+/**
+ * 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 lombok.Cleanup;
+
+import org.apache.pulsar.broker.admin.v2.PersistentTopics;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import javax.ws.rs.core.Response;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+public class AdminTopicApiTest extends ProducerConsumerBase {
+    private static final Logger log = LoggerFactory.getLogger(AdminTopicApiTest.class);
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testPeekMessages() throws Exception {
+        @Cleanup
+        PulsarClient newPulsarClient = PulsarClient.builder()
+            .serviceUrl(lookupUrl.toString())
+            .build();
+
+        final String topic = "persistent://my-property/my-ns/test-publish-timestamp";
+
+        @Cleanup
+        Consumer<byte[]> consumer = newPulsarClient.newConsumer()
+            .topic(topic)
+            .subscriptionName("my-sub")
+            .subscribe();
+
+        final int numMessages = 5;
+
+        @Cleanup
+        Producer<byte[]> producer = newPulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)
+                .batchingMaxPublishDelay(3, TimeUnit.SECONDS)
+                .batchingMaxMessages(5)
+                .create();
+
+        for (int i = 0; i < numMessages; i++) {
+            producer.newMessage()
+                .value(("value-" + i).getBytes(UTF_8))
+                .sendAsync();
+        }
+        producer.flush();
+
+        for (int i = 0; i < numMessages; i++) {
+            Message<byte[]> msg = consumer.receive();
+            log.info("Received message '{}'.", new String(msg.getValue(), UTF_8));
+        }
+        List<Message<byte[]>> messages = admin.topics().peekMessages(topic, "my-sub", 5);
+        Assert.assertEquals(new String(messages.get(0).getValue(), UTF_8), "value-0");
+        Assert.assertEquals(new String(messages.get(1).getValue(), UTF_8), "value-1");
+        Assert.assertEquals(new String(messages.get(2).getValue(), UTF_8), "value-2");
+        Assert.assertEquals(new String(messages.get(3).getValue(), UTF_8), "value-3");
+        Assert.assertEquals(new String(messages.get(4).getValue(), UTF_8), "value-4");
+    }
+}
diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java
index 3a1e685..0b9b3d6 100644
--- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java
+++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java
@@ -1011,11 +1011,11 @@ public class TopicsImpl extends BaseResource implements Topics {
                                                              Map<String, String> properties) {
         List<Message<byte[]>> ret = new ArrayList<>();
         int batchSize = Integer.parseInt(properties.get(BATCH_HEADER));
+        ByteBuf buf = Unpooled.wrappedBuffer(data);
         for (int i = 0; i < batchSize; i++) {
             String batchMsgId = msgId + ":" + i;
             PulsarApi.SingleMessageMetadata.Builder singleMessageMetadataBuilder = PulsarApi.SingleMessageMetadata
                     .newBuilder();
-            ByteBuf buf = Unpooled.wrappedBuffer(data);
             try {
                 ByteBuf singleMessagePayload = Commands.deSerializeSingleMessageInBatch(buf, singleMessageMetadataBuilder, i,
                         batchSize);
@@ -1029,9 +1029,9 @@ public class TopicsImpl extends BaseResource implements Topics {
             } catch (Exception ex) {
                 log.error("Exception occured while trying to get BatchMsgId: {}", batchMsgId, ex);
             }
-            buf.release();
             singleMessageMetadataBuilder.recycle();
         }
+        buf.release();
         return ret;
     }
 


[pulsar] 10/38: fix_msgMetadata_not_recycle (#6745)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 8c2d1cf5251a16404437ed779ea09ca3062e7cb0
Author: liudezhi <33...@users.noreply.github.com>
AuthorDate: Thu Apr 16 23:30:07 2020 +0800

    fix_msgMetadata_not_recycle (#6745)
    
    **Motivation**
    fix when producing encrypted messages for inspection, MessageMetadata objects are not released after they are created. #6744
    ```javascript
    if (topic.isEncryptionRequired()) {
    
                headersAndPayload.markReaderIndex();
                MessageMetadata msgMetadata = Commands.parseMessageMetadata(headersAndPayload);
                headersAndPayload.resetReaderIndex();
                // Check whether the message is encrypted or not
                if (msgMetadata.getEncryptionKeysCount() < 1) {
                    log.warn("[{}] Messages must be encrypted", getTopic().getName());
                    cnx.ctx().channel().eventLoop().execute(() -> {
                        cnx.ctx().writeAndFlush(Commands.newSendError(producerId, sequenceId, ServerError.MetadataError,
                                "Messages must be encrypted"));
                        cnx.completedSendOperation(isNonPersistentTopic, headersAndPayload.readableBytes());
                    });
                    return;
                }
            }
        }
    ```
    MessageMetadata was not recycled
    
    **Changes**
    Replace old value with new value
    ```javascript
     if (topic.isEncryptionRequired()) {
    
                headersAndPayload.markReaderIndex();
                MessageMetadata msgMetadata = Commands.parseMessageMetadata(headersAndPayload);
                headersAndPayload.resetReaderIndex();
                int encryptionKeysCount = msgMetadata.getEncryptionKeysCount();
                metadata.recycle();
                // Check whether the message is encrypted or not
                if (encryptionKeysCount < 1) {
                    log.warn("[{}] Messages must be encrypted", getTopic().getName());
                    cnx.ctx().channel().eventLoop().execute(() -> {
                        cnx.ctx().writeAndFlush(Commands.newSendError(producerId, sequenceId, ServerError.MetadataError,
                                "Messages must be encrypted"));
                        cnx.completedSendOperation(isNonPersistentTopic, headersAndPayload.readableBytes());
                    });
                    return;
                }
            }
    ```
    
    * fix_msgMetadata_not_recycle
    
    Co-authored-by: dezhiliu <de...@tencent.com>(cherry picked from commit dadb878d176b8fb72a910880a316b2926d2d5466)
---
 .../src/main/java/org/apache/pulsar/broker/service/Producer.java     | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java
index a7c97da..c6cbada 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java
@@ -180,9 +180,10 @@ public class Producer {
             headersAndPayload.markReaderIndex();
             MessageMetadata msgMetadata = Commands.parseMessageMetadata(headersAndPayload);
             headersAndPayload.resetReaderIndex();
-
+            int encryptionKeysCount = msgMetadata.getEncryptionKeysCount();
+            msgMetadata.recycle();
             // Check whether the message is encrypted or not
-            if (msgMetadata.getEncryptionKeysCount() < 1) {
+            if (encryptionKeysCount < 1) {
                 log.warn("[{}] Messages must be encrypted", getTopic().getName());
                 cnx.ctx().channel().eventLoop().execute(() -> {
                     cnx.ctx().writeAndFlush(Commands.newSendError(producerId, sequenceId, ServerError.MetadataError,


[pulsar] 30/38: [Broker] Handle all exceptions from `topic.addProducer` (#6881)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 1856f2f8a41f092d5758f88f230c4a3b32e59f85
Author: Addison Higham <ad...@gmail.com>
AuthorDate: Wed May 6 01:15:19 2020 -0600

    [Broker] Handle all exceptions from `topic.addProducer` (#6881)
    
    Fixes #6872
    Fixes #6416
    
    If a producer tries to create a producer to a topic that is currently
    unloading, we can get a `RuntimeException` from
    `BrokerService.checkTopicNsOwnership` which is bubbled up through
    `topic.addProducer`. By only handling a `BrokerServiceException` this
    results in a future that never completes and results in producers not
    being able to be created if this topic is scheduled back to this broker.
    (cherry picked from commit 30e26f84fbd35c2f88665195c426a73336a6fc1f)
---
 .../src/main/java/org/apache/pulsar/broker/service/ServerCnx.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
index ce8411e..db61f8d 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
@@ -993,7 +993,7 @@ public class ServerCnx extends PulsarHandler {
                                 ServerError error = null;
                                 if(!existingProducerFuture.isDone()) {
                                     error = ServerError.ServiceNotReady;
-                                }else {
+                                } else {
                                     error = getErrorCode(existingProducerFuture);
                                     // remove producer with producerId as it's already completed with exception
                                     producers.remove(producerId);
@@ -1077,7 +1077,7 @@ public class ServerCnx extends PulsarHandler {
                                         producerFuture.completeExceptionally(
                                             new IllegalStateException("Producer created after connection was closed"));
                                     }
-                                } catch (BrokerServiceException ise) {
+                                } catch (Exception ise) {
                                     log.error("[{}] Failed to add producer to topic {}: {}", remoteAddress, topicName,
                                         ise.getMessage());
                                     ctx.writeAndFlush(Commands.newError(requestId,


[pulsar] 18/38: [broker] register loadbalance znode should attempt to wait until session expired (#6788)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 4cb536ba25a57f83febdb7ae1cd1d2b84d02d7a8
Author: Sijie Guo <si...@apache.org>
AuthorDate: Wed Apr 22 17:47:24 2020 -0700

    [broker] register loadbalance znode should attempt to wait until session expired (#6788)
    
    *Motivation*
    
    Exceptions `Broker-znode owned by different zk-session` is commonly seen when a broker
    starts up in Kubernetes environment. That's because the previous znode is not expired.
    (cherry picked from commit b119611f18afc159f35c993ef48ad0a5e6537707)
---
 .../loadbalance/impl/ModularLoadManagerImpl.java   | 37 +++++-----
 .../loadbalance/impl/SimpleLoadManagerImpl.java    | 30 +++++----
 .../java/org/apache/pulsar/zookeeper/ZkUtils.java  | 78 ++++++++++++++++++++++
 3 files changed, 112 insertions(+), 33 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java
index c3013a0..e9c896d 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java
@@ -798,18 +798,24 @@ public class ModularLoadManagerImpl implements ModularLoadManager, ZooKeeperCach
             final String timeAverageZPath = TIME_AVERAGE_BROKER_ZPATH + "/" + lookupServiceAddress;
             updateLocalBrokerData();
             try {
-                ZkUtils.createFullPathOptimistic(zkClient, brokerZnodePath, localData.getJsonBytes(),
+                if (!org.apache.pulsar.zookeeper.ZkUtils.checkNodeAndWaitExpired(
+                    zkClient, brokerZnodePath,
+                    pulsar.getConfig().getZooKeeperSessionTimeoutMillis())) {
+                    ZkUtils.createFullPathOptimistic(zkClient, brokerZnodePath, localData.getJsonBytes(),
                         ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
-            } catch (KeeperException.NodeExistsException e) {
-                long ownerZkSessionId = getBrokerZnodeOwner();
-                if (ownerZkSessionId != 0 && ownerZkSessionId != zkClient.getSessionId()) {
-                    log.error("Broker znode - [{}] is own by different zookeeper-ssession {} ", brokerZnodePath,
-                            ownerZkSessionId);
-                    throw new PulsarServerException(
-                            "Broker-znode owned by different zk-session " + ownerZkSessionId);
+                } else {
+                    // Node may already be created by another load manager: in this case update the data.
+                    zkClient.setData(brokerZnodePath, localData.getJsonBytes(), -1);
                 }
-                // Node may already be created by another load manager: in this case update the data.
-                zkClient.setData(brokerZnodePath, localData.getJsonBytes(), -1);
+            } catch (KeeperException.NodeExistsException e) {
+                log.error("Broker znode - [{}] is own by different zookeeper-session", brokerZnodePath);
+                throw new PulsarServerException(
+                    "Broker znode - [" + brokerZnodePath + "] is owned by different zk-session");
+            } catch (InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                // Catching exception here to print the right error message
+                log.error("Interrupted at creating znode - [{}] for load balance on zookeeper ", brokerZnodePath, ie);
+                throw ie;
             } catch (Exception e) {
                 // Catching exception here to print the right error message
                 log.error("Unable to create znode - [{}] for load balance on zookeeper ", brokerZnodePath, e);
@@ -939,17 +945,6 @@ public class ModularLoadManagerImpl implements ModularLoadManager, ZooKeeperCach
         }
     }
 
-    private long getBrokerZnodeOwner() {
-        try {
-            Stat stat = new Stat();
-            zkClient.getData(brokerZnodePath, false, stat);
-            return stat.getEphemeralOwner();
-        } catch (Exception e) {
-            log.warn("Failed to get stat of {}", brokerZnodePath, e);
-        }
-        return 0;
-    }
-
     private void refreshBrokerToFailureDomainMap() {
         if (!pulsar.getConfiguration().isFailureDomainsEnabled()) {
             return;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java
index 765f6c6..56fe777 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java
@@ -301,20 +301,26 @@ public class SimpleLoadManagerImpl implements LoadManager, ZooKeeperCacheListene
                 loadReportJson = ObjectMapperFactory.getThreadLocal().writeValueAsString(loadReport);
             }
             try {
-                ZkUtils.createFullPathOptimistic(pulsar.getZkClient(), brokerZnodePath,
+                if (!org.apache.pulsar.zookeeper.ZkUtils.checkNodeAndWaitExpired(
+                    pulsar.getZkClient(), brokerZnodePath,
+                    pulsar.getConfig().getZooKeeperSessionTimeoutMillis())) {
+                    ZkUtils.createFullPathOptimistic(pulsar.getZkClient(), brokerZnodePath,
                         loadReportJson.getBytes(Charsets.UTF_8), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
-            } catch (KeeperException.NodeExistsException e) {
-                long ownerZkSessionId = getBrokerZnodeOwner();
-                if (ownerZkSessionId != 0 && ownerZkSessionId != pulsar.getZkClient().getSessionId()) {
-                    log.error("Broker znode - [{}] is own by different zookeeper-ssession {} ", brokerZnodePath,
-                            ownerZkSessionId);
-                    throw new PulsarServerException("Broker-znode owned by different zk-session " + ownerZkSessionId);
-                }
-                // Node may already be created by another load manager: in this case update the data.
-                if (loadReport != null) {
-                    pulsar.getZkClient().setData(brokerZnodePath, loadReportJson.getBytes(Charsets.UTF_8), -1);
+                } else {
+                    // Node may already be created by another load manager: in this case update the data.
+                    if (loadReport != null) {
+                        pulsar.getZkClient().setData(brokerZnodePath, loadReportJson.getBytes(Charsets.UTF_8), -1);
+                    }
                 }
-
+            } catch (KeeperException.NodeExistsException e) {
+                log.error("Broker znode - [{}] is own by different zookeeper-session", brokerZnodePath);
+                throw new PulsarServerException(
+                    "Broker znode - [" + brokerZnodePath + "] is owned by different zk-session");
+            } catch (InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                // Catching exception here to print the right error message
+                log.error("Interrupted at creating znode - [{}] for load balance on zookeeper ", brokerZnodePath, ie);
+                throw ie;
             } catch (Exception e) {
                 // Catching excption here to print the right error message
                 log.error("Unable to create znode - [{}] for load balance on zookeeper ", brokerZnodePath, e);
diff --git a/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZkUtils.java b/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZkUtils.java
new file mode 100644
index 0000000..d9e8ad4
--- /dev/null
+++ b/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZkUtils.java
@@ -0,0 +1,78 @@
+/**
+ * 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.zookeeper;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NodeExistsException;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * ZooKeeper utils.
+ */
+public final class ZkUtils {
+
+    private static final Logger log = LoggerFactory.getLogger(ZkUtils.class);
+
+    /**
+     * Check if the provided <i>path</i> exists or not and wait it expired if possible.
+     *
+     * @param zk the zookeeper client instance
+     * @param path the zookeeper path
+     * @param sessionTimeoutMs session timeout in milliseconds
+     * @return true if path exists, otherwise return false
+     * @throws KeeperException when failed to access zookeeper
+     * @throws InterruptedException interrupted when waiting for znode to be expired
+     */
+    public static boolean checkNodeAndWaitExpired(ZooKeeper zk,
+                                                  String path,
+                                                  long sessionTimeoutMs) throws KeeperException, InterruptedException {
+        final CountDownLatch prevNodeLatch = new CountDownLatch(1);
+        Watcher zkPrevNodeWatcher = watchedEvent -> {
+            // check for prev node deletion.
+            if (EventType.NodeDeleted == watchedEvent.getType()) {
+                prevNodeLatch.countDown();
+            }
+        };
+        Stat stat = zk.exists(path, zkPrevNodeWatcher);
+        if (null != stat) {
+            // if the ephemeral owner isn't current zookeeper client
+            // wait for it to be expired
+            if (stat.getEphemeralOwner() != zk.getSessionId()) {
+                log.info("Previous znode : {} still exists, so waiting {} ms for znode deletion",
+                    path, sessionTimeoutMs);
+                if (!prevNodeLatch.await(sessionTimeoutMs, TimeUnit.MILLISECONDS)) {
+                    throw new NodeExistsException(path);
+                } else {
+                    return false;
+                }
+            }
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+}


[pulsar] 09/38: [pulsar-broker] avoid backpressure by skipping dispatching if consumer channel is not writable (#6740)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 0d5d0c1559d695e870aec1feb970b5fb01e8f2b7
Author: Rajan Dhabalia <rd...@apache.org>
AuthorDate: Wed Apr 15 16:00:14 2020 -0700

    [pulsar-broker] avoid backpressure by skipping dispatching if consumer channel is not writable (#6740)
    
    ### Motivation
    Recently we are seeing broker is crashing with OutOfMemory when it has higher dispatch rate with large size messages. High message rate out saturates network and broker will try to write on the channel which is not writable which buffers the message and eventually broker sees OOM and shutdown with below error:
    ```
    java.lang.OutOfMemoryError: Direct buffer memory
            at java.nio.Bits.reserveMemory(Bits.java:175) ~[?:?]
            at java.nio.DirectByteBuffer.<init>(DirectByteBuffer.java:118) ~[?:?]
            at java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:317) ~[?:?]
            at io.netty.buffer.PoolArena$DirectArena.allocateDirect(PoolArena.java:769) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:745) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:244) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.buffer.PoolArena.allocate(PoolArena.java:226) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.buffer.PoolArena.allocate(PoolArena.java:146) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:324) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:185) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at org.apache.bookkeeper.common.allocator.impl.ByteBufAllocatorImpl.newDirectBuffer(ByteBufAllocatorImpl.java:164) ~[bookkeeper-common-allocator-4.9.4.2-yahoo.jar:4.9.4.2-yahoo]
            at org.apache.bookkeeper.common.allocator.impl.ByteBufAllocatorImpl.newDirectBuffer(ByteBufAllocatorImpl.java:158) ~[bookkeeper-common-allocator-4.9.4.2-yahoo.jar:4.9.4.2-yahoo]
            at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:185) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:176) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.handler.ssl.SslHandler.allocate(SslHandler.java:1912) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.handler.ssl.SslHandler.allocateOutNetBuf(SslHandler.java:1923) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.handler.ssl.SslHandler.wrap(SslHandler.java:826) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.handler.ssl.SslHandler.wrapAndFlush(SslHandler.java:797) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.handler.ssl.SslHandler.flush(SslHandler.java:778) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.channel.AbstractChannelHandlerContext.invokeFlush0(AbstractChannelHandlerContext.java:776) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.channel.AbstractChannelHandlerContext.invokeFlush(AbstractChannelHandlerContext.java:768) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.channel.AbstractChannelHandlerContext.flush(AbstractChannelHandlerContext.java:749) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.channel.ChannelOutboundHandlerAdapter.flush(ChannelOutboundHandlerAdapter.java:115) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.channel.AbstractChannelHandlerContext.invokeFlush0(AbstractChannelHandlerContext.java:776) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.channel.AbstractChannelHandlerContext.invokeWriteAndFlush(AbstractChannelHandlerContext.java:802) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:814) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.channel.AbstractChannelHandlerContext.writeAndFlush(AbstractChannelHandlerContext.java:794) ~[netty-all-4.1.32.Final.jar:4.1.32.Final]
            at org.apache.pulsar.broker.service.Consumer.lambda$sendMessages$51(Consumer.java:265) ~[pulsar-broker-2.4.6-yahoo.jar:2.4.6-yahoo]
            at io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) [netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404) [netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:335) [netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:909) [netty-all-4.1.32.Final.jar:4.1.32.Final]
            at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [netty-all-4.1.32.Final.jar:4.1.32.Final]
            at java.lang.Thread.run(Thread.java:834) [?:?]
    ```
    
    ### Modification
    In order to reduce backpressure, broker should slow down dispatching if consumer cnx-channel is writable. Broker does it for replicator and Exclusive consumer but not doing for shared consumer. So, add similar check for shared subscription to avoid OOM for high dispatch rate. It might be helpful for #5896 as well.
    (cherry picked from commit 1fd1b2b440af2477f916999a67752f9f532d1620)
---
 .../PersistentDispatcherMultipleConsumers.java     | 27 +++++++++++++++++++++-
 ...istentStickyKeyDispatcherMultipleConsumers.java |  7 +++++-
 2 files changed, 32 insertions(+), 2 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java
index 08a3a01..b69ee94 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java
@@ -255,6 +255,14 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul
         if (totalAvailablePermits > 0 && isAtleastOneConsumerAvailable()) {
             int messagesToRead = Math.min(totalAvailablePermits, readBatchSize);
 
+            if (!isConsumerWritable()) {
+                // If the connection is not currently writable, we issue the read request anyway, but for a single
+                // message. The intent here is to keep use the request as a notification mechanism while avoiding to
+                // read and dispatch a big batch of messages which will need to wait before getting written to the
+                // socket.
+                messagesToRead = 1;
+            }
+
             // throttle only if: (1) cursor is not active (or flag for throttle-nonBacklogConsumer is enabled) bcz
             // active-cursor reads message from cache rather from bookkeeper (2) if topic has reached message-rate
             // threshold: then schedule the read after MESSAGE_RATE_BACKOFF_MS
@@ -481,8 +489,13 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul
             }
 
             // round-robin dispatch batch size for this consumer
+            int availablePermits = c.isWritable() ? c.getAvailablePermits() : 1;
+            if (log.isDebugEnabled() && !c.isWritable()) {
+                log.debug("[{}-{}] consumer is not writable. dispatching only 1 message to {} ", topic.getName(), name,
+                        c);
+            }
             int messagesForC = Math.min(
-                    Math.min(entriesToDispatch, c.getAvailablePermits()),
+                    Math.min(entriesToDispatch, availablePermits),
                     serviceConfig.getDispatcherMaxRoundRobinBatchSize());
 
             if (messagesForC > 0) {
@@ -619,6 +632,18 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul
         return false;
     }
 
+    private boolean isConsumerWritable() {
+        for (Consumer consumer : consumerList) {
+            if (consumer.isWritable()) {
+                return true;
+            }
+        }
+        if (log.isDebugEnabled()) {
+            log.debug("[{}-{}] consumer is not writable", topic.getName(), name);
+        }
+        return false;
+    }
+
     @Override
     public boolean isConsumerAvailable(Consumer consumer) {
         return consumer != null && !consumer.isBlocked() && consumer.getAvailablePermits() > 0;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java
index b8b85ab..a0b23b3 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java
@@ -92,7 +92,12 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi
                 return;
             }
 
-            int messagesForC = Math.min(entriesWithSameKey.getValue().size(), consumer.getAvailablePermits());
+            int availablePermits = consumer.isWritable() ? consumer.getAvailablePermits() : 1;
+            if (log.isDebugEnabled() && !consumer.isWritable()) {
+                log.debug("[{}-{}] consumer is not writable. dispatching only 1 message to {} ", topic.getName(), name,
+                        consumer);
+            }
+            int messagesForC = Math.min(entriesWithSameKey.getValue().size(), availablePermits);
             if (log.isDebugEnabled()) {
                 log.debug("[{}] select consumer {} for key {} with messages num {}, read type is {}",
                         name, consumer.consumerName(), entriesWithSameKey.getKey(), messagesForC, readType);


[pulsar] 06/38: Fix deadlock by consumer and reader (#6728)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 0fd02bd7569cbeecd8e512b05ab958443738d308
Author: k2la <mz...@gmail.com>
AuthorDate: Tue Apr 14 09:11:51 2020 +0900

    Fix deadlock by consumer and reader (#6728)
    
    ### Motivation
    
    Broker servers were not able to connect clients when consumers and readers connected to broker servers at almost the same time.
    This happened in v2.4.2 and master branch.
    
    As the following threaddump at that time:
    ```
    "bookkeeper-ml-workers-OrderedExecutor-5-0" #52 prio=5 os_prio=0 tid=0x00007ff425fd0800 nid=0x28bf waiting on condition [0x00007ff3478f6000]
       java.lang.Thread.State: WAITING (parking)
            at sun.misc.Unsafe.park(Native Method)
            - parking to wait for  <0x0000000750c51a00> (a org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap$Section)
            at java.util.concurrent.locks.StampedLock.acquireWrite(StampedLock.java:1119)
            at java.util.concurrent.locks.StampedLock.writeLock(StampedLock.java:354)
            at org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap$Section.put(ConcurrentOpenHashMap.java:245)
            at org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap.computeIfAbsent(ConcurrentOpenHashMap.java:129)
            at org.apache.pulsar.broker.service.persistent.PersistentTopic$2.openCursorComplete(PersistentTopic.java:638)
            at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncOpenCursor(ManagedLedgerImpl.java:712)
            - locked <0x0000000750c53a00> (a org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl)
            at org.apache.pulsar.broker.service.persistent.PersistentTopic.getDurableSubscription(PersistentTopic.java:631)
            at org.apache.pulsar.broker.service.persistent.PersistentTopic.subscribe(PersistentTopic.java:578)
            at org.apache.pulsar.broker.service.ServerCnx.lambda$null$10(ServerCnx.java:699)
            at org.apache.pulsar.broker.service.ServerCnx$$Lambda$459/848410492.apply(Unknown Source)
            at java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:966)
            at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:940)
            at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
            at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
            at org.apache.pulsar.broker.service.BrokerService$2.lambda$openLedgerComplete$1(BrokerService.java:687)
            at org.apache.pulsar.broker.service.BrokerService$2$$Lambda$229/1013432130.run(Unknown Source)
            at java.util.concurrent.CompletableFuture.uniRun(CompletableFuture.java:719)
            at java.util.concurrent.CompletableFuture.uniRunStage(CompletableFuture.java:731)
            at java.util.concurrent.CompletableFuture.thenRun(CompletableFuture.java:2023)
            at org.apache.pulsar.broker.service.BrokerService$2.openLedgerComplete(BrokerService.java:680)
            at org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl.lambda$asyncOpen$7(ManagedLedgerFactoryImpl.java:328)
            at org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl$$Lambda$184/272111809.accept(Unknown Source)
            at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
            at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
            at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
            at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
            at org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl$2.initializeComplete(ManagedLedgerFactoryImpl.java:316)
            at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl$3$1.operationComplete(ManagedLedgerImpl.java:464)
            at org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$1.operationComplete(ManagedCursorImpl.java:276)
            at org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$1.operationComplete(ManagedCursorImpl.java:249)
            at org.apache.bookkeeper.mledger.impl.MetaStoreImplZookeeper.lambda$null$7(MetaStoreImplZookeeper.java:241)
            at org.apache.bookkeeper.mledger.impl.MetaStoreImplZookeeper$$Lambda$584/1125537287.run(Unknown Source)
            at org.apache.bookkeeper.mledger.util.SafeRun$1.safeRun(SafeRun.java:32)
            at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36)
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
            at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
            at java.lang.Thread.run(Thread.java:748)
    
    ...
    
    "ForkJoinPool.commonPool-worker-36" #1043 daemon prio=5 os_prio=0 tid=0x00007ff34c0ce800 nid=0x26f2 waiting for monitor entry [0x00007ff32d2eb000]
       java.lang.Thread.State: BLOCKED (on object monitor)
            at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.newNonDurableCursor(ManagedLedgerImpl.java:856)
            - waiting to lock <0x0000000750c53a00> (a org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl)
            at org.apache.pulsar.broker.service.persistent.PersistentTopic.lambda$getNonDurableSubscription$13(PersistentTopic.java:684)
            at org.apache.pulsar.broker.service.persistent.PersistentTopic$$Lambda$572/174683985.apply(Unknown Source)
            at org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap$Section.put(ConcurrentOpenHashMap.java:274)
            at org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap.computeIfAbsent(ConcurrentOpenHashMap.java:129)
            at org.apache.pulsar.broker.service.persistent.PersistentTopic.getNonDurableSubscription(PersistentTopic.java:667)
            at org.apache.pulsar.broker.service.persistent.PersistentTopic.subscribe(PersistentTopic.java:579)
            at org.apache.pulsar.broker.service.ServerCnx.lambda$null$10(ServerCnx.java:699)
            at org.apache.pulsar.broker.service.ServerCnx$$Lambda$459/848410492.apply(Unknown Source)
            at java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:995)
            at java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2137)
            at org.apache.pulsar.broker.service.ServerCnx.lambda$null$13(ServerCnx.java:682)
            at org.apache.pulsar.broker.service.ServerCnx$$Lambda$458/375938934.apply(Unknown Source)
            at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616)
            at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591)
            at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
            at java.util.concurrent.CompletableFuture.postFire(CompletableFuture.java:575)
            at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:943)
            at java.util.concurrent.CompletableFuture$Completion.exec(CompletableFuture.java:457)
            at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
            at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
            at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
            at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:157)
    ```
    `PersistentTopic#getDurableSubscription` locked  `ConcurrentOpenHashMap` after locking `ManagedLedgerImpl`.
    ( `ManagedLedgerImpl` => `ConcurrentOpenHashMap`)
    
    On the other hand, `PersistentTopic#getNonDurableSubscription` tried to lock `ManagedLedgerImpl` after trying to lock `ConcurrentOpenHashMap`.
    ( `ConcurrentOpenHashMap` => `ManagedLedgerImpl`)
    
    So, it seems that deadlock happens.
    
    ### Modifications
    Fixed as `PersistentTopic#getNonDurableSubscription` try to lock `ConcurrentOpenHashMap` after trying to lock `ManagedLedgerImpl`. ( `ManagedLedgerImpl` => `ConcurrentOpenHashMap`)
    (cherry picked from commit 6d304140b6205c6a2e94ad34bfd3cc5d16aca5d1)
---
 .../broker/service/persistent/PersistentTopic.java | 76 +++++++++++-----------
 1 file changed, 39 insertions(+), 37 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
index 13a076a..18d371b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
@@ -671,49 +671,51 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal
         CompletableFuture<Subscription> subscriptionFuture = new CompletableFuture<>();
         log.info("[{}][{}] Creating non-durable subscription at msg id {}", topic, subscriptionName, startMessageId);
 
-        // Create a new non-durable cursor only for the first consumer that connects
-        Subscription subscription = subscriptions.computeIfAbsent(subscriptionName, name -> {
-            MessageIdImpl msgId = startMessageId != null ? (MessageIdImpl) startMessageId
-                    : (MessageIdImpl) MessageId.latest;
-
-            long ledgerId = msgId.getLedgerId();
-            long entryId = msgId.getEntryId();
-            if (ledgerId >= 0
-                && msgId instanceof BatchMessageIdImpl) {
-                // When the start message is relative to a batch, we need to take one step back on the previous message,
-                // because the "batch" might not have been consumed in its entirety.
-                // The client will then be able to discard the first messages if needed.
-                entryId = msgId.getEntryId() - 1;
-            }
+        synchronized (ledger) {
+            // Create a new non-durable cursor only for the first consumer that connects
+            Subscription subscription = subscriptions.computeIfAbsent(subscriptionName, name -> {
+                MessageIdImpl msgId = startMessageId != null ? (MessageIdImpl) startMessageId
+                        : (MessageIdImpl) MessageId.latest;
+
+                long ledgerId = msgId.getLedgerId();
+                long entryId = msgId.getEntryId();
+                if (ledgerId >= 0
+                        && msgId instanceof BatchMessageIdImpl) {
+                    // When the start message is relative to a batch, we need to take one step back on the previous message,
+                    // because the "batch" might not have been consumed in its entirety.
+                    // The client will then be able to discard the first messages if needed.
+                    entryId = msgId.getEntryId() - 1;
+                }
 
-            Position startPosition = new PositionImpl(ledgerId, entryId);
-            ManagedCursor cursor = null;
-            try {
-                cursor = ledger.newNonDurableCursor(startPosition, subscriptionName);
-            } catch (ManagedLedgerException e) {
-                subscriptionFuture.completeExceptionally(e);
-            }
+                Position startPosition = new PositionImpl(ledgerId, entryId);
+                ManagedCursor cursor = null;
+                try {
+                    cursor = ledger.newNonDurableCursor(startPosition, subscriptionName);
+                } catch (ManagedLedgerException e) {
+                    subscriptionFuture.completeExceptionally(e);
+                }
 
-            return new PersistentSubscription(this, subscriptionName, cursor, false);
-        });
+                return new PersistentSubscription(this, subscriptionName, cursor, false);
+            });
 
-        if (!subscriptionFuture.isDone()) {
-            if (startMessageRollbackDurationSec > 0) {
-                long timestamp = System.currentTimeMillis() - TimeUnit.SECONDS.toMillis(startMessageRollbackDurationSec);
-                subscription.resetCursor(timestamp).handle((s, ex) -> {
-                    if (ex != null) {
-                        log.warn("[{}] Failed to reset cursor {} position at timestamp {}", topic, subscriptionName,
-                                startMessageRollbackDurationSec);
-                    }
+            if (!subscriptionFuture.isDone()) {
+                if (startMessageRollbackDurationSec > 0) {
+                    long timestamp = System.currentTimeMillis() - TimeUnit.SECONDS.toMillis(startMessageRollbackDurationSec);
+                    subscription.resetCursor(timestamp).handle((s, ex) -> {
+                        if (ex != null) {
+                            log.warn("[{}] Failed to reset cursor {} position at timestamp {}", topic, subscriptionName,
+                                    startMessageRollbackDurationSec);
+                        }
+                        subscriptionFuture.complete(subscription);
+                        return null;
+                    });
+                } else {
                     subscriptionFuture.complete(subscription);
-                    return null;
-                });
+                }
             } else {
-                subscriptionFuture.complete(subscription);
+                // failed to initialize managed-cursor: clean up created subscription
+                subscriptions.remove(subscriptionName);
             }
-        } else {
-            // failed to initialize managed-cursor: clean up created subscription
-            subscriptions.remove(subscriptionName);
         }
 
         return subscriptionFuture;


[pulsar] 37/38: [pulsar-client] Add support to load tls certs/key dynamically from inputstream (#6760)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 77c7f1cf9197f135e3ceb1e596a682fead9b8a37
Author: Rajan Dhabalia <rd...@apache.org>
AuthorDate: Tue Apr 21 17:05:34 2020 -0700

    [pulsar-client] Add support to load tls certs/key dynamically from inputstream (#6760)
    
    ### Motivation
    Right now, Pulsar-client provides tls authentication support and default TLS provider `AuthenticationTls` expects file path of cert and key files. However, there are usescases where it will be difficult for user-applications to store certs/key file locally for tls authentication.
    eg:
    1. Applications running on docker or K8s containers will not have certs at defined location and app uses KMS or various key-vault system whose API return streams of certs.
    2. Operationally hard to manage key rotation in containers
    3. Need to avoid storing key/trust store files on file system for stronger security
    
    Therefore, it's good to have mechanism in default `AuthenticationTls` provider to read certs from memory/stream without storing certs on file-system.
    
    ### Modification
    Add Stream support in `AuthenticationTls` to provide X509Certs and PrivateKey which also performs auto-refresh when stream changes in a given provider.
    ```
    AuthenticationTls auth = new AuthenticationTls(certStreamProvider, keyStreamProvider);
    ```
    It will be also address: #5241
    (cherry picked from commit 3b48df1577a43509e1fa9afb01243dd87ea8026e)
---
 .../pulsar/client/api/TlsProducerConsumerTest.java | 111 +++++++++++++++++++++
 .../client/impl/auth/AuthenticationDataTls.java    |  47 ++++++++-
 .../pulsar/client/impl/auth/AuthenticationTls.java |  16 ++-
 .../client/impl/conf/ClientConfigurationData.java  |   3 +
 .../apache/pulsar/common/util/SecurityUtility.java |  46 ++++++++-
 5 files changed, 214 insertions(+), 9 deletions(-)

diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java
index 82aa7d3..55bc4a7 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java
@@ -18,14 +18,26 @@
  */
 package org.apache.pulsar.client.api;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
 import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
 
+import org.apache.commons.compress.utils.IOUtils;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.client.impl.auth.AuthenticationTls;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+import lombok.Cleanup;
+
 public class TlsProducerConsumerTest extends TlsProducerConsumerBase {
     private static final Logger log = LoggerFactory.getLogger(TlsProducerConsumerTest.class);
 
@@ -124,4 +136,103 @@ public class TlsProducerConsumerTest extends TlsProducerConsumerBase {
             Assert.fail("Should not fail since certs are sent.");
         }
     }
+
+    @Test(timeOut = 60000)
+    public void testTlsCertsFromDynamicStream() throws Exception {
+        log.info("-- Starting {} test --", methodName);
+        String topicName = "persistent://my-property/use/my-ns/my-topic1";
+        ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrlTls())
+                .tlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH).enableTls(true).allowTlsInsecureConnection(false)
+                .operationTimeout(1000, TimeUnit.MILLISECONDS);
+        AtomicInteger index = new AtomicInteger(0);
+
+        ByteArrayInputStream certStream = createByteInputStream(TLS_CLIENT_CERT_FILE_PATH);
+        ByteArrayInputStream keyStream = createByteInputStream(TLS_CLIENT_KEY_FILE_PATH);
+
+        Supplier<ByteArrayInputStream> certProvider = () -> getStream(index, certStream);
+        Supplier<ByteArrayInputStream> keyProvider = () -> getStream(index, keyStream);
+        AuthenticationTls auth = new AuthenticationTls(certProvider, keyProvider);
+        clientBuilder.authentication(auth);
+        @Cleanup
+        PulsarClient pulsarClient = clientBuilder.build();
+        Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("my-subscriber-name")
+                .subscribe();
+
+        // unload the topic so, new connection will be made and read the cert streams again
+        PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get();
+        topicRef.close(false);
+
+        Producer<byte[]> producer = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic1")
+                .createAsync().get(30, TimeUnit.SECONDS);
+        for (int i = 0; i < 10; i++) {
+            producer.send(("test" + i).getBytes());
+        }
+
+        Message<byte[]> msg = null;
+        for (int i = 0; i < 10; i++) {
+            msg = consumer.receive(5, TimeUnit.SECONDS);
+            String exepctedMsg = "test" + i;
+            Assert.assertEquals(exepctedMsg.getBytes(), msg.getData());
+        }
+        // Acknowledge the consumption of all messages at once
+        consumer.acknowledgeCumulative(msg);
+        consumer.close();
+        log.info("-- Exiting {} test --", methodName);
+    }
+
+    /**
+     * It verifies that AuthenticationTls provides cert refresh functionality.
+     * 
+     * <pre>
+     *  a. Create Auth with invalid cert
+     *  b. Consumer fails with invalid tls certs
+     *  c. refresh cert in provider
+     *  d. Consumer successfully gets created
+     * </pre>
+     * 
+     * @throws Exception
+     */
+    @Test
+    public void testTlsCertsFromDynamicStreamExpiredAndRenewCert() throws Exception {
+        log.info("-- Starting {} test --", methodName);
+        ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrlTls())
+                .tlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH).enableTls(true).allowTlsInsecureConnection(false)
+                .operationTimeout(1000, TimeUnit.MILLISECONDS);
+        AtomicInteger certIndex = new AtomicInteger(1);
+        AtomicInteger keyIndex = new AtomicInteger(0);
+        ByteArrayInputStream certStream = createByteInputStream(TLS_CLIENT_CERT_FILE_PATH);
+        ByteArrayInputStream keyStream = createByteInputStream(TLS_CLIENT_KEY_FILE_PATH);
+        Supplier<ByteArrayInputStream> certProvider = () -> getStream(certIndex, certStream,
+                keyStream/* invalid cert file */);
+        Supplier<ByteArrayInputStream> keyProvider = () -> getStream(keyIndex, keyStream);
+        AuthenticationTls auth = new AuthenticationTls(certProvider, keyProvider);
+        clientBuilder.authentication(auth);
+        @Cleanup
+        PulsarClient pulsarClient = clientBuilder.build();
+        Consumer<byte[]> consumer = null;
+        try {
+            consumer = pulsarClient.newConsumer().topic("persistent://my-property/use/my-ns/my-topic1")
+                    .subscriptionName("my-subscriber-name").subscribe();
+            Assert.fail("should have failed due to invalid tls cert");
+        } catch (PulsarClientException e) {
+            // Ok..
+        }
+
+        certIndex.set(0);
+        consumer = pulsarClient.newConsumer().topic("persistent://my-property/use/my-ns/my-topic1")
+                .subscriptionName("my-subscriber-name").subscribe();
+        consumer.close();
+        log.info("-- Exiting {} test --", methodName);
+    }
+
+    private ByteArrayInputStream createByteInputStream(String filePath) throws IOException {
+        InputStream inStream = new FileInputStream(filePath);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        IOUtils.copy(inStream, baos);
+        return new ByteArrayInputStream(baos.toByteArray());
+    }
+
+    private ByteArrayInputStream getStream(AtomicInteger index, ByteArrayInputStream... streams) {
+        return streams[index.intValue()];
+    } 
 }
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationDataTls.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationDataTls.java
index e355672..0d3df12 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationDataTls.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationDataTls.java
@@ -18,11 +18,17 @@
  */
 package org.apache.pulsar.client.impl.auth;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
 import java.security.KeyManagementException;
 import java.security.PrivateKey;
 import java.security.cert.Certificate;
 import java.security.cert.X509Certificate;
+import java.util.function.Supplier;
 
+import org.apache.commons.compress.utils.IOUtils;
 import org.apache.pulsar.client.api.AuthenticationDataProvider;
 import org.apache.pulsar.common.util.FileModifiedTimeUpdater;
 import org.apache.pulsar.common.util.SecurityUtility;
@@ -32,7 +38,10 @@ import org.slf4j.LoggerFactory;
 public class AuthenticationDataTls implements AuthenticationDataProvider {
     protected X509Certificate[] tlsCertificates;
     protected PrivateKey tlsPrivateKey;
-    protected FileModifiedTimeUpdater certFile, keyFile;
+    private FileModifiedTimeUpdater certFile, keyFile;
+    // key and cert using stream
+    private InputStream certStream, keyStream;
+    private Supplier<ByteArrayInputStream> certStreamProvider, keyStreamProvider;
 
     public AuthenticationDataTls(String certFilePath, String keyFilePath) throws KeyManagementException {
         if (certFilePath == null) {
@@ -47,6 +56,22 @@ public class AuthenticationDataTls implements AuthenticationDataProvider {
         this.tlsPrivateKey = SecurityUtility.loadPrivateKeyFromPemFile(keyFilePath);
     }
 
+    public AuthenticationDataTls(Supplier<ByteArrayInputStream> certStreamProvider,
+            Supplier<ByteArrayInputStream> keyStreamProvider) throws KeyManagementException {
+        if (certStreamProvider == null || certStreamProvider.get() == null) {
+            throw new IllegalArgumentException("certStream provider or stream must not be null");
+        }
+        if (keyStreamProvider == null || keyStreamProvider.get() == null) {
+            throw new IllegalArgumentException("keyStream provider or stream must not be null");
+        }
+        this.certStreamProvider = certStreamProvider;
+        this.keyStreamProvider = keyStreamProvider;
+        this.certStream = certStreamProvider.get();
+        this.keyStream = keyStreamProvider.get();
+        this.tlsCertificates = SecurityUtility.loadCertificatesFromPemStream(certStream);
+        this.tlsPrivateKey = SecurityUtility.loadPrivateKeyFromPemStream(keyStream);
+    }
+
     /*
      * TLS
      */
@@ -58,24 +83,40 @@ public class AuthenticationDataTls implements AuthenticationDataProvider {
 
     @Override
     public Certificate[] getTlsCertificates() {
-        if (this.certFile.checkAndRefresh()) {
+        if (certFile != null && certFile.checkAndRefresh()) {
             try {
                 this.tlsCertificates = SecurityUtility.loadCertificatesFromPemFile(certFile.getFileName());
             } catch (KeyManagementException e) {
                 LOG.error("Unable to refresh authData for cert {}: ", certFile.getFileName(), e);
             }
+        } else if (certStreamProvider != null && certStreamProvider.get() != null
+                && !certStreamProvider.get().equals(certStream)) {
+            try {
+                certStream = certStreamProvider.get();
+                tlsCertificates = SecurityUtility.loadCertificatesFromPemStream(certStream);
+            } catch (KeyManagementException e) {
+                LOG.error("Unable to refresh authData from cert stream ", e);
+            }
         }
         return this.tlsCertificates;
     }
 
     @Override
     public PrivateKey getTlsPrivateKey() {
-        if (this.keyFile.checkAndRefresh()) {
+        if (keyFile != null && keyFile.checkAndRefresh()) {
             try {
                 this.tlsPrivateKey = SecurityUtility.loadPrivateKeyFromPemFile(keyFile.getFileName());
             } catch (KeyManagementException e) {
                 LOG.error("Unable to refresh authData for cert {}: ", keyFile.getFileName(), e);
             }
+        } else if (keyStreamProvider != null && keyStreamProvider.get() != null
+                && !keyStreamProvider.get().equals(keyStream)) {
+            try {
+                keyStream = keyStreamProvider.get();
+                tlsPrivateKey = SecurityUtility.loadPrivateKeyFromPemStream(keyStream);
+            } catch (KeyManagementException e) {
+                LOG.error("Unable to refresh authData from key stream ", e);
+            }
         }
         return this.tlsPrivateKey;
     }
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationTls.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationTls.java
index d75e491..22cd2f5 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationTls.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/AuthenticationTls.java
@@ -18,8 +18,11 @@
  */
 package org.apache.pulsar.client.impl.auth;
 
+import java.io.ByteArrayInputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.util.Map;
+import java.util.function.Supplier;
 
 import org.apache.pulsar.client.api.Authentication;
 import org.apache.pulsar.client.api.AuthenticationDataProvider;
@@ -42,6 +45,7 @@ public class AuthenticationTls implements Authentication, EncodedAuthenticationP
 
     private String certFilePath;
     private String keyFilePath;
+    private Supplier<ByteArrayInputStream> certStreamProvider, keyStreamProvider;
 
     public AuthenticationTls() {
     }
@@ -51,6 +55,11 @@ public class AuthenticationTls implements Authentication, EncodedAuthenticationP
         this.keyFilePath = keyFilePath;
     }
 
+    public AuthenticationTls(Supplier<ByteArrayInputStream> certStreamProvider, Supplier<ByteArrayInputStream> keyStreamProvider) {
+        this.certStreamProvider = certStreamProvider;
+        this.keyStreamProvider = keyStreamProvider;
+    }
+
     @Override
     public void close() throws IOException {
         // noop
@@ -64,10 +73,15 @@ public class AuthenticationTls implements Authentication, EncodedAuthenticationP
     @Override
     public AuthenticationDataProvider getAuthData() throws PulsarClientException {
         try {
-            return new AuthenticationDataTls(certFilePath, keyFilePath);
+            if (certFilePath != null && keyFilePath != null) {
+                return new AuthenticationDataTls(certFilePath, keyFilePath);
+            } else if (certStreamProvider != null && keyStreamProvider != null) {
+                return new AuthenticationDataTls(certStreamProvider, keyStreamProvider);
+            }
         } catch (Exception e) {
             throw new PulsarClientException(e);
         }
+        throw new IllegalArgumentException("cert/key file path or cert/key stream must be present");
     }
 
     @Override
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java
index af478ce..af6ad8d 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java
@@ -80,6 +80,9 @@ public class ClientConfigurationData implements Serializable, Cloneable {
         return authentication;
     }
 
+    public void setAuthentication(Authentication authentication) {
+        this.authentication = authentication;
+    }
     public boolean isUseTls() {
         if (useTls)
             return true;
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java
index 648e9f2..8d1af4a 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java
@@ -26,8 +26,9 @@ import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
-import java.io.FileReader;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
 import java.security.GeneralSecurityException;
 import java.security.KeyFactory;
 import java.security.KeyManagementException;
@@ -40,6 +41,7 @@ import java.security.SecureRandom;
 import java.security.Security;
 import java.security.UnrecoverableKeyException;
 import java.security.cert.Certificate;
+import java.security.cert.CertificateException;
 import java.security.cert.CertificateFactory;
 import java.security.cert.X509Certificate;
 import java.security.spec.KeySpec;
@@ -242,9 +244,7 @@ public class SecurityUtility {
         }
 
         try (FileInputStream input = new FileInputStream(certFilePath)) {
-            CertificateFactory cf = CertificateFactory.getInstance("X.509");
-            Collection<X509Certificate> collection = (Collection<X509Certificate>) cf.generateCertificates(input);
-            certificates = collection.toArray(new X509Certificate[collection.size()]);
+            certificates = loadCertificatesFromPemStream(input);
         } catch (GeneralSecurityException | IOException e) {
             throw new KeyManagementException("Certificate loading error", e);
         }
@@ -252,6 +252,23 @@ public class SecurityUtility {
         return certificates;
     }
 
+    public static X509Certificate[] loadCertificatesFromPemStream(InputStream inStream) throws KeyManagementException  {
+        if (inStream == null) {
+            return null;
+        }
+        CertificateFactory cf;
+        try {
+            if (inStream.markSupported()) {
+                inStream.reset();
+            }
+            cf = CertificateFactory.getInstance("X.509");
+            Collection<X509Certificate> collection = (Collection<X509Certificate>) cf.generateCertificates(inStream);
+            return collection.toArray(new X509Certificate[collection.size()]);
+        } catch (CertificateException | IOException e) {
+            throw new KeyManagementException("Certificate loading error", e);
+        }
+    }
+
     public static PrivateKey loadPrivateKeyFromPemFile(String keyFilePath) throws KeyManagementException {
         PrivateKey privateKey = null;
 
@@ -259,7 +276,26 @@ public class SecurityUtility {
             return privateKey;
         }
 
-        try (BufferedReader reader = new BufferedReader(new FileReader(keyFilePath))) {
+        try (FileInputStream input = new FileInputStream(keyFilePath)) {
+            privateKey = loadPrivateKeyFromPemStream(input);
+        } catch (IOException e) {
+            throw new KeyManagementException("Private key loading error", e);
+        }
+
+        return privateKey;
+    }
+
+    public static PrivateKey loadPrivateKeyFromPemStream(InputStream inStream) throws KeyManagementException {
+        PrivateKey privateKey = null;
+
+        if (inStream == null) {
+            return privateKey;
+        }
+
+        try (BufferedReader reader = new BufferedReader(new InputStreamReader(inStream))) {
+            if (inStream.markSupported()) {
+                inStream.reset();
+            }
             StringBuilder sb = new StringBuilder();
             String previousLine = "";
             String currentLine = null;


[pulsar] 34/38: Fixed dashboard start failed (#6857)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 2aaf4d269c64c17c4dcd884cd7d5bc4d762d7a1a
Author: guangning <gu...@apache.org>
AuthorDate: Thu May 7 14:00:53 2020 +0800

    Fixed dashboard start failed (#6857)
    
    Master Issue: https://github.com/apache/pulsar/issues/5847
    
    ### Motivation
    
    Installation error of dashboard database postgres 11 resulted in startup failure.
    
    ### Modifications
    
    * Update apachepulsar/pulsar-dashboard:2.5.1 image https://hub.docker.com/layers/apachepulsar/pulsar-dashboard/2.5.1/images/sha256-61b47a7302639aba1357d09ca69a842c4a67bff38b230753d6bd638df0461c6b?context=explore
    * Update Docker file for fix postgresql version 11.
    
    ### Verifying this change
    
    Local test pass
    (cherry picked from commit c621d99b430ef6ddd379eb588eb5ed7ee4e85124)
---
 dashboard/Dockerfile       | 5 ++++-
 dashboard/init-postgres.sh | 2 +-
 2 files changed, 5 insertions(+), 2 deletions(-)

diff --git a/dashboard/Dockerfile b/dashboard/Dockerfile
index 7388481..605e8ba 100644
--- a/dashboard/Dockerfile
+++ b/dashboard/Dockerfile
@@ -21,8 +21,11 @@ FROM python:3.7-stretch
 
 MAINTAINER Pulsar
 
+RUN bash -c "echo deb http://apt.postgresql.org/pub/repos/apt/ stretch-pgdg main >> /etc/apt/sources.list.d/pgdg.list"
+RUN wget --quiet -O - https://www.postgresql.org/media/keys/ACCC4CF8.asc | apt-key add -
+
 RUN apt-get update
-RUN apt-get -y install postgresql python sudo nginx supervisor
+RUN apt-get -y install postgresql-11 postgresql-contrib libpq-dev python sudo nginx supervisor
 
 # Postgres configuration
 COPY conf/postgresql.conf /etc/postgresql/11/main/
diff --git a/dashboard/init-postgres.sh b/dashboard/init-postgres.sh
index 5f4c627..fbb8ddb 100755
--- a/dashboard/init-postgres.sh
+++ b/dashboard/init-postgres.sh
@@ -24,7 +24,7 @@ rm -rf /data/*
 chown -R postgres: /data
 chmod 700 /data
 sudo -u postgres /usr/lib/postgresql/11/bin/initdb /data/
-sudo -u postgres /etc/init.d/postgresql start
+sudo -u postgres /usr/lib/postgresql/11/bin/pg_ctl -D /data/ start
 sudo -u postgres psql --command "CREATE USER docker WITH PASSWORD 'docker';"
 sudo -u postgres createdb -O docker pulsar_dashboard
 


[pulsar] 12/38: Ensure that all dangling consumers are cleaned up during failures (#6778)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 3cfa938bfe06ef12839df7754260f7e0b67e42fa
Author: Sanjeev Kulkarni <sa...@gmail.com>
AuthorDate: Tue Apr 21 17:21:40 2020 -0700

    Ensure that all dangling consumers are cleaned up during failures (#6778)
    
    Co-authored-by: Sanjeev Kulkarni <sa...@splunk.com>(cherry picked from commit 98b818b5fa63ee2e4a67887cf96330ae652dafa2)
---
 .../pulsar/functions/source/PulsarSource.java      |  25 +++--
 .../pulsar/functions/source/PulsarSourceTest.java  | 113 ++++++++++++++++-----
 2 files changed, 103 insertions(+), 35 deletions(-)

diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/source/PulsarSource.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/source/PulsarSource.java
index 0d23ce1..fa7146d 100644
--- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/source/PulsarSource.java
+++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/source/PulsarSource.java
@@ -47,7 +47,7 @@ public class PulsarSource<T> extends PushSource<T> implements MessageListener<T>
     private final Map<String, String> properties;
     private final ClassLoader functionClassLoader;
     private List<String> inputTopics;
-    private List<Consumer<T>> inputConsumers = Collections.emptyList();
+    private List<Consumer<T>> inputConsumers = new LinkedList<>();
     private final TopicSchema topicSchema;
 
     public PulsarSource(PulsarClient pulsarClient, PulsarSourceConfig pulsarConfig, Map<String, String> properties,
@@ -65,7 +65,7 @@ public class PulsarSource<T> extends PushSource<T> implements MessageListener<T>
         log.info("Opening pulsar source with config: {}", pulsarSourceConfig);
         Map<String, ConsumerConfig<T>> configs = setupConsumerConfigs();
 
-        inputConsumers = configs.entrySet().stream().map(e -> {
+        for (Map.Entry<String, ConsumerConfig<T>> e : configs.entrySet()) {
             String topic = e.getKey();
             ConsumerConfig<T> conf = e.getValue();
             log.info("Creating consumers for topic : {}, schema : {}, schemaInfo: {}",
@@ -80,17 +80,17 @@ public class PulsarSource<T> extends PushSource<T> implements MessageListener<T>
                     .messageListener(this);
 
             if (conf.isRegexPattern) {
-                cb.topicsPattern(topic);
+                cb = cb.topicsPattern(topic);
             } else {
-                cb.topic(topic);
+                cb = cb.topics(Collections.singletonList(topic));
             }
             if (conf.getReceiverQueueSize() != null) {
-                cb.receiverQueueSize(conf.getReceiverQueueSize());
+                cb = cb.receiverQueueSize(conf.getReceiverQueueSize());
             }
-            cb.properties(properties);
+            cb = cb.properties(properties);
 
             if (pulsarSourceConfig.getTimeoutMs() != null) {
-                cb.ackTimeout(pulsarSourceConfig.getTimeoutMs(), TimeUnit.MILLISECONDS);
+                cb = cb.ackTimeout(pulsarSourceConfig.getTimeoutMs(), TimeUnit.MILLISECONDS);
             }
 
             if (pulsarSourceConfig.getMaxMessageRetries() != null && pulsarSourceConfig.getMaxMessageRetries() >= 0) {
@@ -99,11 +99,12 @@ public class PulsarSource<T> extends PushSource<T> implements MessageListener<T>
                 if (pulsarSourceConfig.getDeadLetterTopic() != null && !pulsarSourceConfig.getDeadLetterTopic().isEmpty()) {
                     deadLetterPolicyBuilder.deadLetterTopic(pulsarSourceConfig.getDeadLetterTopic());
                 }
-                cb.deadLetterPolicy(deadLetterPolicyBuilder.build());
+                cb = cb.deadLetterPolicy(deadLetterPolicyBuilder.build());
             }
 
-            return cb.subscribeAsync();
-        }).collect(Collectors.toList()).stream().map(CompletableFuture::join).collect(Collectors.toList());
+            Consumer<T> consumer = cb.subscribeAsync().join();
+            inputConsumers.add(consumer);
+        }
 
         inputTopics = inputConsumers.stream().flatMap(c -> {
             return (c instanceof MultiTopicsConsumerImpl) ? ((MultiTopicsConsumerImpl<?>) c).getTopics().stream()
@@ -176,6 +177,10 @@ public class PulsarSource<T> extends PushSource<T> implements MessageListener<T>
         return inputTopics;
     }
 
+    public List<Consumer<T>> getInputConsumers() {
+        return inputConsumers;
+    }
+
     @Data
     @Builder
     private static class ConsumerConfig<T> {
diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/source/PulsarSourceTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/source/PulsarSourceTest.java
index d6e03d3..c2e556c 100644
--- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/source/PulsarSourceTest.java
+++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/source/PulsarSourceTest.java
@@ -20,19 +20,19 @@ package org.apache.pulsar.functions.source;
 
 
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyList;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.ArgumentMatchers.anyString;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.*;
 import static org.testng.AssertJUnit.assertEquals;
 import static org.testng.AssertJUnit.assertTrue;
 import static org.testng.AssertJUnit.fail;
 
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
 
 import lombok.Cleanup;
 import lombok.Getter;
@@ -49,6 +49,7 @@ import org.apache.pulsar.common.functions.ConsumerConfig;
 import org.apache.pulsar.common.functions.FunctionConfig;
 import org.apache.pulsar.functions.api.SerDe;
 import org.apache.pulsar.io.core.SourceContext;
+import org.mockito.ArgumentMatcher;
 import org.testng.annotations.Test;
 
 @Slf4j
@@ -56,11 +57,21 @@ public class PulsarSourceTest {
 
     private static Map<String, ConsumerConfig> consumerConfigs = new HashMap<>();
     static {
-        consumerConfigs.put("persistent://sample/standalone/ns1/test_result", ConsumerConfig.builder()
+        consumerConfigs.put("persistent://sample/ns1/test_result", ConsumerConfig.builder()
                 .serdeClassName(TopicSchema.DEFAULT_SERDE).isRegexPattern(false).build());
     }
 
-    public static class TestSerDe implements SerDe<String> {
+    private static Map<String, ConsumerConfig> multipleConsumerConfigs = new HashMap<>();
+    static {
+        multipleConsumerConfigs.put("persistent://sample/ns1/test_result1", ConsumerConfig.builder()
+                .serdeClassName(TopicSchema.DEFAULT_SERDE).isRegexPattern(false).build());
+        multipleConsumerConfigs.put("persistent://sample/ns1/test_result2", ConsumerConfig.builder()
+                .serdeClassName(TopicSchema.DEFAULT_SERDE).isRegexPattern(false).build());
+        multipleConsumerConfigs.put("persistent://sample/ns1/test_result3", ConsumerConfig.builder()
+                .serdeClassName(TopicSchema.DEFAULT_SERDE).isRegexPattern(false).build());
+    }
+
+        public static class TestSerDe implements SerDe<String> {
 
         @Override
         public String deserialize(byte[] input) {
@@ -78,26 +89,61 @@ public class PulsarSourceTest {
      */
     private static PulsarClientImpl getPulsarClient() throws PulsarClientException {
         PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class);
-        ConsumerBuilder<?> consumerBuilder = mock(ConsumerBuilder.class);
-        doReturn(consumerBuilder).when(consumerBuilder).topics(anyList());
-        doReturn(consumerBuilder).when(consumerBuilder).cryptoFailureAction(any());
-        doReturn(consumerBuilder).when(consumerBuilder).subscriptionName(any());
-        doReturn(consumerBuilder).when(consumerBuilder).subscriptionInitialPosition(any());
-        doReturn(consumerBuilder).when(consumerBuilder).subscriptionType(any());
-        doReturn(consumerBuilder).when(consumerBuilder).ackTimeout(anyLong(), any());
-        doReturn(consumerBuilder).when(consumerBuilder).messageListener(any());
+        ConsumerBuilder<?> goodConsumerBuilder = mock(ConsumerBuilder.class);
+        ConsumerBuilder<?> badConsumerBuilder = mock(ConsumerBuilder.class);
+        doReturn(goodConsumerBuilder).when(goodConsumerBuilder).topics(argThat(new TopicMatcher("persistent://sample/ns1/test_result")));
+        doReturn(goodConsumerBuilder).when(goodConsumerBuilder).topics(argThat(new TopicMatcher("persistent://sample/ns1/test_result1")));
+        doReturn(badConsumerBuilder).when(goodConsumerBuilder).topics(argThat(new TopicMatcher("persistent://sample/ns1/test_result2")));
+        doReturn(goodConsumerBuilder).when(goodConsumerBuilder).topics(argThat(new TopicMatcher("persistent://sample/ns1/test_result3")));
+        doReturn(goodConsumerBuilder).when(goodConsumerBuilder).cryptoFailureAction(any());
+        doReturn(goodConsumerBuilder).when(goodConsumerBuilder).subscriptionName(any());
+        doReturn(goodConsumerBuilder).when(goodConsumerBuilder).subscriptionInitialPosition(any());
+        doReturn(goodConsumerBuilder).when(goodConsumerBuilder).subscriptionType(any());
+        doReturn(goodConsumerBuilder).when(goodConsumerBuilder).ackTimeout(anyLong(), any());
+        doReturn(goodConsumerBuilder).when(goodConsumerBuilder).messageListener(any());
+        doReturn(goodConsumerBuilder).when(goodConsumerBuilder).properties(any());
+        doReturn(badConsumerBuilder).when(badConsumerBuilder).cryptoFailureAction(any());
+        doReturn(badConsumerBuilder).when(badConsumerBuilder).subscriptionName(any());
+        doReturn(badConsumerBuilder).when(badConsumerBuilder).subscriptionInitialPosition(any());
+        doReturn(badConsumerBuilder).when(badConsumerBuilder).subscriptionType(any());
+        doReturn(badConsumerBuilder).when(badConsumerBuilder).ackTimeout(anyLong(), any());
+        doReturn(badConsumerBuilder).when(badConsumerBuilder).messageListener(any());
+        doReturn(badConsumerBuilder).when(badConsumerBuilder).properties(any());
+
         Consumer<?> consumer = mock(Consumer.class);
-        doReturn(consumer).when(consumerBuilder).subscribe();
-        doReturn(consumerBuilder).when(pulsarClient).newConsumer(any());
-        doReturn(CompletableFuture.completedFuture(consumer)).when(consumerBuilder).subscribeAsync();
+        doReturn(consumer).when(goodConsumerBuilder).subscribe();
+        doReturn(goodConsumerBuilder).when(pulsarClient).newConsumer(any());
+        doReturn(CompletableFuture.completedFuture(consumer)).when(goodConsumerBuilder).subscribeAsync();
+        CompletableFuture<Consumer<?>> badFuture = new CompletableFuture<>();
+        badFuture.completeExceptionally(new PulsarClientException("Some Error"));
+        doReturn(badFuture).when(badConsumerBuilder).subscribeAsync();
+        doThrow(PulsarClientException.class).when(badConsumerBuilder).subscribe();
         doReturn(CompletableFuture.completedFuture(Optional.empty())).when(pulsarClient).getSchema(anyString());
         return pulsarClient;
     }
 
-    private static PulsarSourceConfig getPulsarConfigs() {
+    private static class TopicMatcher implements ArgumentMatcher<List<String>> {
+        private final String topic;
+
+        public TopicMatcher(String topic) {
+            this.topic = topic;
+        }
+
+        @Override
+        public boolean matches(List<String> arg) {
+            return arg.contains(topic);
+        }
+    }
+
+
+    private static PulsarSourceConfig getPulsarConfigs(boolean multiple) {
         PulsarSourceConfig pulsarConfig = new PulsarSourceConfig();
         pulsarConfig.setProcessingGuarantees(FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE);
-        pulsarConfig.setTopicSchema(consumerConfigs);
+        if (multiple) {
+            pulsarConfig.setTopicSchema(multipleConsumerConfigs);
+        } else {
+            pulsarConfig.setTopicSchema(consumerConfigs);
+        }
         pulsarConfig.setTypeClassName(String.class.getName());
         pulsarConfig.setSubscriptionPosition(SubscriptionInitialPosition.Latest);
         pulsarConfig.setSubscriptionType(SubscriptionType.Shared);
@@ -126,7 +172,7 @@ public class PulsarSourceTest {
 
     @Test
     public void testVoidInputClasses() throws Exception {
-        PulsarSourceConfig pulsarConfig = getPulsarConfigs();
+        PulsarSourceConfig pulsarConfig = getPulsarConfigs(false);
         // set type to void
         pulsarConfig.setTypeClassName(Void.class.getName());
 
@@ -150,11 +196,11 @@ public class PulsarSourceTest {
      */
     @Test
     public void testInconsistentInputType() throws Exception {
-        PulsarSourceConfig pulsarConfig = getPulsarConfigs();
+        PulsarSourceConfig pulsarConfig = getPulsarConfigs(false);
         // set type to be inconsistent to that of SerDe
         pulsarConfig.setTypeClassName(Integer.class.getName());
         Map<String, ConsumerConfig> topicSerdeClassNameMap = new HashMap<>();
-        topicSerdeClassNameMap.put("persistent://sample/standalone/ns1/test_result",
+        topicSerdeClassNameMap.put("persistent://sample/ns1/test_result",
                 ConsumerConfig.builder().serdeClassName(TestSerDe.class.getName()).build());
         pulsarConfig.setTopicSchema(topicSerdeClassNameMap);
 
@@ -178,10 +224,10 @@ public class PulsarSourceTest {
     @Test
     public void testDefaultSerDe() throws Exception {
 
-        PulsarSourceConfig pulsarConfig = getPulsarConfigs();
+        PulsarSourceConfig pulsarConfig = getPulsarConfigs(false);
         // set type to void
         pulsarConfig.setTypeClassName(String.class.getName());
-        consumerConfigs.put("persistent://sample/standalone/ns1/test_result",
+        consumerConfigs.put("persistent://sample/ns1/test_result",
                 ConsumerConfig.builder().serdeClassName(TopicSchema.DEFAULT_SERDE).build());
         pulsarConfig.setTopicSchema(consumerConfigs);
 
@@ -193,10 +239,10 @@ public class PulsarSourceTest {
 
     @Test
     public void testComplexOuputType() throws Exception {
-        PulsarSourceConfig pulsarConfig = getPulsarConfigs();
+        PulsarSourceConfig pulsarConfig = getPulsarConfigs(false);
         // set type to void
         pulsarConfig.setTypeClassName(ComplexUserDefinedType.class.getName());
-        consumerConfigs.put("persistent://sample/standalone/ns1/test_result",
+        consumerConfigs.put("persistent://sample/ns1/test_result",
                 ConsumerConfig.builder().serdeClassName(ComplexSerDe.class.getName()).build());
         pulsarConfig.setTopicSchema(consumerConfigs);
 
@@ -205,4 +251,21 @@ public class PulsarSourceTest {
 
         pulsarSource.setupConsumerConfigs();
     }
+
+    @Test
+    public void testDanglingSubscriptions() throws Exception {
+        PulsarSourceConfig pulsarConfig = getPulsarConfigs(true);
+
+        PulsarSource<?> pulsarSource = new PulsarSource<>(getPulsarClient(), pulsarConfig, new HashMap<>(), Thread.currentThread().getContextClassLoader());
+        try {
+            pulsarSource.open(new HashMap<>(), mock(SourceContext.class));
+            fail();
+        } catch (CompletionException e) {
+            pulsarSource.close();
+            assertEquals(pulsarSource.getInputConsumers().size(), 1);
+        } catch (Exception e) {
+            fail();
+        }
+
+    }
 }


[pulsar] 01/38: [hotfix]Python function protobuf missing field and broker test failed (#6641)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit d39113abe50c1de540f0f1d52a78007b5459afe3
Author: guangning <gu...@apache.org>
AuthorDate: Tue Mar 31 20:58:29 2020 +0800

    [hotfix]Python function protobuf missing field and broker test failed (#6641)
    
    At present, in the test, we found that due to the addition of a field `forwardSourceMessageProperty` in the proto file of function, this field was lost in the proto file generated by python and go. Due to python parsing with the following code:
    
    ```
    json_format.Parse(args.function_details, function_details)
    ```
    
    the following exception will be thrown.
    
    ```
    2020-03-30T13:13:25.2339031Z 13:13:24.379 [pulsar-external-listener-20-1] INFO  org.apache.pulsar.functions.runtime.process.ProcessRuntime - Started process successfully
    2020-03-30T13:13:25.2339190Z Traceback (most recent call last):
    2020-03-30T13:13:25.2340782Z   File "/pulsar/instances/python-instance/python_instance_main.py", line 211, in <module>
    2020-03-30T13:13:25.2340944Z     main()
    2020-03-30T13:13:25.2342589Z   File "/pulsar/instances/python-instance/python_instance_main.py", line 98, in main
    2020-03-30T13:13:25.2342744Z     json_format.Parse(args.function_details, function_details)
    2020-03-30T13:13:25.2354119Z   File "/usr/local/lib/python2.7/dist-packages/google/protobuf/json_format.py", line 430, in Parse
    2020-03-30T13:13:25.2354284Z     return ParseDict(js, message, ignore_unknown_fields, descriptor_pool)
    2020-03-30T13:13:25.2354689Z   File "/usr/local/lib/python2.7/dist-packages/google/protobuf/json_format.py", line 450, in ParseDict
    2020-03-30T13:13:25.2354882Z     parser.ConvertMessage(js_dict, message)
    2020-03-30T13:13:25.2355386Z   File "/usr/local/lib/python2.7/dist-packages/google/protobuf/json_format.py", line 481, in ConvertMessage
    2020-03-30T13:13:25.2355537Z     self._ConvertFieldValuePair(value, message)
    2020-03-30T13:13:25.2356082Z   File "/usr/local/lib/python2.7/dist-packages/google/protobuf/json_format.py", line 590, in _ConvertFieldValuePair
    2020-03-30T13:13:25.2356559Z     raise ParseError('Failed to parse {0} field: {1}.'.format(name, e))
    2020-03-30T13:13:25.2357199Z google.protobuf.json_format.ParseError: Failed to parse sink field: Message type "proto.SinkSpec" has no field named "forwardSourceMessageProperty".
    2020-03-30T13:13:25.2357634Z  Available Fields(except extensions): ['className', 'configs', 'typeClassName', 'topic', 'serDeClassName', 'builtin', 'schemaType'].
    ```
    
    This pr is mainly to fix the proto file generated by python first. In order to make the test pass smoothly, I will gradually fix this problem in other languages in the next pull request.
    
    * Enable build docker image to pulsar and pulsar-all.
    * Add new generated protobuf file for python
    * Disable go function integration test
    * Add sleep for method testGetPartitionedStatsInternal
    
    The integration process test passed https://github.com/AmateurEvents/pulsar/pull/22
    (cherry picked from commit c955ff9ebf363175991704d5dd17da7bdc5e7468)
---
 .github/workflows/ci-integration-process.yaml      |   8 +
 .github/workflows/ci-integration-thread.yaml       |   8 +
 .../apache/pulsar/broker/admin/AdminApiTest.java   |   2 +
 .../instance/src/main/python/Function_pb2.py       | 320 ++++++++++++---------
 4 files changed, 201 insertions(+), 137 deletions(-)

diff --git a/.github/workflows/ci-integration-process.yaml b/.github/workflows/ci-integration-process.yaml
index 74f6960..cf762da 100644
--- a/.github/workflows/ci-integration-process.yaml
+++ b/.github/workflows/ci-integration-process.yaml
@@ -63,6 +63,14 @@ jobs:
         run: mvn -B -f docker/pulsar-all/pom.xml install -am -Pdocker -DskipTests -Ddocker.nocache=true
 
       - name: build artifacts and docker image
+        if: steps.docs.outputs.changed_only == 'no'
+        run: mvn -B -f docker/pulsar/pom.xml install -am -Pdocker -DskipTests -Ddocker.nocache=true
+
+      - name: build pulsar-all image
+        if: steps.docs.outputs.changed_only == 'no'
+        run: mvn -B -f docker/pulsar-all/pom.xml install -am -Pdocker -DskipTests -Ddocker.nocache=true
+
+      - name: build artifacts and docker image
         run: mvn -B install -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR -Pdocker -DskipTests
 
       - name: run integration tests
diff --git a/.github/workflows/ci-integration-thread.yaml b/.github/workflows/ci-integration-thread.yaml
index 79a702a..687b1b2 100644
--- a/.github/workflows/ci-integration-thread.yaml
+++ b/.github/workflows/ci-integration-thread.yaml
@@ -63,6 +63,14 @@ jobs:
         run: mvn -B -f docker/pulsar-all/pom.xml install -am -Pdocker -DskipTests -Ddocker.nocache=true
 
       - name: build artifacts and docker image
+        if: steps.docs.outputs.changed_only == 'no'
+        run: mvn -B -f docker/pulsar/pom.xml install -am -Pdocker -DskipTests -Ddocker.nocache=true
+
+      - name: build pulsar-all image
+        if: steps.docs.outputs.changed_only == 'no'
+        run: mvn -B -f docker/pulsar-all/pom.xml install -am -Pdocker -DskipTests -Ddocker.nocache=true
+
+      - name: build artifacts and docker image
         run: mvn -B install -Dorg.slf4j.simpleLogger.defaultLogLevel=ERROR -Pdocker -DskipTests
 
       - name: run integration tests
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
index 1598be7..ae98235 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
@@ -48,11 +48,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
+import javax.validation.constraints.AssertTrue;
 import javax.ws.rs.client.InvocationCallback;
 import javax.ws.rs.client.WebTarget;
 import javax.ws.rs.core.Response.Status;
diff --git a/pulsar-functions/instance/src/main/python/Function_pb2.py b/pulsar-functions/instance/src/main/python/Function_pb2.py
index 409f749..108ed79 100644
--- a/pulsar-functions/instance/src/main/python/Function_pb2.py
+++ b/pulsar-functions/instance/src/main/python/Function_pb2.py
@@ -21,8 +21,6 @@
 # Generated by the protocol buffer compiler.  DO NOT EDIT!
 # source: Function.proto
 
-import sys
-_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1'))
 from google.protobuf.internal import enum_type_wrapper
 from google.protobuf import descriptor as _descriptor
 from google.protobuf import message as _message
@@ -39,8 +37,8 @@ DESCRIPTOR = _descriptor.FileDescriptor(
   name='Function.proto',
   package='proto',
   syntax='proto3',
-  serialized_options=_b('\n!org.apache.pulsar.functions.protoB\010Function'),
-  serialized_pb=_b('\n\x0e\x46unction.proto\x12\x05proto\"3\n\tResources\x12\x0b\n\x03\x63pu\x18\x01 \x01(\x01\x12\x0b\n\x03ram\x18\x02 \x01(\x03\x12\x0c\n\x04\x64isk\x18\x03 \x01(\x03\"B\n\x0cRetryDetails\x12\x19\n\x11maxMessageRetries\x18\x01 \x01(\x05\x12\x17\n\x0f\x64\x65\x61\x64LetterTopic\x18\x02 \x01(\t\"\x85\x05\n\x0f\x46unctionDetails\x12\x0e\n\x06tenant\x18\x01 \x01(\t\x12\x11\n\tnamespace\x18\x02 \x01(\t\x12\x0c\n\x04name\x18\x03 \x01(\t\x12\x11\n\tclassName\x18\x04 \x01(\t\x1 [...]
+  serialized_options=b'\n!org.apache.pulsar.functions.protoB\010Function',
+  serialized_pb=b'\n\x0e\x46unction.proto\x12\x05proto\"3\n\tResources\x12\x0b\n\x03\x63pu\x18\x01 \x01(\x01\x12\x0b\n\x03ram\x18\x02 \x01(\x03\x12\x0c\n\x04\x64isk\x18\x03 \x01(\x03\"B\n\x0cRetryDetails\x12\x19\n\x11maxMessageRetries\x18\x01 \x01(\x05\x12\x17\n\x0f\x64\x65\x61\x64LetterTopic\x18\x02 \x01(\t\"\xa3\x05\n\x0f\x46unctionDetails\x12\x0e\n\x06tenant\x18\x01 \x01(\t\x12\x11\n\tnamespace\x18\x02 \x01(\t\x12\x0c\n\x04name\x18\x03 \x01(\t\x12\x11\n\tclassName\x18\x04 \x01(\t\x12\ [...]
 )
 
 _PROCESSINGGUARANTEES = _descriptor.EnumDescriptor(
@@ -64,8 +62,8 @@ _PROCESSINGGUARANTEES = _descriptor.EnumDescriptor(
   ],
   containing_type=None,
   serialized_options=None,
-  serialized_start=2302,
-  serialized_end=2381,
+  serialized_start=2429,
+  serialized_end=2508,
 )
 _sym_db.RegisterEnumDescriptor(_PROCESSINGGUARANTEES)
 
@@ -87,12 +85,35 @@ _SUBSCRIPTIONTYPE = _descriptor.EnumDescriptor(
   ],
   containing_type=None,
   serialized_options=None,
-  serialized_start=2383,
-  serialized_end=2427,
+  serialized_start=2510,
+  serialized_end=2554,
 )
 _sym_db.RegisterEnumDescriptor(_SUBSCRIPTIONTYPE)
 
 SubscriptionType = enum_type_wrapper.EnumTypeWrapper(_SUBSCRIPTIONTYPE)
+_SUBSCRIPTIONPOSITION = _descriptor.EnumDescriptor(
+  name='SubscriptionPosition',
+  full_name='proto.SubscriptionPosition',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='LATEST', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='EARLIEST', index=1, number=1,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=2556,
+  serialized_end=2604,
+)
+_sym_db.RegisterEnumDescriptor(_SUBSCRIPTIONPOSITION)
+
+SubscriptionPosition = enum_type_wrapper.EnumTypeWrapper(_SUBSCRIPTIONPOSITION)
 _FUNCTIONSTATE = _descriptor.EnumDescriptor(
   name='FunctionState',
   full_name='proto.FunctionState',
@@ -110,8 +131,8 @@ _FUNCTIONSTATE = _descriptor.EnumDescriptor(
   ],
   containing_type=None,
   serialized_options=None,
-  serialized_start=2429,
-  serialized_end=2470,
+  serialized_start=2606,
+  serialized_end=2647,
 )
 _sym_db.RegisterEnumDescriptor(_FUNCTIONSTATE)
 
@@ -121,6 +142,8 @@ ATMOST_ONCE = 1
 EFFECTIVELY_ONCE = 2
 SHARED = 0
 FAILOVER = 1
+LATEST = 0
+EARLIEST = 1
 RUNNING = 0
 STOPPED = 1
 
@@ -146,8 +169,8 @@ _FUNCTIONDETAILS_RUNTIME = _descriptor.EnumDescriptor(
   ],
   containing_type=None,
   serialized_options=None,
-  serialized_start=687,
-  serialized_end=726,
+  serialized_start=717,
+  serialized_end=756,
 )
 _sym_db.RegisterEnumDescriptor(_FUNCTIONDETAILS_RUNTIME)
 
@@ -176,8 +199,8 @@ _FUNCTIONDETAILS_COMPONENTTYPE = _descriptor.EnumDescriptor(
   ],
   containing_type=None,
   serialized_options=None,
-  serialized_start=728,
-  serialized_end=792,
+  serialized_start=758,
+  serialized_end=822,
 )
 _sym_db.RegisterEnumDescriptor(_FUNCTIONDETAILS_COMPONENTTYPE)
 
@@ -244,7 +267,7 @@ _RETRYDETAILS = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='deadLetterTopic', full_name='proto.RetryDetails.deadLetterTopic', index=1,
       number=2, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -275,35 +298,35 @@ _FUNCTIONDETAILS = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='tenant', full_name='proto.FunctionDetails.tenant', index=0,
       number=1, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='namespace', full_name='proto.FunctionDetails.namespace', index=1,
       number=2, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='name', full_name='proto.FunctionDetails.name', index=2,
       number=3, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='className', full_name='proto.FunctionDetails.className', index=3,
       number=4, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='logTopic', full_name='proto.FunctionDetails.logTopic', index=4,
       number=5, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -317,14 +340,14 @@ _FUNCTIONDETAILS = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='userConfig', full_name='proto.FunctionDetails.userConfig', index=6,
       number=7, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='secretsMap', full_name='proto.FunctionDetails.secretsMap', index=7,
       number=16, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -373,7 +396,7 @@ _FUNCTIONDETAILS = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='packageUrl', full_name='proto.FunctionDetails.packageUrl', index=14,
       number=14, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -387,7 +410,7 @@ _FUNCTIONDETAILS = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='runtimeFlags', full_name='proto.FunctionDetails.runtimeFlags', index=16,
       number=17, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -398,6 +421,13 @@ _FUNCTIONDETAILS = _descriptor.Descriptor(
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='customRuntimeOptions', full_name='proto.FunctionDetails.customRuntimeOptions', index=18,
+      number=19, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=b"".decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
   ],
   extensions=[
   ],
@@ -413,7 +443,7 @@ _FUNCTIONDETAILS = _descriptor.Descriptor(
   oneofs=[
   ],
   serialized_start=147,
-  serialized_end=792,
+  serialized_end=822,
 )
 
 
@@ -443,8 +473,8 @@ _CONSUMERSPEC_RECEIVERQUEUESIZE = _descriptor.Descriptor(
   extension_ranges=[],
   oneofs=[
   ],
-  serialized_start=945,
-  serialized_end=979,
+  serialized_start=975,
+  serialized_end=1009,
 )
 
 _CONSUMERSPEC = _descriptor.Descriptor(
@@ -457,14 +487,14 @@ _CONSUMERSPEC = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='schemaType', full_name='proto.ConsumerSpec.schemaType', index=0,
       number=1, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='serdeClassName', full_name='proto.ConsumerSpec.serdeClassName', index=1,
       number=2, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -494,8 +524,8 @@ _CONSUMERSPEC = _descriptor.Descriptor(
   extension_ranges=[],
   oneofs=[
   ],
-  serialized_start=795,
-  serialized_end=979,
+  serialized_start=825,
+  serialized_end=1009,
 )
 
 
@@ -509,14 +539,14 @@ _SOURCESPEC_TOPICSTOSERDECLASSNAMEENTRY = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='key', full_name='proto.SourceSpec.TopicsToSerDeClassNameEntry.key', index=0,
       number=1, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='value', full_name='proto.SourceSpec.TopicsToSerDeClassNameEntry.value', index=1,
       number=2, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -526,14 +556,14 @@ _SOURCESPEC_TOPICSTOSERDECLASSNAMEENTRY = _descriptor.Descriptor(
   nested_types=[],
   enum_types=[
   ],
-  serialized_options=_b('8\001'),
+  serialized_options=b'8\001',
   is_extendable=False,
   syntax='proto3',
   extension_ranges=[],
   oneofs=[
   ],
-  serialized_start=1362,
-  serialized_end=1423,
+  serialized_start=1451,
+  serialized_end=1512,
 )
 
 _SOURCESPEC_INPUTSPECSENTRY = _descriptor.Descriptor(
@@ -546,7 +576,7 @@ _SOURCESPEC_INPUTSPECSENTRY = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='key', full_name='proto.SourceSpec.InputSpecsEntry.key', index=0,
       number=1, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -563,14 +593,14 @@ _SOURCESPEC_INPUTSPECSENTRY = _descriptor.Descriptor(
   nested_types=[],
   enum_types=[
   ],
-  serialized_options=_b('8\001'),
+  serialized_options=b'8\001',
   is_extendable=False,
   syntax='proto3',
   extension_ranges=[],
   oneofs=[
   ],
-  serialized_start=1425,
-  serialized_end=1495,
+  serialized_start=1514,
+  serialized_end=1584,
 )
 
 _SOURCESPEC = _descriptor.Descriptor(
@@ -583,21 +613,21 @@ _SOURCESPEC = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='className', full_name='proto.SourceSpec.className', index=0,
       number=1, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='configs', full_name='proto.SourceSpec.configs', index=1,
       number=2, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='typeClassName', full_name='proto.SourceSpec.typeClassName', index=2,
       number=5, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -614,7 +644,7 @@ _SOURCESPEC = _descriptor.Descriptor(
       has_default_value=False, default_value=[],
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
-      serialized_options=_b('\030\001'), file=DESCRIPTOR),
+      serialized_options=b'\030\001', file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='inputSpecs', full_name='proto.SourceSpec.inputSpecs', index=5,
       number=10, type=11, cpp_type=10, label=3,
@@ -632,21 +662,21 @@ _SOURCESPEC = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='topicsPattern', full_name='proto.SourceSpec.topicsPattern', index=7,
       number=7, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
-      serialized_options=_b('\030\001'), file=DESCRIPTOR),
+      serialized_options=b'\030\001', file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='builtin', full_name='proto.SourceSpec.builtin', index=8,
       number=8, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='subscriptionName', full_name='proto.SourceSpec.subscriptionName', index=9,
       number=9, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -657,6 +687,13 @@ _SOURCESPEC = _descriptor.Descriptor(
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='subscriptionPosition', full_name='proto.SourceSpec.subscriptionPosition', index=11,
+      number=12, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
   ],
   extensions=[
   ],
@@ -669,8 +706,8 @@ _SOURCESPEC = _descriptor.Descriptor(
   extension_ranges=[],
   oneofs=[
   ],
-  serialized_start=982,
-  serialized_end=1495,
+  serialized_start=1012,
+  serialized_end=1584,
 )
 
 
@@ -684,49 +721,56 @@ _SINKSPEC = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='className', full_name='proto.SinkSpec.className', index=0,
       number=1, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='configs', full_name='proto.SinkSpec.configs', index=1,
       number=2, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='typeClassName', full_name='proto.SinkSpec.typeClassName', index=2,
       number=5, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='topic', full_name='proto.SinkSpec.topic', index=3,
       number=3, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='serDeClassName', full_name='proto.SinkSpec.serDeClassName', index=4,
       number=4, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='builtin', full_name='proto.SinkSpec.builtin', index=5,
       number=6, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='schemaType', full_name='proto.SinkSpec.schemaType', index=6,
       number=7, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='forwardSourceMessageProperty', full_name='proto.SinkSpec.forwardSourceMessageProperty', index=7,
+      number=8, type=8, cpp_type=7, label=1,
+      has_default_value=False, default_value=False,
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -742,8 +786,8 @@ _SINKSPEC = _descriptor.Descriptor(
   extension_ranges=[],
   oneofs=[
   ],
-  serialized_start=1498,
-  serialized_end=1643,
+  serialized_start=1587,
+  serialized_end=1770,
 )
 
 
@@ -757,14 +801,14 @@ _PACKAGELOCATIONMETADATA = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='packagePath', full_name='proto.PackageLocationMetaData.packagePath', index=0,
       number=1, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='originalFileName', full_name='proto.PackageLocationMetaData.originalFileName', index=1,
       number=2, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -780,8 +824,8 @@ _PACKAGELOCATIONMETADATA = _descriptor.Descriptor(
   extension_ranges=[],
   oneofs=[
   ],
-  serialized_start=1645,
-  serialized_end=1717,
+  serialized_start=1772,
+  serialized_end=1844,
 )
 
 
@@ -812,14 +856,14 @@ _FUNCTIONMETADATA_INSTANCESTATESENTRY = _descriptor.Descriptor(
   nested_types=[],
   enum_types=[
   ],
-  serialized_options=_b('8\001'),
+  serialized_options=b'8\001',
   is_extendable=False,
   syntax='proto3',
   extension_ranges=[],
   oneofs=[
   ],
-  serialized_start=2013,
-  serialized_end=2088,
+  serialized_start=2140,
+  serialized_end=2215,
 )
 
 _FUNCTIONMETADATA = _descriptor.Descriptor(
@@ -883,8 +927,8 @@ _FUNCTIONMETADATA = _descriptor.Descriptor(
   extension_ranges=[],
   oneofs=[
   ],
-  serialized_start=1720,
-  serialized_end=2088,
+  serialized_start=1847,
+  serialized_end=2215,
 )
 
 
@@ -898,14 +942,14 @@ _FUNCTIONAUTHENTICATIONSPEC = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='data', full_name='proto.FunctionAuthenticationSpec.data', index=0,
       number=1, type=12, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b(""),
+      has_default_value=False, default_value=b"",
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
     _descriptor.FieldDescriptor(
       name='provider', full_name='proto.FunctionAuthenticationSpec.provider', index=1,
       number=2, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -921,8 +965,8 @@ _FUNCTIONAUTHENTICATIONSPEC = _descriptor.Descriptor(
   extension_ranges=[],
   oneofs=[
   ],
-  serialized_start=2090,
-  serialized_end=2150,
+  serialized_start=2217,
+  serialized_end=2277,
 )
 
 
@@ -959,8 +1003,8 @@ _INSTANCE = _descriptor.Descriptor(
   extension_ranges=[],
   oneofs=[
   ],
-  serialized_start=2152,
-  serialized_end=2233,
+  serialized_start=2279,
+  serialized_end=2360,
 )
 
 
@@ -981,7 +1025,7 @@ _ASSIGNMENT = _descriptor.Descriptor(
     _descriptor.FieldDescriptor(
       name='workerId', full_name='proto.Assignment.workerId', index=1,
       number=2, type=9, cpp_type=9, label=1,
-      has_default_value=False, default_value=_b("").decode('utf-8'),
+      has_default_value=False, default_value=b"".decode('utf-8'),
       message_type=None, enum_type=None, containing_type=None,
       is_extension=False, extension_scope=None,
       serialized_options=None, file=DESCRIPTOR),
@@ -997,8 +1041,8 @@ _ASSIGNMENT = _descriptor.Descriptor(
   extension_ranges=[],
   oneofs=[
   ],
-  serialized_start=2235,
-  serialized_end=2300,
+  serialized_start=2362,
+  serialized_end=2427,
 )
 
 _FUNCTIONDETAILS.fields_by_name['processingGuarantees'].enum_type = _PROCESSINGGUARANTEES
@@ -1018,6 +1062,7 @@ _SOURCESPEC_INPUTSPECSENTRY.containing_type = _SOURCESPEC
 _SOURCESPEC.fields_by_name['subscriptionType'].enum_type = _SUBSCRIPTIONTYPE
 _SOURCESPEC.fields_by_name['topicsToSerDeClassName'].message_type = _SOURCESPEC_TOPICSTOSERDECLASSNAMEENTRY
 _SOURCESPEC.fields_by_name['inputSpecs'].message_type = _SOURCESPEC_INPUTSPECSENTRY
+_SOURCESPEC.fields_by_name['subscriptionPosition'].enum_type = _SUBSCRIPTIONPOSITION
 _FUNCTIONMETADATA_INSTANCESTATESENTRY.fields_by_name['value'].enum_type = _FUNCTIONSTATE
 _FUNCTIONMETADATA_INSTANCESTATESENTRY.containing_type = _FUNCTIONMETADATA
 _FUNCTIONMETADATA.fields_by_name['functionDetails'].message_type = _FUNCTIONDETAILS
@@ -1039,116 +1084,117 @@ DESCRIPTOR.message_types_by_name['Instance'] = _INSTANCE
 DESCRIPTOR.message_types_by_name['Assignment'] = _ASSIGNMENT
 DESCRIPTOR.enum_types_by_name['ProcessingGuarantees'] = _PROCESSINGGUARANTEES
 DESCRIPTOR.enum_types_by_name['SubscriptionType'] = _SUBSCRIPTIONTYPE
+DESCRIPTOR.enum_types_by_name['SubscriptionPosition'] = _SUBSCRIPTIONPOSITION
 DESCRIPTOR.enum_types_by_name['FunctionState'] = _FUNCTIONSTATE
 _sym_db.RegisterFileDescriptor(DESCRIPTOR)
 
-Resources = _reflection.GeneratedProtocolMessageType('Resources', (_message.Message,), dict(
-  DESCRIPTOR = _RESOURCES,
-  __module__ = 'Function_pb2'
+Resources = _reflection.GeneratedProtocolMessageType('Resources', (_message.Message,), {
+  'DESCRIPTOR' : _RESOURCES,
+  '__module__' : 'Function_pb2'
   # @@protoc_insertion_point(class_scope:proto.Resources)
-  ))
+  })
 _sym_db.RegisterMessage(Resources)
 
-RetryDetails = _reflection.GeneratedProtocolMessageType('RetryDetails', (_message.Message,), dict(
-  DESCRIPTOR = _RETRYDETAILS,
-  __module__ = 'Function_pb2'
+RetryDetails = _reflection.GeneratedProtocolMessageType('RetryDetails', (_message.Message,), {
+  'DESCRIPTOR' : _RETRYDETAILS,
+  '__module__' : 'Function_pb2'
   # @@protoc_insertion_point(class_scope:proto.RetryDetails)
-  ))
+  })
 _sym_db.RegisterMessage(RetryDetails)
 
-FunctionDetails = _reflection.GeneratedProtocolMessageType('FunctionDetails', (_message.Message,), dict(
-  DESCRIPTOR = _FUNCTIONDETAILS,
-  __module__ = 'Function_pb2'
+FunctionDetails = _reflection.GeneratedProtocolMessageType('FunctionDetails', (_message.Message,), {
+  'DESCRIPTOR' : _FUNCTIONDETAILS,
+  '__module__' : 'Function_pb2'
   # @@protoc_insertion_point(class_scope:proto.FunctionDetails)
-  ))
+  })
 _sym_db.RegisterMessage(FunctionDetails)
 
-ConsumerSpec = _reflection.GeneratedProtocolMessageType('ConsumerSpec', (_message.Message,), dict(
+ConsumerSpec = _reflection.GeneratedProtocolMessageType('ConsumerSpec', (_message.Message,), {
 
-  ReceiverQueueSize = _reflection.GeneratedProtocolMessageType('ReceiverQueueSize', (_message.Message,), dict(
-    DESCRIPTOR = _CONSUMERSPEC_RECEIVERQUEUESIZE,
-    __module__ = 'Function_pb2'
+  'ReceiverQueueSize' : _reflection.GeneratedProtocolMessageType('ReceiverQueueSize', (_message.Message,), {
+    'DESCRIPTOR' : _CONSUMERSPEC_RECEIVERQUEUESIZE,
+    '__module__' : 'Function_pb2'
     # @@protoc_insertion_point(class_scope:proto.ConsumerSpec.ReceiverQueueSize)
-    ))
+    })
   ,
-  DESCRIPTOR = _CONSUMERSPEC,
-  __module__ = 'Function_pb2'
+  'DESCRIPTOR' : _CONSUMERSPEC,
+  '__module__' : 'Function_pb2'
   # @@protoc_insertion_point(class_scope:proto.ConsumerSpec)
-  ))
+  })
 _sym_db.RegisterMessage(ConsumerSpec)
 _sym_db.RegisterMessage(ConsumerSpec.ReceiverQueueSize)
 
-SourceSpec = _reflection.GeneratedProtocolMessageType('SourceSpec', (_message.Message,), dict(
+SourceSpec = _reflection.GeneratedProtocolMessageType('SourceSpec', (_message.Message,), {
 
-  TopicsToSerDeClassNameEntry = _reflection.GeneratedProtocolMessageType('TopicsToSerDeClassNameEntry', (_message.Message,), dict(
-    DESCRIPTOR = _SOURCESPEC_TOPICSTOSERDECLASSNAMEENTRY,
-    __module__ = 'Function_pb2'
+  'TopicsToSerDeClassNameEntry' : _reflection.GeneratedProtocolMessageType('TopicsToSerDeClassNameEntry', (_message.Message,), {
+    'DESCRIPTOR' : _SOURCESPEC_TOPICSTOSERDECLASSNAMEENTRY,
+    '__module__' : 'Function_pb2'
     # @@protoc_insertion_point(class_scope:proto.SourceSpec.TopicsToSerDeClassNameEntry)
-    ))
+    })
   ,
 
-  InputSpecsEntry = _reflection.GeneratedProtocolMessageType('InputSpecsEntry', (_message.Message,), dict(
-    DESCRIPTOR = _SOURCESPEC_INPUTSPECSENTRY,
-    __module__ = 'Function_pb2'
+  'InputSpecsEntry' : _reflection.GeneratedProtocolMessageType('InputSpecsEntry', (_message.Message,), {
+    'DESCRIPTOR' : _SOURCESPEC_INPUTSPECSENTRY,
+    '__module__' : 'Function_pb2'
     # @@protoc_insertion_point(class_scope:proto.SourceSpec.InputSpecsEntry)
-    ))
+    })
   ,
-  DESCRIPTOR = _SOURCESPEC,
-  __module__ = 'Function_pb2'
+  'DESCRIPTOR' : _SOURCESPEC,
+  '__module__' : 'Function_pb2'
   # @@protoc_insertion_point(class_scope:proto.SourceSpec)
-  ))
+  })
 _sym_db.RegisterMessage(SourceSpec)
 _sym_db.RegisterMessage(SourceSpec.TopicsToSerDeClassNameEntry)
 _sym_db.RegisterMessage(SourceSpec.InputSpecsEntry)
 
-SinkSpec = _reflection.GeneratedProtocolMessageType('SinkSpec', (_message.Message,), dict(
-  DESCRIPTOR = _SINKSPEC,
-  __module__ = 'Function_pb2'
+SinkSpec = _reflection.GeneratedProtocolMessageType('SinkSpec', (_message.Message,), {
+  'DESCRIPTOR' : _SINKSPEC,
+  '__module__' : 'Function_pb2'
   # @@protoc_insertion_point(class_scope:proto.SinkSpec)
-  ))
+  })
 _sym_db.RegisterMessage(SinkSpec)
 
-PackageLocationMetaData = _reflection.GeneratedProtocolMessageType('PackageLocationMetaData', (_message.Message,), dict(
-  DESCRIPTOR = _PACKAGELOCATIONMETADATA,
-  __module__ = 'Function_pb2'
+PackageLocationMetaData = _reflection.GeneratedProtocolMessageType('PackageLocationMetaData', (_message.Message,), {
+  'DESCRIPTOR' : _PACKAGELOCATIONMETADATA,
+  '__module__' : 'Function_pb2'
   # @@protoc_insertion_point(class_scope:proto.PackageLocationMetaData)
-  ))
+  })
 _sym_db.RegisterMessage(PackageLocationMetaData)
 
-FunctionMetaData = _reflection.GeneratedProtocolMessageType('FunctionMetaData', (_message.Message,), dict(
+FunctionMetaData = _reflection.GeneratedProtocolMessageType('FunctionMetaData', (_message.Message,), {
 
-  InstanceStatesEntry = _reflection.GeneratedProtocolMessageType('InstanceStatesEntry', (_message.Message,), dict(
-    DESCRIPTOR = _FUNCTIONMETADATA_INSTANCESTATESENTRY,
-    __module__ = 'Function_pb2'
+  'InstanceStatesEntry' : _reflection.GeneratedProtocolMessageType('InstanceStatesEntry', (_message.Message,), {
+    'DESCRIPTOR' : _FUNCTIONMETADATA_INSTANCESTATESENTRY,
+    '__module__' : 'Function_pb2'
     # @@protoc_insertion_point(class_scope:proto.FunctionMetaData.InstanceStatesEntry)
-    ))
+    })
   ,
-  DESCRIPTOR = _FUNCTIONMETADATA,
-  __module__ = 'Function_pb2'
+  'DESCRIPTOR' : _FUNCTIONMETADATA,
+  '__module__' : 'Function_pb2'
   # @@protoc_insertion_point(class_scope:proto.FunctionMetaData)
-  ))
+  })
 _sym_db.RegisterMessage(FunctionMetaData)
 _sym_db.RegisterMessage(FunctionMetaData.InstanceStatesEntry)
 
-FunctionAuthenticationSpec = _reflection.GeneratedProtocolMessageType('FunctionAuthenticationSpec', (_message.Message,), dict(
-  DESCRIPTOR = _FUNCTIONAUTHENTICATIONSPEC,
-  __module__ = 'Function_pb2'
+FunctionAuthenticationSpec = _reflection.GeneratedProtocolMessageType('FunctionAuthenticationSpec', (_message.Message,), {
+  'DESCRIPTOR' : _FUNCTIONAUTHENTICATIONSPEC,
+  '__module__' : 'Function_pb2'
   # @@protoc_insertion_point(class_scope:proto.FunctionAuthenticationSpec)
-  ))
+  })
 _sym_db.RegisterMessage(FunctionAuthenticationSpec)
 
-Instance = _reflection.GeneratedProtocolMessageType('Instance', (_message.Message,), dict(
-  DESCRIPTOR = _INSTANCE,
-  __module__ = 'Function_pb2'
+Instance = _reflection.GeneratedProtocolMessageType('Instance', (_message.Message,), {
+  'DESCRIPTOR' : _INSTANCE,
+  '__module__' : 'Function_pb2'
   # @@protoc_insertion_point(class_scope:proto.Instance)
-  ))
+  })
 _sym_db.RegisterMessage(Instance)
 
-Assignment = _reflection.GeneratedProtocolMessageType('Assignment', (_message.Message,), dict(
-  DESCRIPTOR = _ASSIGNMENT,
-  __module__ = 'Function_pb2'
+Assignment = _reflection.GeneratedProtocolMessageType('Assignment', (_message.Message,), {
+  'DESCRIPTOR' : _ASSIGNMENT,
+  '__module__' : 'Function_pb2'
   # @@protoc_insertion_point(class_scope:proto.Assignment)
-  ))
+  })
 _sym_db.RegisterMessage(Assignment)
 
 


[pulsar] 33/38: Fix validation of function's update (#6888)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 54329f2bdb010ed25849b076aab779db6c1ba531
Author: Sergii Zhevzhyk <vz...@users.noreply.github.com>
AuthorDate: Thu May 7 09:50:12 2020 +0200

    Fix validation of function's update (#6888)
    
    ### Motivation
    
    The validation of parameters for function's update was not properly implemented for the outputSerdeClassName parameter. It was checking the outputSchemaType field instead.
    
    ### Modifications
    
    Updated the if conditions and added tests.
    (cherry picked from commit bfec5231f3e9d65dfe5a919d38411309bb255750)
---
 .../pulsar/functions/utils/FunctionConfigUtils.java       |  2 +-
 .../pulsar/functions/utils/FunctionConfigUtilsTest.java   | 15 +++++++++++++++
 2 files changed, 16 insertions(+), 1 deletion(-)

diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java
index 3398a5a..4876a82 100644
--- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java
+++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java
@@ -727,7 +727,7 @@ public class FunctionConfigUtils {
                 mergedConfig.getInputSpecs().put(topicName, consumerConfig);
             });
         }
-        if (!StringUtils.isEmpty(newConfig.getOutputSchemaType()) && !newConfig.getOutputSchemaType().equals(existingConfig.getOutputSchemaType())) {
+        if (!StringUtils.isEmpty(newConfig.getOutputSerdeClassName()) && !newConfig.getOutputSerdeClassName().equals(existingConfig.getOutputSerdeClassName())) {
             throw new IllegalArgumentException("Output Serde mismatch");
         }
         if (!StringUtils.isEmpty(newConfig.getOutputSchemaType()) && !newConfig.getOutputSchemaType().equals(existingConfig.getOutputSchemaType())) {
diff --git a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java
index 29c5e55..e0f1cea 100644
--- a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java
+++ b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java
@@ -416,6 +416,7 @@ public class FunctionConfigUtilsTest {
         functionConfig.setInputSpecs(inputSpecs);
         functionConfig.setOutput("test-output");
         functionConfig.setOutputSerdeClassName("test-serde");
+        functionConfig.setOutputSchemaType("json");
         functionConfig.setRuntime(FunctionConfig.Runtime.JAVA);
         functionConfig.setProcessingGuarantees(FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE);
         functionConfig.setRetainOrdering(false);
@@ -503,4 +504,18 @@ public class FunctionConfigUtilsTest {
         assertEquals(functionConfig.getInputSpecs().keySet(), sourceSpec.getInputSpecsMap().keySet());
         assertEquals(functionConfig.getCleanupSubscription().booleanValue(), sourceSpec.getCleanupSubscription());
     }
+
+    @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Output Serde mismatch")
+    public void testMergeDifferentSerde() {
+        FunctionConfig functionConfig = createFunctionConfig();
+        FunctionConfig newFunctionConfig = createUpdatedFunctionConfig("outputSerdeClassName", "test-updated-serde");
+        FunctionConfigUtils.validateUpdate(functionConfig, newFunctionConfig);
+    }
+
+    @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Output Schema mismatch")
+    public void testMergeDifferentOutputSchemaTypes() {
+        FunctionConfig functionConfig = createFunctionConfig();
+        FunctionConfig newFunctionConfig = createUpdatedFunctionConfig("outputSchemaType", "avro");
+        FunctionConfigUtils.validateUpdate(functionConfig, newFunctionConfig);
+    }
 }


[pulsar] 28/38: Fix Get schema by version can get the deleted schema info #6754 (#6764)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit cadc852b36c18d6496a8c6ca82eda6b63087ef80
Author: feynmanlin <31...@qq.com>
AuthorDate: Thu Apr 30 21:27:38 2020 +0800

    Fix Get schema by version can get the deleted schema info #6754 (#6764)
    
    (cherry picked from commit 466b0b89af61a0d9c7aa7e6220c91f366f2d3831)
---
 .../service/schema/BookkeeperSchemaStorage.java    | 13 +++----
 .../service/schema/SchemaRegistryServiceImpl.java  | 37 ++++++++++++++-----
 .../broker/service/PersistentTopicE2ETest.java     | 42 ++++++++++++++++++++++
 .../broker/service/schema/SchemaServiceTest.java   |  4 +--
 4 files changed, 80 insertions(+), 16 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
index df41e71..b013c8b 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
@@ -237,10 +237,10 @@ public class BookkeeperSchemaStorage implements SchemaStorage {
 
             return findSchemaEntryByVersion(schemaLocator.getIndexList(), version)
                 .thenApply(entry ->
-                    new StoredSchema(
-                        entry.getSchemaData().toByteArray(),
-                        new LongSchemaVersion(version)
-                    )
+                        new StoredSchema(
+                            entry.getSchemaData().toByteArray(),
+                            new LongSchemaVersion(version)
+                        )
                 );
         });
     }
@@ -387,11 +387,12 @@ public class BookkeeperSchemaStorage implements SchemaStorage {
                 .setPosition(position)
                 .setHash(copyFrom(hash))
                 .build();
+
         return updateSchemaLocator(getSchemaPath(schemaId),
             SchemaStorageFormat.SchemaLocator.newBuilder()
                 .setInfo(info)
                 .addAllIndex(
-                    concat(locator.getIndexList(), newArrayList(info))
+                        concat(locator.getIndexList(), newArrayList(info))
                 ).build(), locatorEntry.zkZnodeVersion
         ).thenApply(ignore -> nextVersion);
     }
@@ -409,7 +410,7 @@ public class BookkeeperSchemaStorage implements SchemaStorage {
         SchemaStorageFormat.IndexEntry lowest = index.get(0);
         if (version < lowest.getVersion()) {
             return readSchemaEntry(lowest.getPosition())
-                .thenCompose(entry -> findSchemaEntryByVersion(entry.getIndexList(), version));
+                    .thenCompose(entry -> findSchemaEntryByVersion(entry.getIndexList(), version));
         }
 
         for (SchemaStorageFormat.IndexEntry entry : index) {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java
index 7307e4c..30d5d77 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java
@@ -44,12 +44,14 @@ import java.util.stream.Collectors;
 import javax.validation.constraints.NotNull;
 
 import org.apache.bookkeeper.common.concurrent.FutureUtils;
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException;
 import org.apache.pulsar.broker.service.schema.proto.SchemaRegistryFormat;
 import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy;
 import org.apache.pulsar.common.protocol.schema.SchemaData;
 import org.apache.pulsar.common.protocol.schema.SchemaHash;
 import org.apache.pulsar.common.protocol.schema.SchemaStorage;
+import org.apache.pulsar.common.protocol.schema.StoredSchema;
 import org.apache.pulsar.common.schema.SchemaType;
 import org.apache.pulsar.common.protocol.schema.SchemaVersion;
 import org.apache.pulsar.common.util.FutureUtil;
@@ -87,15 +89,34 @@ public class SchemaRegistryServiceImpl implements SchemaRegistryService {
     @Override
     @NotNull
     public CompletableFuture<SchemaAndMetadata> getSchema(String schemaId, SchemaVersion version) {
-        return schemaStorage.get(schemaId, version).thenCompose(stored -> {
-                if (isNull(stored)) {
-                    return completedFuture(null);
-                } else {
-                    return Functions.bytesToSchemaInfo(stored.data)
-                        .thenApply(Functions::schemaInfoToSchema)
-                        .thenApply(schema -> new SchemaAndMetadata(schemaId, schema, stored.version));
+        CompletableFuture<StoredSchema> completableFuture;
+        if (version == SchemaVersion.Latest) {
+            completableFuture = schemaStorage.get(schemaId, version);
+        } else {
+            long longVersion = ((LongSchemaVersion) version).getVersion();
+            //If the schema has been deleted, it cannot be obtained
+            completableFuture = trimDeletedSchemaAndGetList(schemaId)
+                .thenApply(metadataList -> metadataList.stream().filter(schemaAndMetadata ->
+                        ((LongSchemaVersion) schemaAndMetadata.version).getVersion() == longVersion)
+                        .collect(Collectors.toList())
+                ).thenCompose(metadataList -> {
+                        if (CollectionUtils.isNotEmpty(metadataList)) {
+                            return schemaStorage.get(schemaId, version);
+                        }
+                        return completedFuture(null);
+                    }
+                );
+        }
+
+        return completableFuture.thenCompose(stored -> {
+                    if (isNull(stored)) {
+                        return completedFuture(null);
+                    } else {
+                        return Functions.bytesToSchemaInfo(stored.data)
+                                .thenApply(Functions::schemaInfoToSchema)
+                                .thenApply(schema -> new SchemaAndMetadata(schemaId, schema, stored.version));
+                    }
                 }
-            }
         );
     }
 
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java
index 42190c7..38e85d8 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java
@@ -26,6 +26,7 @@ import static org.testng.Assert.assertTrue;
 import static org.testng.Assert.fail;
 import com.google.common.collect.Sets;
 import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -38,6 +39,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
+
+import lombok.Cleanup;
 import lombok.Data;
 import lombok.EqualsAndHashCode;
 import lombok.ToString;
@@ -65,8 +68,10 @@ import org.apache.pulsar.client.api.SubscriptionType;
 import org.apache.pulsar.client.api.TypedMessageBuilder;
 import org.apache.pulsar.client.api.schema.SchemaDefinition;
 import org.apache.pulsar.client.impl.ConsumerImpl;
+import org.apache.pulsar.client.impl.LookupService;
 import org.apache.pulsar.client.impl.MessageIdImpl;
 import org.apache.pulsar.client.impl.ProducerImpl;
+import org.apache.pulsar.client.impl.PulsarClientImpl;
 import org.apache.pulsar.client.impl.TypedMessageBuilderImpl;
 import org.apache.pulsar.client.impl.schema.JSONSchema;
 import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
@@ -682,6 +687,43 @@ public class PersistentTopicE2ETest extends BrokerTestBase {
         assertFalse(topicHasSchema(topicName));
     }
 
+    @Test
+    public void testDeleteSchema() throws Exception {
+        PulsarClientImpl httpProtocolClient = (PulsarClientImpl) PulsarClient.builder().serviceUrl(brokerUrl.toString()).build();
+        PulsarClientImpl binaryProtocolClient = (PulsarClientImpl) pulsarClient;
+        LookupService binaryLookupService = binaryProtocolClient.getLookup();
+        LookupService httpLookupService = httpProtocolClient.getLookup();
+
+        String topicName = "persistent://prop/ns-abc/topic-1";
+        //Topic is not GCed with live connection
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer().topic(topicName).create();
+
+        Optional<Topic> topic = getTopic(topicName);
+        assertTrue(topic.isPresent());
+
+        byte[] data = JSONSchema.of(SchemaDefinition.builder()
+                .withPojo(Foo.class).build()).getSchemaInfo().getSchema();
+        SchemaData schemaData = SchemaData.builder()
+                .data(data)
+                .type(SchemaType.BYTES)
+                .user("foo").build();
+
+        topic.get().addSchema(schemaData).join();
+        assertTrue(topicHasSchema(topicName));
+
+        Assert.assertEquals(admin.schemas().getAllSchemas(topicName).size(), 1);
+        assertTrue(httpLookupService.getSchema(TopicName.get(topicName), ByteBuffer.allocate(8).putLong(0).array()).get().isPresent());
+        assertTrue(binaryLookupService.getSchema(TopicName.get(topicName), ByteBuffer.allocate(8).putLong(0).array()).get().isPresent());
+
+        topic.get().deleteSchema().join();
+        Assert.assertEquals(admin.schemas().getAllSchemas(topicName).size(), 0);
+        assertFalse(httpLookupService.getSchema(TopicName.get(topicName), ByteBuffer.allocate(8).putLong(0).array()).get().isPresent());
+        assertFalse(binaryLookupService.getSchema(TopicName.get(topicName), ByteBuffer.allocate(8).putLong(0).array()).get().isPresent());
+
+        assertFalse(topicHasSchema(topicName));
+    }
+
     /**
      * A topic that has retention policy set to non-0, should not be GCed until it has been inactive for at least the
      * retention time.
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java
index ab7e910..00b09cb 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java
@@ -226,8 +226,8 @@ public class SchemaServiceTest extends MockedPulsarServiceBaseTest {
 
         deleteSchema(schemaId1, version(7));
 
-        SchemaData version7 = getSchema(schemaId1, version(7));
-        assertTrue(version7.isDeleted());
+        SchemaRegistry.SchemaAndMetadata version7 = schemaRegistryService.getSchema(schemaId1, version(7)).get();
+        assertNull(version7);
 
     }
 


[pulsar] 27/38: Make SchemaStorage accessible in Offloader (#6567)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit d8b6e3fe6075417650bd0620f655db9a8cb37a99
Author: Yijie Shen <he...@gmail.com>
AuthorDate: Sat Mar 21 01:15:00 2020 +0800

    Make SchemaStorage accessible in Offloader (#6567)
    
    While offloading ledgers from bookies to 2nd storage, we could offload the ledgers in columnar format. Columnar data could accelerate analytical workloads' execution by skipping unnecessary columns or data blocks (also known as column pruning and filter push down in analytical systems).
    
    The only blocker in Pulsar side is that offloaders cannot get the schema of the ledgers, this PR makes the schema storage accessible from offloaders.
    (cherry picked from commit 322347714301371b2e8df7b107c5fd52bb7b17fa)
---
 .../bookkeeper/mledger/LedgerOffloaderFactory.java | 18 ++++++++++++
 .../org/apache/pulsar/broker/PulsarService.java    | 23 +++++++++++++++-
 .../service/schema/BookkeeperSchemaStorage.java    |  2 ++
 .../schema/BookkeeperSchemaStorageFactory.java     |  1 +
 .../service/schema/SchemaRegistryService.java      | 32 ++++++++--------------
 .../service/schema/SchemaRegistryServiceImpl.java  |  1 +
 .../service/schema/SchemaStorageFactory.java       |  1 +
 .../org/apache/pulsar/client/impl/MessageImpl.java |  2 +-
 .../pulsar/common/api/raw/MessageParser.java       |  2 +-
 .../pulsar/common/api/raw/RawMessageIdImpl.java    |  3 ++
 .../common/protocol}/schema/SchemaStorage.java     |  6 ++--
 .../common/protocol}/schema/StoredSchema.java      | 12 ++++----
 12 files changed, 72 insertions(+), 31 deletions(-)

diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloaderFactory.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloaderFactory.java
index 7a0e6dc..bffffa5 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloaderFactory.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloaderFactory.java
@@ -25,6 +25,7 @@ import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate;
 import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving;
 import org.apache.bookkeeper.common.util.OrderedScheduler;
 import org.apache.pulsar.common.policies.data.OffloadPolicies;
+import org.apache.pulsar.common.protocol.schema.SchemaStorage;
 
 /**
  * Factory to create {@link LedgerOffloader} to offload ledgers into long-term storage.
@@ -55,4 +56,21 @@ public interface LedgerOffloaderFactory<T extends LedgerOffloader> {
              OrderedScheduler scheduler)
         throws IOException;
 
+    /**
+     * Create a ledger offloader with the provided configuration, user-metadata, schema storage and scheduler.
+     *
+     * @param offloadPolicies offload policies
+     * @param userMetadata user metadata
+     * @param schemaStorage used for schema lookup in offloader
+     * @param scheduler scheduler
+     * @return the offloader instance
+     * @throws IOException when fail to create an offloader
+     */
+    default T create(OffloadPolicies offloadPolicies,
+             Map<String, String> userMetadata,
+             SchemaStorage schemaStorage,
+             OrderedScheduler scheduler)
+            throws IOException {
+        return create(offloadPolicies, userMetadata, scheduler);
+    }
 }
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 5236efa..eeb3b1a 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
@@ -33,6 +33,7 @@ import io.netty.channel.socket.SocketChannel;
 import io.netty.util.concurrent.DefaultThreadFactory;
 
 import java.io.IOException;
+import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.Collections;
@@ -104,6 +105,7 @@ import org.apache.pulsar.common.policies.data.OffloadPolicies;
 import org.apache.pulsar.common.policies.data.Policies;
 import org.apache.pulsar.common.policies.data.RetentionPolicies;
 import org.apache.pulsar.common.policies.data.TenantInfo;
+import org.apache.pulsar.common.protocol.schema.SchemaStorage;
 import org.apache.pulsar.common.util.FutureUtil;
 import org.apache.pulsar.common.util.ObjectMapperFactory;
 import org.apache.pulsar.compaction.Compactor;
@@ -180,6 +182,7 @@ public class PulsarService implements AutoCloseable {
     private String brokerServiceUrl;
     private String brokerServiceUrlTls;
     private final String brokerVersion;
+    private SchemaStorage schemaStorage = null;
     private SchemaRegistryService schemaRegistryService = null;
     private final Optional<WorkerService> functionWorkerService;
     private ProtocolHandlers protocolHandlers = null;
@@ -403,7 +406,10 @@ public class PulsarService implements AutoCloseable {
 
             // needs load management service and before start broker service,
             this.startNamespaceService();
-            schemaRegistryService = SchemaRegistryService.create(this);
+
+            schemaStorage = createAndStartSchemaStorage();
+            schemaRegistryService = SchemaRegistryService.create(
+                    schemaStorage, config.getSchemaRegistryCompatibilityCheckers());
 
             this.defaultOffloader = createManagedLedgerOffloader(
                     OffloadPolicies.create(this.getConfiguration().getProperties()));
@@ -817,6 +823,7 @@ public class PulsarService implements AutoCloseable {
                             LedgerOffloader.METADATA_SOFTWARE_VERSION_KEY.toLowerCase(), PulsarVersion.getVersion(),
                             LedgerOffloader.METADATA_SOFTWARE_GITSHA_KEY.toLowerCase(), PulsarVersion.getGitSha()
                         ),
+                        schemaStorage,
                         getOffloaderScheduler(offloadPolicies));
                 } catch (IOException ioe) {
                     throw new PulsarServerException(ioe.getMessage(), ioe.getCause());
@@ -830,6 +837,20 @@ public class PulsarService implements AutoCloseable {
         }
     }
 
+    private SchemaStorage createAndStartSchemaStorage() {
+        SchemaStorage schemaStorage = null;
+        try {
+            final Class<?> storageClass = Class.forName(config.getSchemaRegistryStorageClassName());
+            Object factoryInstance = storageClass.newInstance();
+            Method createMethod = storageClass.getMethod("create", PulsarService.class);
+            schemaStorage = (SchemaStorage) createMethod.invoke(factoryInstance, this);
+            schemaStorage.start();
+        } catch (Exception e) {
+            LOG.warn("Unable to create schema registry storage");
+        }
+        return schemaStorage;
+    }
+
     public ZooKeeperCache getLocalZkCache() {
         return localZkCache;
     }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
index 02e48c8..df41e71 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java
@@ -50,7 +50,9 @@ import org.apache.bookkeeper.mledger.impl.LedgerMetadataUtils;
 import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.common.protocol.schema.SchemaStorage;
 import org.apache.pulsar.common.protocol.schema.SchemaVersion;
+import org.apache.pulsar.common.protocol.schema.StoredSchema;
 import org.apache.pulsar.zookeeper.ZooKeeperCache;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageFactory.java
index 4b25374..8304ed1 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageFactory.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageFactory.java
@@ -20,6 +20,7 @@ package org.apache.pulsar.broker.service.schema;
 
 import javax.validation.constraints.NotNull;
 import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.common.protocol.schema.SchemaStorage;
 
 @SuppressWarnings("unused")
 public class BookkeeperSchemaStorageFactory implements SchemaStorageFactory {
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryService.java
index 321ef7c..07d37e1 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryService.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryService.java
@@ -25,12 +25,12 @@ import java.util.Set;
 import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.ServiceConfiguration;
 import org.apache.pulsar.broker.service.schema.validator.SchemaRegistryServiceWithSchemaDataValidator;
+import org.apache.pulsar.common.protocol.schema.SchemaStorage;
 import org.apache.pulsar.common.schema.SchemaType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public interface SchemaRegistryService extends SchemaRegistry {
-    String CreateMethodName = "create";
     Logger log = LoggerFactory.getLogger(SchemaRegistryService.class);
     long NO_SCHEMA_VERSION = -1L;
 
@@ -44,26 +44,16 @@ public interface SchemaRegistryService extends SchemaRegistry {
         return checkers;
     }
 
-    static SchemaRegistryService create(PulsarService pulsar) {
-        try {
-            ServiceConfiguration config = pulsar.getConfiguration();
-            final Class<?> storageClass = Class.forName(config.getSchemaRegistryStorageClassName());
-            Object factoryInstance = storageClass.newInstance();
-            Method createMethod = storageClass.getMethod(CreateMethodName, PulsarService.class);
-
-            SchemaStorage schemaStorage = (SchemaStorage) createMethod.invoke(factoryInstance, pulsar);
-
-            Map<SchemaType, SchemaCompatibilityCheck> checkers =
-                getCheckers(config.getSchemaRegistryCompatibilityCheckers());
-
-            checkers.put(SchemaType.KEY_VALUE, new KeyValueSchemaCompatibilityCheck(checkers));
-
-            schemaStorage.start();
-
-            return SchemaRegistryServiceWithSchemaDataValidator.of(
-                new SchemaRegistryServiceImpl(schemaStorage, checkers));
-        } catch (Exception e) {
-            log.warn("Unable to create schema registry storage, defaulting to empty storage", e);
+    static SchemaRegistryService create(SchemaStorage schemaStorage, Set<String> schemaRegistryCompatibilityCheckers) {
+        if (schemaStorage != null) {
+            try {
+                Map<SchemaType, SchemaCompatibilityCheck> checkers = getCheckers(schemaRegistryCompatibilityCheckers);
+                checkers.put(SchemaType.KEY_VALUE, new KeyValueSchemaCompatibilityCheck(checkers));
+                return SchemaRegistryServiceWithSchemaDataValidator.of(
+                        new SchemaRegistryServiceImpl(schemaStorage, checkers));
+            } catch (Exception e) {
+                log.warn("Unable to create schema registry storage, defaulting to empty storage", e);
+            }
         }
         return new DefaultSchemaRegistryService();
     }
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java
index 4211882..7307e4c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java
@@ -49,6 +49,7 @@ import org.apache.pulsar.broker.service.schema.proto.SchemaRegistryFormat;
 import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy;
 import org.apache.pulsar.common.protocol.schema.SchemaData;
 import org.apache.pulsar.common.protocol.schema.SchemaHash;
+import org.apache.pulsar.common.protocol.schema.SchemaStorage;
 import org.apache.pulsar.common.schema.SchemaType;
 import org.apache.pulsar.common.protocol.schema.SchemaVersion;
 import org.apache.pulsar.common.util.FutureUtil;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaStorageFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaStorageFactory.java
index c4cff34..483e310 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaStorageFactory.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaStorageFactory.java
@@ -20,6 +20,7 @@ package org.apache.pulsar.broker.service.schema;
 
 import javax.validation.constraints.NotNull;
 import org.apache.pulsar.broker.PulsarService;
+import org.apache.pulsar.common.protocol.schema.SchemaStorage;
 
 public interface SchemaStorageFactory {
     @NotNull
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
index f11ae77..0b1f397 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
@@ -65,7 +65,7 @@ public class MessageImpl<T> implements Message<T> {
     private final int redeliveryCount;
 
     // Constructor for out-going message
-    static <T> MessageImpl<T> create(MessageMetadata.Builder msgMetadataBuilder, ByteBuffer payload, Schema<T> schema) {
+    public static <T> MessageImpl<T> create(MessageMetadata.Builder msgMetadataBuilder, ByteBuffer payload, Schema<T> schema) {
         @SuppressWarnings("unchecked")
         MessageImpl<T> msg = (MessageImpl<T>) RECYCLER.get();
         msg.msgMetadataBuilder = msgMetadataBuilder;
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/MessageParser.java b/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/MessageParser.java
index b791a80..aeb60ef 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/MessageParser.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/MessageParser.java
@@ -45,7 +45,7 @@ public class MessageParser {
      * Definition of an interface to process a raw Pulsar entry payload.
      */
     public interface MessageProcessor {
-        void process(RawMessage message);
+        void process(RawMessage message) throws IOException;
     }
 
     /**
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessageIdImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessageIdImpl.java
index d388cc7..f48cd4c 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessageIdImpl.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessageIdImpl.java
@@ -18,6 +18,9 @@
  */
 package org.apache.pulsar.common.api.raw;
 
+import lombok.Getter;
+
+@Getter
 public class RawMessageIdImpl implements RawMessageId {
 
     long ledgerId;
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaStorage.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/schema/SchemaStorage.java
similarity index 92%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaStorage.java
rename to pulsar-common/src/main/java/org/apache/pulsar/common/protocol/schema/SchemaStorage.java
index f133666..9f007aa 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaStorage.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/schema/SchemaStorage.java
@@ -16,12 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pulsar.broker.service.schema;
+package org.apache.pulsar.common.protocol.schema;
 
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
-import org.apache.pulsar.common.protocol.schema.SchemaVersion;
 
+/**
+ * Schema storage.
+ */
 public interface SchemaStorage {
 
     CompletableFuture<SchemaVersion> put(String key, byte[] value, byte[] hash);
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/StoredSchema.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/schema/StoredSchema.java
similarity index 86%
rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/StoredSchema.java
rename to pulsar-common/src/main/java/org/apache/pulsar/common/protocol/schema/StoredSchema.java
index b1866ec..7952b4e 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/StoredSchema.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/schema/StoredSchema.java
@@ -16,18 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pulsar.broker.service.schema;
+package org.apache.pulsar.common.protocol.schema;
 
 import com.google.common.base.MoreObjects;
 import java.util.Arrays;
 import java.util.Objects;
-import org.apache.pulsar.common.protocol.schema.SchemaVersion;
 
+/**
+ * Stored schema with version.
+ */
 public class StoredSchema {
     public final byte[] data;
     public final SchemaVersion version;
 
-    StoredSchema(byte[] data, SchemaVersion version) {
+    public StoredSchema(byte[] data, SchemaVersion version) {
         this.data = data;
         this.version = version;
     }
@@ -41,8 +43,8 @@ public class StoredSchema {
             return false;
         }
         StoredSchema that = (StoredSchema) o;
-        return Arrays.equals(data, that.data) &&
-            Objects.equals(version, that.version);
+        return Arrays.equals(data, that.data)
+                && Objects.equals(version, that.version);
     }
 
     @Override


[pulsar] 08/38: [issue 6694][AVRO ENCODE] Reset cursor if message encode fails. (#6695)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 14a1a2b9e8d81294625810413ff36191391103b7
Author: Shivji Kumar Jha <sh...@nutanix.com>
AuthorDate: Tue Apr 14 14:13:46 2020 +0530

    [issue 6694][AVRO ENCODE] Reset cursor if message encode fails. (#6695)
    
    Fixes #6694
    
    ### Motivation
    
    If the avro encode for message fails after writing a few bytes, the cursor in the stream is not reset. The following **flush()** that would normally reset the cursor is skipped in the event of an exception.
    
    ### Modifications
    
    Add **flush()** in the finally block.
    
    ### Test
    Added test for verifying the fix
    (cherry picked from commit 7cffe2ab7420b0001f1337ef6007448124fecf18)
---
 .../client/impl/schema/writer/AvroWriter.java      | 10 ++++++++--
 .../pulsar/client/impl/schema/AvroSchemaTest.java  | 23 ++++++++++++++++++++++
 2 files changed, 31 insertions(+), 2 deletions(-)

diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/writer/AvroWriter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/writer/AvroWriter.java
index 41ea50f..a260ebf 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/writer/AvroWriter.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/writer/AvroWriter.java
@@ -48,14 +48,20 @@ public class AvroWriter<T> implements SchemaWriter<T> {
 
     @Override
     public synchronized byte[] write(T message) {
+        byte[] outputBytes = null;
         try {
             writer.write(message, this.encoder);
-            this.encoder.flush();
-            return this.byteArrayOutputStream.toByteArray();
         } catch (Exception e) {
             throw new SchemaSerializationException(e);
         } finally {
+            try {
+                this.encoder.flush();
+                outputBytes = this.byteArrayOutputStream.toByteArray();
+            } catch (Exception ex) {
+                throw new SchemaSerializationException(ex);
+            }
             this.byteArrayOutputStream.reset();
         }
+        return outputBytes;
     }
 }
diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/AvroSchemaTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/AvroSchemaTest.java
index 49d5da9..8fd8154 100644
--- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/AvroSchemaTest.java
+++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/AvroSchemaTest.java
@@ -38,6 +38,8 @@ import lombok.Data;
 import lombok.extern.slf4j.Slf4j;
 
 import org.apache.avro.Schema;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.BufferedBinaryEncoder;
 import org.apache.pulsar.client.api.SchemaSerializationException;
 import org.apache.pulsar.client.api.schema.SchemaDefinition;
 import org.apache.avro.SchemaValidationException;
@@ -51,12 +53,14 @@ import org.apache.pulsar.client.api.schema.SchemaBuilder;
 import org.apache.pulsar.client.avro.generated.NasaMission;
 import org.apache.pulsar.client.impl.schema.SchemaTestUtils.Bar;
 import org.apache.pulsar.client.impl.schema.SchemaTestUtils.Foo;
+import org.apache.pulsar.client.impl.schema.writer.AvroWriter;
 import org.apache.pulsar.common.schema.SchemaInfo;
 import org.apache.pulsar.common.schema.SchemaType;
 import org.joda.time.DateTime;
 import org.joda.time.chrono.ISOChronology;
 import org.json.JSONException;
 import org.skyscreamer.jsonassert.JSONAssert;
+import org.powermock.reflect.Whitebox;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -366,7 +370,26 @@ public class AvroSchemaTest {
         Foo object1 = avroSchema.decode(byteBuf);
         Assert.assertTrue(bytes1.length > 0);
         assertEquals(object1, foo1);
+    }
+
+    @Test
+    public void discardBufferIfBadAvroData() {
+        AvroWriter<NasaMission> avroWriter = new AvroWriter<>(
+                ReflectData.AllowNull.get().getSchema(NasaMission.class));
+
+        NasaMission badNasaMissionData = new NasaMission();
+        badNasaMissionData.setId(1);
+        // set null in the non-null field. The java set will accept it but going ahead, the avro encode will crash.
+        badNasaMissionData.setName(null);
+
+        // Because data does not conform to schema expect a crash
+        Assert.assertThrows( SchemaSerializationException.class, () -> avroWriter.write(badNasaMissionData));
+
+        // Get the buffered data using powermock
+        BinaryEncoder encoder = Whitebox.getInternalState(avroWriter, "encoder");
 
+        // Assert that the buffer position is reset to zero
+        Assert.assertEquals(((BufferedBinaryEncoder)encoder).bytesBuffered(), 0);
     }
 
 }


[pulsar] 26/38: Add null check for offload policy(#6775)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 561328f09cfc77d2f3239218eefc630c01c69cb3
Author: luceneReader <54...@qq.com>
AuthorDate: Thu Apr 30 21:23:54 2020 +0800

    Add null check for offload policy(#6775)
    
    (cherry picked from commit 0951828d66430b587fdb9e5642a8fb24a106e17a)
---
 .../java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java     | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
index 535361e..2d1c6d1 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java
@@ -2355,8 +2355,9 @@ public abstract class NamespacesBase extends AdminResource {
             byte[] content = globalZk().getData(path, null, nodeStat);
             Policies policies = jsonMapper().readValue(content, Policies.class);
 
-            if (offloadPolicies.getManagedLedgerOffloadDeletionLagInMillis()
-                    .equals(OffloadPolicies.DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS)) {
+            if (offloadPolicies.getManagedLedgerOffloadDeletionLagInMillis() == null && OffloadPolicies.DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS == null
+                    || offloadPolicies.getManagedLedgerOffloadDeletionLagInMillis() != null
+                    && offloadPolicies.getManagedLedgerOffloadDeletionLagInMillis().equals(OffloadPolicies.DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS)) {
                 offloadPolicies.setManagedLedgerOffloadDeletionLagInMillis(policies.offload_deletion_lag_ms);
             } else {
                 policies.offload_deletion_lag_ms = offloadPolicies.getManagedLedgerOffloadDeletionLagInMillis();


[pulsar] 11/38: Bump netty version to 4.1.48.Final (#6746)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 219bec913bbac535dcf769991edd1e6dc937cf16
Author: Masahiro Sakamoto <ma...@yahoo-corp.jp>
AuthorDate: Sat Apr 18 02:31:02 2020 +0900

    Bump netty version to 4.1.48.Final (#6746)
    
    (cherry picked from commit 466ba07a04f75f532180b0ac5cf5af52b372baa7)
---
 distribution/server/src/assemble/LICENSE.bin.txt | 34 ++++++++++++------------
 pom.xml                                          |  4 +--
 pulsar-sql/presto-distribution/LICENSE           | 32 +++++++++++-----------
 3 files changed, 35 insertions(+), 35 deletions(-)

diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt
index 827864d..b578678 100644
--- a/distribution/server/src/assemble/LICENSE.bin.txt
+++ b/distribution/server/src/assemble/LICENSE.bin.txt
@@ -346,23 +346,23 @@ The Apache Software License, Version 2.0
     - org.apache.commons-commons-compress-1.19.jar
     - org.apache.commons-commons-lang3-3.4.jar
  * Netty
-    - io.netty-netty-buffer-4.1.45.Final.jar
-    - io.netty-netty-codec-4.1.45.Final.jar
-    - io.netty-netty-codec-dns-4.1.45.Final.jar
-    - io.netty-netty-codec-http-4.1.45.Final.jar
-    - io.netty-netty-codec-http2-4.1.45.Final.jar
-    - io.netty-netty-codec-socks-4.1.45.Final.jar
-    - io.netty-netty-common-4.1.45.Final.jar
-    - io.netty-netty-handler-4.1.45.Final.jar
-    - io.netty-netty-handler-proxy-4.1.45.Final.jar
-    - io.netty-netty-resolver-4.1.45.Final.jar
-    - io.netty-netty-resolver-dns-4.1.45.Final.jar
-    - io.netty-netty-transport-4.1.45.Final.jar
-    - io.netty-netty-transport-native-epoll-4.1.45.Final-linux-x86_64.jar
-    - io.netty-netty-transport-native-epoll-4.1.45.Final.jar
-    - io.netty-netty-transport-native-unix-common-4.1.45.Final.jar
-    - io.netty-netty-transport-native-unix-common-4.1.45.Final-linux-x86_64.jar
-    - io.netty-netty-tcnative-boringssl-static-2.0.26.Final.jar
+    - io.netty-netty-buffer-4.1.48.Final.jar
+    - io.netty-netty-codec-4.1.48.Final.jar
+    - io.netty-netty-codec-dns-4.1.48.Final.jar
+    - io.netty-netty-codec-http-4.1.48.Final.jar
+    - io.netty-netty-codec-http2-4.1.48.Final.jar
+    - io.netty-netty-codec-socks-4.1.48.Final.jar
+    - io.netty-netty-common-4.1.48.Final.jar
+    - io.netty-netty-handler-4.1.48.Final.jar
+    - io.netty-netty-handler-proxy-4.1.48.Final.jar
+    - io.netty-netty-resolver-4.1.48.Final.jar
+    - io.netty-netty-resolver-dns-4.1.48.Final.jar
+    - io.netty-netty-transport-4.1.48.Final.jar
+    - io.netty-netty-transport-native-epoll-4.1.48.Final-linux-x86_64.jar
+    - io.netty-netty-transport-native-epoll-4.1.48.Final.jar
+    - io.netty-netty-transport-native-unix-common-4.1.48.Final.jar
+    - io.netty-netty-transport-native-unix-common-4.1.48.Final-linux-x86_64.jar
+    - io.netty-netty-tcnative-boringssl-static-2.0.30.Final.jar
  * Prometheus client
     - io.prometheus-simpleclient-0.5.0.jar
     - io.prometheus-simpleclient_common-0.5.0.jar
diff --git a/pom.xml b/pom.xml
index e7e91ce..9d84a34 100644
--- a/pom.xml
+++ b/pom.xml
@@ -156,8 +156,8 @@ flexible messaging model and an intuitive client API.</description>
 
     <bookkeeper.version>4.10.0</bookkeeper.version>
     <zookeeper.version>3.5.7</zookeeper.version>
-    <netty.version>4.1.45.Final</netty.version>
-    <netty-tc-native.version>2.0.26.Final</netty-tc-native.version>
+    <netty.version>4.1.48.Final</netty.version>
+    <netty-tc-native.version>2.0.30.Final</netty-tc-native.version>
     <storm.version>2.0.0</storm.version>
     <jetty.version>9.4.20.v20190813</jetty.version>
     <jersey.version>2.27</jersey.version>
diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE
index f32bf5b..efa6671 100644
--- a/pulsar-sql/presto-distribution/LICENSE
+++ b/pulsar-sql/presto-distribution/LICENSE
@@ -231,23 +231,23 @@ The Apache Software License, Version 2.0
     - commons-lang3-3.4.jar
  * Netty
     - netty-3.10.6.Final.jar
-    - netty-buffer-4.1.45.Final.jar
-    - netty-codec-4.1.45.Final.jar
-    - netty-codec-dns-4.1.45.Final.jar
-    - netty-codec-http-4.1.45.Final.jar
-    - netty-codec-socks-4.1.45.Final.jar
-    - netty-common-4.1.45.Final.jar
-    - netty-handler-4.1.45.Final.jar
-    - netty-handler-proxy-4.1.45.Final.jar
+    - netty-buffer-4.1.48.Final.jar
+    - netty-codec-4.1.48.Final.jar
+    - netty-codec-dns-4.1.48.Final.jar
+    - netty-codec-http-4.1.48.Final.jar
+    - netty-codec-socks-4.1.48.Final.jar
+    - netty-common-4.1.48.Final.jar
+    - netty-handler-4.1.48.Final.jar
+    - netty-handler-proxy-4.1.48.Final.jar
     - netty-reactive-streams-2.0.0.jar
-    - netty-resolver-4.1.45.Final.jar
-    - netty-resolver-dns-4.1.45.Final.jar
-    - netty-tcnative-boringssl-static-2.0.26.Final.jar
-    - netty-transport-4.1.45.Final.jar
-    - netty-transport-native-epoll-4.1.45.Final.jar
-    - netty-transport-native-epoll-4.1.45.Final-linux-x86_64.jar
-    - netty-transport-native-unix-common-4.1.45.Final.jar
-    - netty-transport-native-unix-common-4.1.45.Final-linux-x86_64.jar
+    - netty-resolver-4.1.48.Final.jar
+    - netty-resolver-dns-4.1.48.Final.jar
+    - netty-tcnative-boringssl-static-2.0.30.Final.jar
+    - netty-transport-4.1.48.Final.jar
+    - netty-transport-native-epoll-4.1.48.Final.jar
+    - netty-transport-native-epoll-4.1.48.Final-linux-x86_64.jar
+    - netty-transport-native-unix-common-4.1.48.Final.jar
+    - netty-transport-native-unix-common-4.1.48.Final-linux-x86_64.jar
  * Joda Time
     - joda-time-2.9.9.jar
     - joda-time-2.10.1.jar


[pulsar] 17/38: Change nodurable cursor to active (#6769)

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaijia pushed a commit to branch branch-2.5
in repository https://gitbox.apache.org/repos/asf/pulsar.git

commit 0ce7a66999393340d5446d535240179fad7bbd5c
Author: zhaorongsheng <zh...@users.noreply.github.com>
AuthorDate: Wed Apr 22 17:27:58 2020 +0800

    Change nodurable cursor to active (#6769)
    
    ### Motivation
    
    When use non-durable subscription the cursor is not active which lead to the written entries are not put into cache. This situation would degrade reading performance.
    
    ### Modifications
    
    Change the `NonDurableCursorImpl` to active and remove the three override methods: `setActive()/isActive()/setInactive()`
    
    ### Verifying this change
    
    This change added tests and can be verified as follows:
    
    *ManagedCursorTest.testNonDurableCursorActive()*
      - add test to check `NonDurableCursorImpl` activity
    
    (cherry picked from commit 16ab351455778540831342ba55d1fb2cdaa596dc)
---
 .../apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java |  3 ++-
 .../bookkeeper/mledger/impl/NonDurableCursorImpl.java     | 15 ---------------
 .../apache/bookkeeper/mledger/impl/ManagedCursorTest.java | 11 +++++++++++
 3 files changed, 13 insertions(+), 16 deletions(-)

diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
index a64f8f5..bf7ad2b 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
@@ -854,6 +854,7 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
 
         NonDurableCursorImpl cursor = new NonDurableCursorImpl(bookKeeper, config, this, cursorName,
                 (PositionImpl) startCursorPosition);
+        cursor.setActive();
 
         log.info("[{}] Opened new cursor: {}", name, cursor);
         synchronized (this) {
@@ -2899,7 +2900,7 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
     }
 
     public boolean isCursorActive(ManagedCursor cursor) {
-        return cursor.isDurable() && activeCursors.get(cursor.getName()) != null;
+        return activeCursors.get(cursor.getName()) != null;
     }
 
     private boolean currentLedgerIsFull() {
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java
index bc57f79..a0720d4 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java
@@ -93,21 +93,6 @@ public class NonDurableCursorImpl extends ManagedCursorImpl {
     }
 
     @Override
-    public void setActive() {
-        /// No-Op
-    }
-
-    @Override
-    public boolean isActive() {
-        return false;
-    }
-
-    @Override
-    public void setInactive() {
-        /// No-Op
-    }
-
-    @Override
     public void asyncClose(CloseCallback callback, Object ctx) {
         // No-Op
         callback.closeComplete(ctx);
diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java
index 7bc0a39..6768e79 100644
--- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java
@@ -3001,6 +3001,17 @@ public class ManagedCursorTest extends MockedBookKeeperTestCase {
     }
 
     @Test
+    void testNonDurableCursorActive() throws Exception {
+        ManagedLedger ml = factory.open("testInactive");
+        ManagedCursor cursor = ml.newNonDurableCursor(PositionImpl.latest, "c1");
+
+        assertTrue(cursor.isActive());
+
+        cursor.setInactive();
+        assertFalse(cursor.isActive());
+    }
+
+    @Test
     public void deleteMessagesCheckhMarkDelete() throws Exception {
         ManagedLedger ledger = factory.open("my_test_ledger");
         ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1");