From 51f24298ec08487a234ca30d87b376e4b1fe3d12 Mon Sep 17 00:00:00 2001 From: Dian Qi Date: Fri, 18 Nov 2022 14:42:35 +0800 Subject: [PATCH] to #123, Implemented Pulsar source connector Added pulsar source v1 Pulsar source connector adapt to bitsail v1 source reader. Wired up bitsail config for Pulsar connector Simple implementation of pollNext function in Pulsar connector --- bitsail-connectors/connector-pulsar/pom.xml | 183 ++++ .../common/config/PulsarConfigUtils.java | 275 +++++ .../pulsar/common/config/PulsarOptions.java | 546 ++++++++++ .../common/config/v1/PulsarOptionsV1.java | 247 +++++ .../pulsar/common/config/v1/PulsarUtils.java | 426 ++++++++ .../pulsar/common/schema/PulsarSchema.java | 222 ++++ .../common/schema/PulsarSchemaFactory.java | 43 + .../schema/PulsarSchemaTypeInformation.java | 98 ++ .../schema/PulsarSchemaTypeSerializer.java | 203 ++++ .../common/schema/PulsarSchemaUtils.java | 201 ++++ .../schema/factories/AvroSchemaFactory.java | 48 + .../factories/BaseStructSchemaFactory.java | 41 + .../schema/factories/JSONSchemaFactory.java | 41 + .../factories/KeyValueSchemaFactory.java | 83 ++ .../factories/PrimitiveSchemaFactory.java | 89 ++ .../ProtobufNativeSchemaFactory.java | 43 + .../factories/ProtobufSchemaFactory.java | 43 + .../schema/factories/StringSchemaFactory.java | 47 + .../common/utils/PulsarExceptionUtils.java | 82 ++ .../pulsar/common/utils/PulsarSerdeUtils.java | 136 +++ .../common/utils/PulsarTransactionUtils.java | 117 +++ .../connector/pulsar/source/PulsarSource.java | 190 ++++ .../pulsar/source/PulsarSourceBuilder.java | 518 +++++++++ .../pulsar/source/PulsarSourceOptions.java | 610 +++++++++++ .../pulsar/source/PulsarSourceOptionsV1.java | 245 +++++ .../pulsar/source/PulsarSourceV1.java | 313 ++++++ .../source/config/CursorVerification.java | 32 + .../config/PulsarSourceConfigUtils.java | 197 ++++ .../source/config/SourceConfiguration.java | 203 ++++ .../coordinator/PulsarSourceEnumStateV1.java | 90 ++ .../PulsarSourceSplitCoordinator.java | 265 +++++ .../coordinator/SplitsAssignmentStateV1.java | 239 +++++ .../enumerator/PulsarSourceEnumState.java | 89 ++ .../PulsarSourceEnumStateSerializer.java | 122 +++ .../enumerator/PulsarSourceEnumerator.java | 269 +++++ .../enumerator/SplitsAssignmentState.java | 238 +++++ .../enumerator/cursor/CursorPosition.java | 89 ++ .../source/enumerator/cursor/StartCursor.java | 81 ++ .../source/enumerator/cursor/StopCursor.java | 76 ++ .../cursor/start/MessageIdStartCursor.java | 65 ++ .../cursor/start/TimestampStartCursor.java | 38 + .../cursor/stop/LatestMessageStopCursor.java | 51 + .../cursor/stop/MessageIdStopCursor.java | 54 + .../cursor/stop/NeverStopCursor.java | 32 + .../cursor/stop/TimestampStopCursor.java | 38 + .../subscriber/PulsarSubscriber.java | 70 ++ .../subscriber/impl/BasePulsarSubscriber.java | 79 ++ .../subscriber/impl/TopicListSubscriber.java | 56 + .../impl/TopicPatternSubscriber.java | 104 ++ .../enumerator/topic/TopicMetadata.java | 57 + .../enumerator/topic/TopicNameUtils.java | 44 + .../enumerator/topic/TopicPartition.java | 117 +++ .../source/enumerator/topic/TopicRange.java | 104 ++ .../topic/range/FixedRangeGenerator.java | 40 + .../topic/range/FullRangeGenerator.java | 39 + .../topic/range/RangeGenerator.java | 62 ++ .../topic/range/UniformRangeGenerator.java | 52 + .../reader/PulsarSourceReaderFactory.java | 135 +++ .../PulsarDeserializationSchema.java | 127 +++ ...ializationSchemaInitializationContext.java | 45 + .../PulsarDeserializationSchemaWrapper.java | 63 ++ .../deserializer/PulsarSchemaWrapper.java | 71 ++ .../PulsarTypeInformationWrapper.java | 64 ++ .../v1/PulsarDeserializationSchema.java | 59 ++ .../reader/emitter/PulsarRecordEmitter.java | 45 + .../fetcher/PulsarFetcherManagerBase.java | 121 +++ .../fetcher/PulsarOrderedFetcherManager.java | 74 ++ .../PulsarUnorderedFetcherManager.java | 66 ++ .../source/reader/message/PulsarMessage.java | 73 ++ .../message/PulsarMessageCollector.java | 59 ++ .../source/PulsarOrderedSourceReader.java | 204 ++++ .../reader/source/PulsarSourceReaderBase.java | 88 ++ .../source/PulsarUnorderedSourceReader.java | 146 +++ .../v1/PulsarOrderedSourceReaderV1.java | 205 ++++ .../source/v1/PulsarSourceReaderBaseV1.java | 302 ++++++ .../v1/PulsarUnorderedSourceReaderV1.java | 147 +++ .../PulsarOrderedPartitionSplitReader.java | 115 ++ .../split/PulsarPartitionSplitReaderBase.java | 234 +++++ .../PulsarUnorderedPartitionSplitReader.java | 183 ++++ .../v1/PulsarOrderedPartitionSplitReader.java | 114 ++ .../split/v1/PulsarPartitionSplitReader.java | 227 ++++ .../PulsarUnorderedPartitionSplitReader.java | 173 ++++ .../source/split/PulsarPartitionSplit.java | 119 +++ .../split/PulsarPartitionSplitSerializer.java | 158 +++ .../split/PulsarPartitionSplitState.java | 73 ++ .../source/split/v1/PulsarPartitionSplit.java | 124 +++ .../v1/PulsarPartitionSplitSerializer.java | 158 +++ .../split/v1/PulsarPartitionSplitState.java | 73 ++ .../pulsar/source/PulsarConnectorTest.java | 308 ++++++ .../pulsar/testutils/IntegerSource.java | 136 +++ .../test/resources/bitsail_pulsar_print.json | 48 + .../test/resources/pulsar/auth-client.conf | 64 ++ .../resources/pulsar/auth-standalone.conf | 832 +++++++++++++++ .../test/resources/pulsar/txnStandalone.conf | 980 ++++++++++++++++++ bitsail-connectors/pom.xml | 1 + bitsail-test/bitsail-connector-test/pom.xml | 6 + pom.xml | 19 +- 97 files changed, 14420 insertions(+), 1 deletion(-) create mode 100644 bitsail-connectors/connector-pulsar/pom.xml create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/PulsarConfigUtils.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/PulsarOptions.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/v1/PulsarOptionsV1.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/v1/PulsarUtils.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchema.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaFactory.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaTypeInformation.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaTypeSerializer.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaUtils.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/AvroSchemaFactory.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/BaseStructSchemaFactory.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/JSONSchemaFactory.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/PrimitiveSchemaFactory.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/ProtobufNativeSchemaFactory.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/ProtobufSchemaFactory.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/StringSchemaFactory.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/utils/PulsarExceptionUtils.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/utils/PulsarSerdeUtils.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/utils/PulsarTransactionUtils.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSource.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceBuilder.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceOptions.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceOptionsV1.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceV1.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/config/CursorVerification.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/config/PulsarSourceConfigUtils.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/config/SourceConfiguration.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/coordinator/PulsarSourceEnumStateV1.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/coordinator/PulsarSourceSplitCoordinator.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/coordinator/SplitsAssignmentStateV1.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/PulsarSourceEnumState.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/PulsarSourceEnumStateSerializer.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/SplitsAssignmentState.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/CursorPosition.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/StartCursor.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/StopCursor.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/TimestampStopCursor.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/impl/TopicListSubscriber.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicMetadata.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicNameUtils.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicPartition.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicRange.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/FixedRangeGenerator.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/FullRangeGenerator.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/RangeGenerator.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/UniformRangeGenerator.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/PulsarSourceReaderFactory.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaInitializationContext.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/v1/PulsarDeserializationSchema.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/emitter/PulsarRecordEmitter.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/fetcher/PulsarOrderedFetcherManager.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/fetcher/PulsarUnorderedFetcherManager.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/message/PulsarMessage.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/message/PulsarMessageCollector.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/v1/PulsarOrderedSourceReaderV1.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/v1/PulsarSourceReaderBaseV1.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/v1/PulsarUnorderedSourceReaderV1.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/v1/PulsarOrderedPartitionSplitReader.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/v1/PulsarPartitionSplitReader.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/v1/PulsarUnorderedPartitionSplitReader.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/PulsarPartitionSplit.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/PulsarPartitionSplitSerializer.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/PulsarPartitionSplitState.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/v1/PulsarPartitionSplit.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/v1/PulsarPartitionSplitSerializer.java create mode 100644 bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/v1/PulsarPartitionSplitState.java create mode 100644 bitsail-connectors/connector-pulsar/src/test/java/com/bytedance/bitsail/connector/pulsar/source/PulsarConnectorTest.java create mode 100644 bitsail-connectors/connector-pulsar/src/test/java/com/bytedance/bitsail/connector/pulsar/testutils/IntegerSource.java create mode 100644 bitsail-connectors/connector-pulsar/src/test/resources/bitsail_pulsar_print.json create mode 100644 bitsail-connectors/connector-pulsar/src/test/resources/pulsar/auth-client.conf create mode 100644 bitsail-connectors/connector-pulsar/src/test/resources/pulsar/auth-standalone.conf create mode 100644 bitsail-connectors/connector-pulsar/src/test/resources/pulsar/txnStandalone.conf diff --git a/bitsail-connectors/connector-pulsar/pom.xml b/bitsail-connectors/connector-pulsar/pom.xml new file mode 100644 index 000000000..dda61396c --- /dev/null +++ b/bitsail-connectors/connector-pulsar/pom.xml @@ -0,0 +1,183 @@ + + + + bitsail-connectors + com.bytedance.bitsail + ${revision} + + 4.0.0 + + connector-pulsar + + + 8 + 8 + 2.8.0 + + + 0.6.1 + 3.20.2 + 3.11 + 1.33.0 + 3.17.3 + + + + + + com.bytedance.bitsail + connector-base + ${revision} + + + com.bytedance.bitsail + bitsail-common + ${revision} + + + org.apache.flink + flink-connector-base + + + io.streamnative.connectors + pulsar-flink-connector_${scala.binary.version} + ${pulsar-flink-connector.version} + + + com.bytedance.bitsail + bitsail-connector-test + ${revision} + test + + + org.apache.flink + flink-test-utils_${scala.binary.version} + test + + + + + + com.google.protobuf + protobuf-java + ${protoc.version} + + + junit + junit + + + org.apache.logging.log4j + log4j-slf4j-impl + + + provided + true + + + + + + org.assertj + assertj-core + ${assertj-core.version} + test + + + + org.apache.flink + flink-core + ${flink.version} + test + test-jar + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + test + test-jar + + + + + + org.apache.pulsar + testmocks + ${pulsar.version} + test + + + org.testng + testng + + + org.powermock + powermock-module-testng + + + + + org.apache.pulsar + pulsar-broker + ${pulsar.version} + test + + + + + org.apache.commons + commons-lang3 + ${commons-lang3.version} + test + + + + + + + org.apache.pulsar + pulsar-client-all + ${pulsar.version} + + + org.apache.pulsar + pulsar-package-core + + + + + org.apache.flink + flink-annotations + ${flink.version} + + + com.bytedance.bitsail + bitsail-component-format-json + ${revision} + + + com.bytedance.bitsail + bitsail-connector-print + ${revision} + test + + + + + + + + io.grpc + grpc-bom + ${grpc.version} + pom + import + + + + + \ No newline at end of file diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/PulsarConfigUtils.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/PulsarConfigUtils.java new file mode 100644 index 000000000..628780907 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/PulsarConfigUtils.java @@ -0,0 +1,275 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.common.config; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminBuilder; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; +import org.apache.pulsar.client.api.ClientBuilder; +import org.apache.pulsar.client.api.ProxyProtocol; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; + +import java.util.Map; +import java.util.TreeSet; +import java.util.function.Consumer; +import java.util.function.Function; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; +import static java.util.function.Function.identity; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAMS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAM_MAP; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PLUGIN_CLASS_NAME; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTO_CERT_REFRESH_TIME; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_CONCURRENT_LOOKUP_REQUEST; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_CONNECTIONS_PER_BROKER; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_CONNECTION_TIMEOUT_MS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_CONNECT_TIMEOUT; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_ENABLE_BUSY_WAIT; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_ENABLE_TRANSACTION; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_INITIAL_BACKOFF_INTERVAL_NANOS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_KEEP_ALIVE_INTERVAL_SECONDS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_LISTENER_NAME; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_MAX_BACKOFF_INTERVAL_NANOS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_MAX_LOOKUP_REDIRECTS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_MAX_LOOKUP_REQUEST; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_MAX_NUMBER_OF_REJECTED_REQUEST_PER_CONNECTION; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_MEMORY_LIMIT_BYTES; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_NUM_IO_THREADS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_NUM_LISTENER_THREADS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_OPERATION_TIMEOUT_MS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_PROXY_PROTOCOL; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_PROXY_SERVICE_URL; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_READ_TIMEOUT; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_REQUEST_TIMEOUT; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_SSL_PROVIDER; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_STATS_INTERVAL_SECONDS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_TLS_ALLOW_INSECURE_CONNECTION; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_TLS_CIPHERS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_TLS_HOSTNAME_VERIFICATION_ENABLE; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_TLS_PROTOCOLS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_TLS_TRUST_CERTS_FILE_PATH; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_TLS_TRUST_STORE_PASSWORD; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_TLS_TRUST_STORE_PATH; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_TLS_TRUST_STORE_TYPE; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_USE_KEY_STORE_TLS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_USE_TCP_NO_DELAY; +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; +import static org.apache.pulsar.client.api.SizeUnit.BYTES; + +/** The util for creating pulsar configuration class from flink's {@link Configuration}. */ +@Internal +public final class PulsarConfigUtils { + + private PulsarConfigUtils() { + // No need to create instance. + } + + /** Create a PulsarClient by using the flink Configuration and the config customizer. */ + public static PulsarClient createClient(Configuration configuration) { + ClientBuilder builder = PulsarClient.builder(); + + setOptionValue(configuration, PULSAR_SERVICE_URL, builder::serviceUrl); + setOptionValue(configuration, PULSAR_LISTENER_NAME, builder::listenerName); + builder.authentication(createAuthentication(configuration)); + setOptionValue( + configuration, + PULSAR_OPERATION_TIMEOUT_MS, + timeout -> builder.operationTimeout(timeout, MILLISECONDS)); + setOptionValue(configuration, PULSAR_NUM_IO_THREADS, builder::ioThreads); + setOptionValue(configuration, PULSAR_NUM_LISTENER_THREADS, builder::listenerThreads); + setOptionValue(configuration, PULSAR_CONNECTIONS_PER_BROKER, builder::connectionsPerBroker); + setOptionValue(configuration, PULSAR_USE_TCP_NO_DELAY, builder::enableTcpNoDelay); + setOptionValue( + configuration, PULSAR_TLS_TRUST_CERTS_FILE_PATH, builder::tlsTrustCertsFilePath); + setOptionValue( + configuration, + PULSAR_TLS_ALLOW_INSECURE_CONNECTION, + builder::allowTlsInsecureConnection); + setOptionValue( + configuration, + PULSAR_TLS_HOSTNAME_VERIFICATION_ENABLE, + builder::enableTlsHostnameVerification); + setOptionValue(configuration, PULSAR_USE_KEY_STORE_TLS, builder::useKeyStoreTls); + setOptionValue(configuration, PULSAR_SSL_PROVIDER, builder::sslProvider); + setOptionValue(configuration, PULSAR_TLS_TRUST_STORE_TYPE, builder::tlsTrustStoreType); + setOptionValue(configuration, PULSAR_TLS_TRUST_STORE_PATH, builder::tlsTrustStorePath); + setOptionValue( + configuration, PULSAR_TLS_TRUST_STORE_PASSWORD, builder::tlsTrustStorePassword); + setOptionValue(configuration, PULSAR_TLS_CIPHERS, TreeSet::new, builder::tlsCiphers); + setOptionValue(configuration, PULSAR_TLS_PROTOCOLS, TreeSet::new, builder::tlsProtocols); + setOptionValue( + configuration, + PULSAR_MEMORY_LIMIT_BYTES, + bytes -> builder.memoryLimit(bytes, BYTES)); + setOptionValue( + configuration, + PULSAR_STATS_INTERVAL_SECONDS, + v -> builder.statsInterval(v, SECONDS)); + setOptionValue( + configuration, + PULSAR_CONCURRENT_LOOKUP_REQUEST, + builder::maxConcurrentLookupRequests); + setOptionValue(configuration, PULSAR_MAX_LOOKUP_REQUEST, builder::maxLookupRequests); + setOptionValue(configuration, PULSAR_MAX_LOOKUP_REDIRECTS, builder::maxLookupRedirects); + setOptionValue( + configuration, + PULSAR_MAX_NUMBER_OF_REJECTED_REQUEST_PER_CONNECTION, + builder::maxNumberOfRejectedRequestPerConnection); + setOptionValue( + configuration, + PULSAR_KEEP_ALIVE_INTERVAL_SECONDS, + v -> builder.keepAliveInterval(v, SECONDS)); + setOptionValue( + configuration, + PULSAR_CONNECTION_TIMEOUT_MS, + v -> builder.connectionTimeout(v, MILLISECONDS)); + setOptionValue( + configuration, + PULSAR_INITIAL_BACKOFF_INTERVAL_NANOS, + v -> builder.startingBackoffInterval(v, NANOSECONDS)); + setOptionValue( + configuration, + PULSAR_MAX_BACKOFF_INTERVAL_NANOS, + v -> builder.maxBackoffInterval(v, NANOSECONDS)); + setOptionValue(configuration, PULSAR_ENABLE_BUSY_WAIT, builder::enableBusyWait); + if (configuration.contains(PULSAR_PROXY_SERVICE_URL)) { + String proxyServiceUrl = configuration.get(PULSAR_PROXY_SERVICE_URL); + ProxyProtocol proxyProtocol = configuration.get(PULSAR_PROXY_PROTOCOL); + builder.proxyServiceUrl(proxyServiceUrl, proxyProtocol); + } + setOptionValue(configuration, PULSAR_ENABLE_TRANSACTION, builder::enableTransaction); + + return sneakyClient(builder::build); + } + + /** + * PulsarAdmin shares almost the same configuration with PulsarClient, but we separate this + * create method for directly create it. + */ + public static PulsarAdmin createAdmin(Configuration configuration) { + PulsarAdminBuilder builder = PulsarAdmin.builder(); + + setOptionValue(configuration, PULSAR_ADMIN_URL, builder::serviceHttpUrl); + builder.authentication(createAuthentication(configuration)); + setOptionValue( + configuration, PULSAR_TLS_TRUST_CERTS_FILE_PATH, builder::tlsTrustCertsFilePath); + setOptionValue( + configuration, + PULSAR_TLS_ALLOW_INSECURE_CONNECTION, + builder::allowTlsInsecureConnection); + setOptionValue( + configuration, + PULSAR_TLS_HOSTNAME_VERIFICATION_ENABLE, + builder::enableTlsHostnameVerification); + setOptionValue(configuration, PULSAR_USE_KEY_STORE_TLS, builder::useKeyStoreTls); + setOptionValue(configuration, PULSAR_SSL_PROVIDER, builder::sslProvider); + setOptionValue(configuration, PULSAR_TLS_TRUST_STORE_TYPE, builder::tlsTrustStoreType); + setOptionValue(configuration, PULSAR_TLS_TRUST_STORE_PATH, builder::tlsTrustStorePath); + setOptionValue( + configuration, PULSAR_TLS_TRUST_STORE_PASSWORD, builder::tlsTrustStorePassword); + setOptionValue(configuration, PULSAR_TLS_CIPHERS, TreeSet::new, builder::tlsCiphers); + setOptionValue(configuration, PULSAR_TLS_PROTOCOLS, TreeSet::new, builder::tlsProtocols); + setOptionValue( + configuration, + PULSAR_CONNECT_TIMEOUT, + v -> builder.connectionTimeout(v, MILLISECONDS)); + setOptionValue( + configuration, PULSAR_READ_TIMEOUT, v -> builder.readTimeout(v, MILLISECONDS)); + setOptionValue( + configuration, + PULSAR_REQUEST_TIMEOUT, + v -> builder.requestTimeout(v, MILLISECONDS)); + setOptionValue( + configuration, + PULSAR_AUTO_CERT_REFRESH_TIME, + v -> builder.autoCertRefreshTime(v, MILLISECONDS)); + + return sneakyClient(builder::build); + } + + /** + * Create the {@link Authentication} instance for both {@code PulsarClient} and {@code + * PulsarAdmin}. If the user didn't provide configuration, a {@link AuthenticationDisabled} + * instance would be returned. + * + *

This method behavior is the same as the pulsar command line tools. + */ + private static Authentication createAuthentication(Configuration configuration) { + if (configuration.contains(PULSAR_AUTH_PLUGIN_CLASS_NAME)) { + String authPluginClassName = configuration.get(PULSAR_AUTH_PLUGIN_CLASS_NAME); + + if (configuration.contains(PULSAR_AUTH_PARAMS)) { + String authParamsString = configuration.get(PULSAR_AUTH_PARAMS); + return sneakyClient( + () -> AuthenticationFactory.create(authPluginClassName, authParamsString)); + } else if (configuration.contains(PULSAR_AUTH_PARAM_MAP)) { + Map paramsMap = configuration.get(PULSAR_AUTH_PARAM_MAP); + return sneakyClient( + () -> AuthenticationFactory.create(authPluginClassName, paramsMap)); + } else { + throw new IllegalArgumentException( + String.format( + "No %s or %s provided", + PULSAR_AUTH_PARAMS.key(), PULSAR_AUTH_PARAM_MAP.key())); + } + } + + return AuthenticationDisabled.INSTANCE; + } + + /** Get the option value str from given config, convert it into the real value instance. */ + public static T getOptionValue( + Configuration configuration, ConfigOption option, Function convertor) { + F value = configuration.get(option); + if (value != null) { + return convertor.apply(value); + } else { + return null; + } + } + + /** Set the config option's value to a given builder. */ + public static void setOptionValue( + Configuration configuration, ConfigOption option, Consumer setter) { + setOptionValue(configuration, option, identity(), setter); + } + + /** + * Query the config option's value, convert it into a required type, set it to a given builder. + */ + public static void setOptionValue( + Configuration configuration, + ConfigOption option, + Function convertor, + Consumer setter) { + if (configuration.contains(option)) { + V value = getOptionValue(configuration, option, convertor); + setter.accept(value); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/PulsarOptions.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/PulsarOptions.java new file mode 100644 index 000000000..2ac17ea0e --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/PulsarOptions.java @@ -0,0 +1,546 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.common.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.docs.ConfigGroup; +import org.apache.flink.annotation.docs.ConfigGroups; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.description.Description; +import org.apache.pulsar.client.api.ProxyProtocol; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.flink.configuration.description.TextElement.code; +import static org.apache.flink.configuration.description.TextElement.text; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.ADMIN_CONFIG_PREFIX; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.CLIENT_CONFIG_PREFIX; + +/** + * Configuration for Pulsar Client, these config options would be used for both source, sink and + * table. + */ +@PublicEvolving +@ConfigGroups( + groups = { + @ConfigGroup(name = "PulsarClient", keyPrefix = CLIENT_CONFIG_PREFIX), + @ConfigGroup(name = "PulsarAdmin", keyPrefix = ADMIN_CONFIG_PREFIX) + }) +@SuppressWarnings("java:S1192") +public final class PulsarOptions { + + // Pulsar client API config prefix. + public static final String CLIENT_CONFIG_PREFIX = "pulsar.client."; + // Pulsar admin API config prefix. + public static final String ADMIN_CONFIG_PREFIX = "pulsar.admin."; + + private PulsarOptions() { + // This is a constant class + } + + /////////////////////////////////////////////////////////////////////////////// + // + // The configuration for ClientConfigurationData part. + // All the configuration listed below should have the pulsar.client prefix. + // + /////////////////////////////////////////////////////////////////////////////// + + public static final ConfigOption PULSAR_SERVICE_URL = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "serviceUrl") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text("Service URL provider for Pulsar service.") + .linebreak() + .text( + "To connect to Pulsar using client libraries, you need to specify a Pulsar protocol URL.") + .linebreak() + .text( + "You can assign Pulsar protocol URLs to specific clusters and use the %s scheme.", + code("pulsar")) + .linebreak() + .list( + text( + "This is an example of %s: %s.", + code("localhost"), + code("pulsar://localhost:6650")), + text( + "If you have multiple brokers, the URL is as: %s", + code( + "pulsar://localhost:6550,localhost:6651,localhost:6652")), + text( + "A URL for a production Pulsar cluster is as: %s", + code( + "pulsar://pulsar.us-west.example.com:6650")), + text( + "If you use TLS authentication, the URL is as %s", + code( + "pulsar+ssl://pulsar.us-west.example.com:6651"))) + .build()); + + public static final ConfigOption PULSAR_AUTH_PLUGIN_CLASS_NAME = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "authPluginClassName") + .stringType() + .noDefaultValue() + .withDescription("Name of the authentication plugin."); + + public static final ConfigOption PULSAR_AUTH_PARAMS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "authParams") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text("Parameters for the authentication plugin.") + .linebreak() + .linebreak() + .text("Example:") + .linebreak() + .add(code("key1:val1,key2:val2")) + .build()); + + public static final ConfigOption> PULSAR_AUTH_PARAM_MAP = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "authParamMap") + .mapType() + .noDefaultValue() + .withDescription("Parameters for the authentication plugin."); + + public static final ConfigOption PULSAR_OPERATION_TIMEOUT_MS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "operationTimeoutMs") + .intType() + .defaultValue(30000) + .withDescription( + Description.builder() + .text("Operation timeout (in ms).") + .text( + " Operations such as creating producers, subscribing or unsubscribing topics are retried during this interval.") + .text( + " If the operation is not completed during this interval, the operation will be marked as failed.") + .build()); + + public static final ConfigOption PULSAR_STATS_INTERVAL_SECONDS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "statsIntervalSeconds") + .longType() + .defaultValue(60L) + .withDescription( + Description.builder() + .text("Interval between each stats info.") + .linebreak() + .list( + text( + "Stats is activated with positive %s", + code("statsInterval")), + text( + "Set %s to 1 second at least.", + code("statsIntervalSeconds"))) + .build()); + + public static final ConfigOption PULSAR_NUM_IO_THREADS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "numIoThreads") + .intType() + .defaultValue(1) + .withDescription( + "The number of threads used for handling connections to brokers."); + + public static final ConfigOption PULSAR_NUM_LISTENER_THREADS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "numListenerThreads") + .intType() + .defaultValue(1) + .withDescription( + Description.builder() + .text( + "The number of threads used for handling message listeners.") + .text( + " The listener thread pool is shared across all the consumers and readers that are using a %s model to get messages.", + code("listener")) + .text( + " For a given consumer, the listener is always invoked from the same thread to ensure ordering.") + .build()); + + public static final ConfigOption PULSAR_CONNECTIONS_PER_BROKER = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "connectionsPerBroker") + .intType() + .defaultValue(1) + .withDescription( + Description.builder() + .text( + "The maximum number of connections that the client library will open to a single broker.") + .linebreak() + .text( + " By default, the connection pool will use a single connection for all the producers and consumers.") + .text( + " Increasing this parameter may improve throughput when using many producers over a high latency connection.") + .build()); + + public static final ConfigOption PULSAR_USE_TCP_NO_DELAY = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "useTcpNoDelay") + .booleanType() + .defaultValue(true) + .withDescription( + Description.builder() + .text( + "Whether to use the TCP no-delay flag on the connection to disable Nagle algorithm.") + .linebreak() + .text( + "No-delay features ensures that packets are sent out on the network as soon as possible,") + .text(" and it is critical to achieve low latency publishes.") + .text( + " On the other hand, sending out a huge number of small packets might limit the overall throughput.") + .text( + " Therefore, if latency is not a concern, it is recommended to set the %s flag to %s.", + code("useTcpNoDelay"), code("false")) + .linebreak() + .text("By default, it is set to %s.", code("true")) + .build()); + + public static final ConfigOption PULSAR_TLS_TRUST_CERTS_FILE_PATH = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsTrustCertsFilePath") + .stringType() + .defaultValue("") + .withDescription("Path to the trusted TLS certificate file."); + + public static final ConfigOption PULSAR_TLS_ALLOW_INSECURE_CONNECTION = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsAllowInsecureConnection") + .booleanType() + .defaultValue(false) + .withDescription( + "Whether the Pulsar client accepts untrusted TLS certificate from the broker."); + + public static final ConfigOption PULSAR_TLS_HOSTNAME_VERIFICATION_ENABLE = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsHostnameVerificationEnable") + .booleanType() + .defaultValue(false) + .withDescription( + Description.builder() + .text("Whether to enable TLS hostname verification.") + .text( + " It allows to validate hostname verification when a client connects to the broker over TLS.") + .text( + " It validates incoming x509 certificate and matches provided hostname (CN/SAN) with the expected broker's host name.") + .text( + " It follows RFC 2818, 3.1. Server Identity hostname verification.") + .build()); + + public static final ConfigOption PULSAR_CONCURRENT_LOOKUP_REQUEST = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "concurrentLookupRequest") + .intType() + .defaultValue(5000) + .withDescription( + Description.builder() + .text( + "The number of concurrent lookup requests allowed to send on each broker connection to prevent overload on the broker.") + .text( + " It should be configured with a higher value only in case of it requires to produce or subscribe on thousands of topic using a created %s", + code("PulsarClient")) + .build()); + + public static final ConfigOption PULSAR_MAX_LOOKUP_REQUEST = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "maxLookupRequest") + .intType() + .defaultValue(50000) + .withDescription( + Description.builder() + .text( + "The maximum number of lookup requests allowed on each broker connection to prevent overload on the broker.") + .text( + " It should be greater than %s.", + code("maxConcurrentLookupRequests")) + .text( + " Requests that inside %s are already sent to broker,", + code("maxConcurrentLookupRequests")) + .text( + " and requests beyond %s and under %s will wait in each client cnx.", + code("maxConcurrentLookupRequests"), + code("maxLookupRequests")) + .build()); + + public static final ConfigOption PULSAR_MAX_LOOKUP_REDIRECTS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "maxLookupRedirects") + .intType() + .defaultValue(20) + .withDescription( + "The maximum number of times a lookup-request redirections to a broker."); + + public static final ConfigOption PULSAR_MAX_NUMBER_OF_REJECTED_REQUEST_PER_CONNECTION = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "maxNumberOfRejectedRequestPerConnection") + .intType() + .defaultValue(50) + .withDescription( + Description.builder() + .text( + "The maximum number of rejected requests of a broker in a certain period (30s) after the current connection is closed") + .text( + " and the client creates a new connection to connect to a different broker.") + .build()); + + public static final ConfigOption PULSAR_KEEP_ALIVE_INTERVAL_SECONDS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "keepAliveIntervalSeconds") + .intType() + .defaultValue(30) + .withDescription( + "Interval (in seconds) for keeping connection between the Pulsar client and broker alive."); + + public static final ConfigOption PULSAR_CONNECTION_TIMEOUT_MS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "connectionTimeoutMs") + .intType() + .defaultValue(10000) + .withDescription( + Description.builder() + .text( + "Duration (in ms) of waiting for a connection to a broker to be established.") + .linebreak() + .text( + "If the duration passes without a response from a broker, the connection attempt is dropped.") + .build()); + + // TODO This option would be exposed by Pulsar's ClientBuilder in the next Pulsar release. + public static final ConfigOption PULSAR_REQUEST_TIMEOUT_MS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "requestTimeoutMs") + .intType() + .defaultValue(60000) + .withDescription("Maximum duration (in ms) for completing a request."); + + public static final ConfigOption PULSAR_INITIAL_BACKOFF_INTERVAL_NANOS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "initialBackoffIntervalNanos") + .longType() + .defaultValue(TimeUnit.MILLISECONDS.toNanos(100)) + .withDescription("Default duration (in nanoseconds) for a backoff interval."); + + public static final ConfigOption PULSAR_MAX_BACKOFF_INTERVAL_NANOS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "maxBackoffIntervalNanos") + .longType() + .defaultValue(SECONDS.toNanos(60)) + .withDescription( + "The maximum duration (in nanoseconds) for a backoff interval."); + + public static final ConfigOption PULSAR_ENABLE_BUSY_WAIT = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "enableBusyWait") + .booleanType() + .defaultValue(false) + .withDescription( + Description.builder() + .text("Option to enable busy-wait settings.") + .linebreak() + .text( + "This option will enable spin-waiting on executors and IO threads in order to reduce latency during context switches.") + .text( + " The spinning will consume 100% CPU even when the broker is not doing any work.") + .text( + " It is recommended to reduce the number of IO threads and BookKeeper client threads to only have fewer CPU cores busy.") + .build()); + + public static final ConfigOption PULSAR_LISTENER_NAME = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "listenerName") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "Configure the %s that the broker will return the corresponding %s.", + code("listenerName"), code("advertisedListener")) + .build()); + + public static final ConfigOption PULSAR_USE_KEY_STORE_TLS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "useKeyStoreTls") + .booleanType() + .defaultValue(false) + .withDescription( + Description.builder() + .text( + "If TLS is enabled, whether use the KeyStore type as the TLS configuration parameter.") + .text( + " If it is set to %s, it means to use the default pem type configuration.", + code("false")) + .build()); + + public static final ConfigOption PULSAR_SSL_PROVIDER = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "sslProvider") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "The name of the security provider used for SSL connections.") + .text( + " The default value is the default security provider of the JVM.") + .build()); + + public static final ConfigOption PULSAR_TLS_TRUST_STORE_TYPE = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsTrustStoreType") + .stringType() + .defaultValue("JKS") + .withDescription("The file format of the trust store file."); + + public static final ConfigOption PULSAR_TLS_TRUST_STORE_PATH = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsTrustStorePath") + .stringType() + .noDefaultValue() + .withDescription("The location of the trust store file."); + + public static final ConfigOption PULSAR_TLS_TRUST_STORE_PASSWORD = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsTrustStorePassword") + .stringType() + .noDefaultValue() + .withDescription("The store password for the key store file."); + + // The real config type is Set, you should provided a json str here. + public static final ConfigOption> PULSAR_TLS_CIPHERS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsCiphers") + .stringType() + .asList() + .defaultValues() + .withDescription( + Description.builder() + .text("A list of cipher suites.") + .text( + " This is a named combination of authentication, encryption,") + .text( + " MAC and the key exchange algorithm used to negotiate the security settings for a network connection using the TLS or SSL network protocol.") + .text( + " By default all the available cipher suites are supported.") + .build()); + + public static final ConfigOption> PULSAR_TLS_PROTOCOLS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsProtocols") + .stringType() + .asList() + .defaultValues() + .withDescription( + Description.builder() + .text("The SSL protocol used to generate the SSLContext.") + .text( + " By default, it is set TLS, which is fine for most cases.") + .text( + " Allowed values in recent JVMs are TLS, TLSv1.3, TLSv1.2 and TLSv1.1.") + .build()); + + public static final ConfigOption PULSAR_MEMORY_LIMIT_BYTES = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "memoryLimitBytes") + .longType() + .defaultValue(0L) + .withDescription( + Description.builder() + .text( + "The limit (in bytes) on the amount of direct memory that will be allocated by this client instance.") + .linebreak() + .text( + "Note: at this moment this is only limiting the memory for producers.") + .text(" Setting this to %s will disable the limit.", code("0")) + .build()); + + public static final ConfigOption PULSAR_PROXY_SERVICE_URL = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "proxyServiceUrl") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "Proxy-service URL when a client connects to the broker via the proxy.") + .text(" The client can choose the type of proxy-routing.") + .build()); + + public static final ConfigOption PULSAR_PROXY_PROTOCOL = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "proxyProtocol") + .enumType(ProxyProtocol.class) + .defaultValue(ProxyProtocol.SNI) + .withDescription( + Description.builder() + .text( + "Protocol type to determine the type of proxy routing when a client connects to the proxy using %s.", + code("pulsar.client.proxyServiceUrl")) + .build()); + + public static final ConfigOption PULSAR_ENABLE_TRANSACTION = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "enableTransaction") + .booleanType() + .defaultValue(false) + .withDescription( + Description.builder() + .text( + "If transaction is enabled, start the %s with %s.", + code("transactionCoordinatorClient"), + code("PulsarClient")) + .build()); + + /////////////////////////////////////////////////////////////////////////////// + // + // The configuration for PulsarAdmin part. + // All the configuration listed below should have the pulsar.admin prefix. + // + /////////////////////////////////////////////////////////////////////////////// + + public static final ConfigOption PULSAR_ADMIN_URL = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "adminUrl") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "The Pulsar service HTTP URL for the admin endpoint. For example, %s, or %s for TLS.", + code("http://my-broker.example.com:8080"), + code("https://my-broker.example.com:8443")) + .build()); + public static final ConfigOption PULSAR_TOPIC_MODE = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "topicMode") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text("Mode to subscribe to topics. Available options are `list` and `pattern`") + .build()); + public static final ConfigOption PULSAR_TOPICS = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "topics") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text("The topics to subscribe. Can be a regex pattern or comma-delimited string") + .build()); + + public static final ConfigOption PULSAR_CONNECT_TIMEOUT = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "connectTimeout") + .intType() + .defaultValue(60000) + .withDescription("The connection time out (in ms) for the PulsarAdmin client."); + + public static final ConfigOption PULSAR_READ_TIMEOUT = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "readTimeout") + .intType() + .defaultValue(60000) + .withDescription( + "The server response read timeout (in ms) for the PulsarAdmin client for any request."); + + public static final ConfigOption PULSAR_REQUEST_TIMEOUT = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "requestTimeout") + .intType() + .defaultValue(300000) + .withDescription( + "The server request timeout (in ms) for the PulsarAdmin client for any request."); + + public static final ConfigOption PULSAR_AUTO_CERT_REFRESH_TIME = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "autoCertRefreshTime") + .intType() + .defaultValue(300000) + .withDescription( + "The auto cert refresh time (in ms) if Pulsar admin supports TLS authentication."); +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/v1/PulsarOptionsV1.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/v1/PulsarOptionsV1.java new file mode 100644 index 000000000..2edabee72 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/v1/PulsarOptionsV1.java @@ -0,0 +1,247 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.common.config.v1; + +import com.bytedance.bitsail.common.option.ConfigOption; +import com.bytedance.bitsail.common.option.ConfigOptions; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.docs.ConfigGroup; +import org.apache.flink.annotation.docs.ConfigGroups; +import org.apache.flink.configuration.description.Description; +import org.apache.pulsar.client.api.ProxyProtocol; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.ADMIN_CONFIG_PREFIX; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.CLIENT_CONFIG_PREFIX; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.flink.configuration.description.TextElement.code; +import static org.apache.flink.configuration.description.TextElement.text; + +/** + * Configuration for Pulsar Client, these config options would be used for both source, sink and + * table. + */ +@PublicEvolving +@ConfigGroups( + groups = { + @ConfigGroup(name = "PulsarClient", keyPrefix = CLIENT_CONFIG_PREFIX), + @ConfigGroup(name = "PulsarAdmin", keyPrefix = ADMIN_CONFIG_PREFIX) + }) +@SuppressWarnings("java:S1192") +public final class PulsarOptionsV1 { + public static final String READER_PREFIX = "job.reader."; + + + // Pulsar client API config prefix. + public static final String CLIENT_CONFIG_PREFIX = READER_PREFIX + "pulsar.client."; + // Pulsar admin API config prefix. + public static final String ADMIN_CONFIG_PREFIX = READER_PREFIX + "pulsar.admin."; + + private PulsarOptionsV1() { + // This is a constant class + } + + /////////////////////////////////////////////////////////////////////////////// + // + // The configuration for ClientConfigurationData part. + // All the configuration listed below should have the pulsar.client prefix. + // + /////////////////////////////////////////////////////////////////////////////// + + public static final ConfigOption PULSAR_SERVICE_URL = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "serviceUrl") + .noDefaultValue(String.class); + + public static final ConfigOption PULSAR_AUTH_PLUGIN_CLASS_NAME = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "authPluginClassName") + .noDefaultValue(String.class); + + public static final ConfigOption PULSAR_AUTH_PARAMS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "authParams") + .noDefaultValue(String.class); + + public static final ConfigOption PULSAR_AUTH_PARAM_MAP = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "authParamMap") + .noDefaultValue(Map.class); + + public static final ConfigOption PULSAR_OPERATION_TIMEOUT_MS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "operationTimeoutMs") + .defaultValue(30000); + + public static final ConfigOption PULSAR_STATS_INTERVAL_SECONDS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "statsIntervalSeconds") + .defaultValue(60L); + + public static final ConfigOption PULSAR_NUM_IO_THREADS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "numIoThreads") + .defaultValue(1); + + public static final ConfigOption PULSAR_NUM_LISTENER_THREADS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "numListenerThreads") + .defaultValue(1); + + public static final ConfigOption PULSAR_CONNECTIONS_PER_BROKER = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "connectionsPerBroker") + .defaultValue(1); + + public static final ConfigOption PULSAR_USE_TCP_NO_DELAY = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "useTcpNoDelay") + .defaultValue(true); + + public static final ConfigOption PULSAR_TLS_TRUST_CERTS_FILE_PATH = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsTrustCertsFilePath") + .defaultValue(""); + + public static final ConfigOption PULSAR_TLS_ALLOW_INSECURE_CONNECTION = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsAllowInsecureConnection") + .defaultValue(false); + + public static final ConfigOption PULSAR_TLS_HOSTNAME_VERIFICATION_ENABLE = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsHostnameVerificationEnable") + .defaultValue(false); + + public static final ConfigOption PULSAR_CONCURRENT_LOOKUP_REQUEST = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "concurrentLookupRequest") + .defaultValue(5000); + + public static final ConfigOption PULSAR_MAX_LOOKUP_REQUEST = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "maxLookupRequest") + .defaultValue(50000); + + public static final ConfigOption PULSAR_MAX_LOOKUP_REDIRECTS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "maxLookupRedirects") + .defaultValue(20); + + public static final ConfigOption PULSAR_MAX_NUMBER_OF_REJECTED_REQUEST_PER_CONNECTION = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "maxNumberOfRejectedRequestPerConnection") + .defaultValue(50); + + public static final ConfigOption PULSAR_KEEP_ALIVE_INTERVAL_SECONDS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "keepAliveIntervalSeconds") + .defaultValue(30); + + public static final ConfigOption PULSAR_CONNECTION_TIMEOUT_MS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "connectionTimeoutMs") + .defaultValue(10000); + + // TODO This option would be exposed by Pulsar's ClientBuilder in the next Pulsar release. + public static final ConfigOption PULSAR_REQUEST_TIMEOUT_MS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "requestTimeoutMs") + .defaultValue(60000); + + public static final ConfigOption PULSAR_INITIAL_BACKOFF_INTERVAL_NANOS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "initialBackoffIntervalNanos") + .defaultValue(TimeUnit.MILLISECONDS.toNanos(100)); + + public static final ConfigOption PULSAR_MAX_BACKOFF_INTERVAL_NANOS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "maxBackoffIntervalNanos") + .defaultValue(SECONDS.toNanos(60)); + + public static final ConfigOption PULSAR_ENABLE_BUSY_WAIT = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "enableBusyWait") + .defaultValue(false); + + public static final ConfigOption PULSAR_LISTENER_NAME = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "listenerName") + .noDefaultValue(String.class); + + public static final ConfigOption PULSAR_USE_KEY_STORE_TLS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "useKeyStoreTls") + .defaultValue(false); + + public static final ConfigOption PULSAR_SSL_PROVIDER = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "sslProvider") + .noDefaultValue(String.class); + + public static final ConfigOption PULSAR_TLS_TRUST_STORE_TYPE = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsTrustStoreType") + .defaultValue("JKS"); + + public static final ConfigOption PULSAR_TLS_TRUST_STORE_PATH = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsTrustStorePath") + .noDefaultValue(String.class); + + public static final ConfigOption PULSAR_TLS_TRUST_STORE_PASSWORD = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsTrustStorePassword") + .noDefaultValue(String.class); + + // The real config type is Set, you should provided a json str here. + public static final ConfigOption> PULSAR_TLS_CIPHERS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsCiphers") + .defaultValue(new ArrayList<>()); + + public static final ConfigOption> PULSAR_TLS_PROTOCOLS = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "tlsProtocols") + .defaultValue(new ArrayList<>()); + + public static final ConfigOption PULSAR_MEMORY_LIMIT_BYTES = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "memoryLimitBytes") + .defaultValue(0L); + + public static final ConfigOption PULSAR_PROXY_SERVICE_URL = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "proxyServiceUrl") + .noDefaultValue(String.class); + + public static final ConfigOption PULSAR_PROXY_PROTOCOL = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "proxyProtocol") + .defaultValue(ProxyProtocol.SNI); + + public static final ConfigOption PULSAR_ENABLE_TRANSACTION = + ConfigOptions.key(CLIENT_CONFIG_PREFIX + "enableTransaction") + .defaultValue(false); + + /////////////////////////////////////////////////////////////////////////////// + // + // The configuration for PulsarAdmin part. + // All the configuration listed below should have the pulsar.admin prefix. + // + /////////////////////////////////////////////////////////////////////////////// + + public static final ConfigOption PULSAR_ADMIN_URL = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "adminUrl") + .noDefaultValue(String.class); + public static final ConfigOption PULSAR_TOPIC_MODE = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "topicMode") + .noDefaultValue(String.class); + public static final ConfigOption PULSAR_TOPICS = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "topics") + .noDefaultValue(String.class); + + public static final ConfigOption PULSAR_CONNECT_TIMEOUT = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "connectTimeout") + .defaultValue(60000); + + public static final ConfigOption PULSAR_READ_TIMEOUT = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "readTimeout") + .defaultValue(60000); + + public static final ConfigOption PULSAR_REQUEST_TIMEOUT = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "requestTimeout") + .defaultValue(300000); + + public static final ConfigOption PULSAR_AUTO_CERT_REFRESH_TIME = + ConfigOptions.key(ADMIN_CONFIG_PREFIX + "autoCertRefreshTime") + .defaultValue(300000); +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/v1/PulsarUtils.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/v1/PulsarUtils.java new file mode 100644 index 000000000..8c7217784 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/config/v1/PulsarUtils.java @@ -0,0 +1,426 @@ +package com.bytedance.bitsail.connector.pulsar.common.config.v1; + +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.common.option.ConfigOption; +import com.bytedance.bitsail.connector.pulsar.source.config.CursorVerification; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicMetadata; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicNameUtils; + +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.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; +import org.apache.pulsar.client.api.ClientBuilder; +import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; +import org.apache.pulsar.client.api.DeadLetterPolicy; +import org.apache.pulsar.client.api.ProxyProtocol; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.RegexSubscriptionMode; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionMode; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; + +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.TreeSet; +import java.util.function.Consumer; +import java.util.function.Function; + +import static com.bytedance.bitsail.connector.pulsar.common.config.v1.PulsarOptionsV1.*; +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptionsV1.*; +import static java.util.concurrent.TimeUnit.MICROSECONDS; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; +import static java.util.function.Function.identity; +import static org.apache.pulsar.client.api.SizeUnit.BYTES; + +public class PulsarUtils { + + public static final String TOPIC_MODE_LIST = "list"; + public static final String TOPIC_MODE_PATTERN = "pattern"; + public static final String TOPIC_LIST_DELIMITER = ","; + + /** + * PulsarAdmin shares almost the same configuration with PulsarClient, but we separate this + * create method for directly create it. + */ + public static PulsarAdmin createAdmin(BitSailConfiguration configuration) { + PulsarAdminBuilder builder = PulsarAdmin.builder(); + + setOptionValue(configuration, PULSAR_ADMIN_URL, builder::serviceHttpUrl); + builder.authentication(createAuthentication(configuration)); + setOptionValue( + configuration, PULSAR_TLS_TRUST_CERTS_FILE_PATH, builder::tlsTrustCertsFilePath); + setOptionValue( + configuration, + PULSAR_TLS_ALLOW_INSECURE_CONNECTION, + builder::allowTlsInsecureConnection); + setOptionValue( + configuration, + PULSAR_TLS_HOSTNAME_VERIFICATION_ENABLE, + builder::enableTlsHostnameVerification); + setOptionValue(configuration, PULSAR_USE_KEY_STORE_TLS, builder::useKeyStoreTls); + setOptionValue(configuration, PULSAR_SSL_PROVIDER, builder::sslProvider); + setOptionValue(configuration, PULSAR_TLS_TRUST_STORE_TYPE, builder::tlsTrustStoreType); + setOptionValue(configuration, PULSAR_TLS_TRUST_STORE_PATH, builder::tlsTrustStorePath); + setOptionValue( + configuration, PULSAR_TLS_TRUST_STORE_PASSWORD, builder::tlsTrustStorePassword); + setOptionValue(configuration, PULSAR_TLS_CIPHERS, TreeSet::new, builder::tlsCiphers); + setOptionValue(configuration, PULSAR_TLS_PROTOCOLS, TreeSet::new, builder::tlsProtocols); + setOptionValue( + configuration, + PULSAR_CONNECT_TIMEOUT, + v -> builder.connectionTimeout(v, MILLISECONDS)); + setOptionValue( + configuration, PULSAR_READ_TIMEOUT, v -> builder.readTimeout(v, MILLISECONDS)); + setOptionValue( + configuration, + PULSAR_REQUEST_TIMEOUT, + v -> builder.requestTimeout(v, MILLISECONDS)); + setOptionValue( + configuration, + PULSAR_AUTO_CERT_REFRESH_TIME, + v -> builder.autoCertRefreshTime(v, MILLISECONDS)); + + return sneakyClient(builder::build); + } + + public static PulsarClient createClient(BitSailConfiguration configuration) { + ClientBuilder builder = PulsarClient.builder(); + + setOptionValue(configuration, PULSAR_SERVICE_URL, builder::serviceUrl); + setOptionValue(configuration, PULSAR_LISTENER_NAME, builder::listenerName); + builder.authentication(createAuthentication(configuration)); + setOptionValue( + configuration, + PULSAR_OPERATION_TIMEOUT_MS, + timeout -> builder.operationTimeout(timeout, MILLISECONDS)); + setOptionValue(configuration, PULSAR_NUM_IO_THREADS, builder::ioThreads); + setOptionValue(configuration, PULSAR_NUM_LISTENER_THREADS, builder::listenerThreads); + setOptionValue(configuration, PULSAR_CONNECTIONS_PER_BROKER, builder::connectionsPerBroker); + setOptionValue(configuration, PULSAR_USE_TCP_NO_DELAY, builder::enableTcpNoDelay); + setOptionValue( + configuration, PULSAR_TLS_TRUST_CERTS_FILE_PATH, builder::tlsTrustCertsFilePath); + setOptionValue( + configuration, + PULSAR_TLS_ALLOW_INSECURE_CONNECTION, + builder::allowTlsInsecureConnection); + setOptionValue( + configuration, + PULSAR_TLS_HOSTNAME_VERIFICATION_ENABLE, + builder::enableTlsHostnameVerification); + setOptionValue(configuration, PULSAR_USE_KEY_STORE_TLS, builder::useKeyStoreTls); + setOptionValue(configuration, PULSAR_SSL_PROVIDER, builder::sslProvider); + setOptionValue(configuration, PULSAR_TLS_TRUST_STORE_TYPE, builder::tlsTrustStoreType); + setOptionValue(configuration, PULSAR_TLS_TRUST_STORE_PATH, builder::tlsTrustStorePath); + setOptionValue( + configuration, PULSAR_TLS_TRUST_STORE_PASSWORD, builder::tlsTrustStorePassword); + setOptionValue(configuration, PULSAR_TLS_CIPHERS, TreeSet::new, builder::tlsCiphers); + setOptionValue(configuration, PULSAR_TLS_PROTOCOLS, TreeSet::new, builder::tlsProtocols); + setOptionValue( + configuration, + PULSAR_MEMORY_LIMIT_BYTES, + bytes -> builder.memoryLimit(bytes, BYTES)); + setOptionValue( + configuration, + PULSAR_STATS_INTERVAL_SECONDS, + v -> builder.statsInterval(v, SECONDS)); + setOptionValue( + configuration, + PULSAR_CONCURRENT_LOOKUP_REQUEST, + builder::maxConcurrentLookupRequests); + setOptionValue(configuration, PULSAR_MAX_LOOKUP_REQUEST, builder::maxLookupRequests); + setOptionValue(configuration, PULSAR_MAX_LOOKUP_REDIRECTS, builder::maxLookupRedirects); + setOptionValue( + configuration, + PULSAR_MAX_NUMBER_OF_REJECTED_REQUEST_PER_CONNECTION, + builder::maxNumberOfRejectedRequestPerConnection); + setOptionValue( + configuration, + PULSAR_KEEP_ALIVE_INTERVAL_SECONDS, + v -> builder.keepAliveInterval(v, SECONDS)); + setOptionValue( + configuration, + PULSAR_CONNECTION_TIMEOUT_MS, + v -> builder.connectionTimeout(v, MILLISECONDS)); + setOptionValue( + configuration, + PULSAR_INITIAL_BACKOFF_INTERVAL_NANOS, + v -> builder.startingBackoffInterval(v, NANOSECONDS)); + setOptionValue( + configuration, + PULSAR_MAX_BACKOFF_INTERVAL_NANOS, + v -> builder.maxBackoffInterval(v, NANOSECONDS)); + setOptionValue(configuration, PULSAR_ENABLE_BUSY_WAIT, builder::enableBusyWait); + if (configuration.fieldExists(PULSAR_PROXY_SERVICE_URL.key())) { + String proxyServiceUrl = configuration.getString(PULSAR_PROXY_SERVICE_URL.key()); + ProxyProtocol proxyProtocol = getProxyProtocol(configuration.getString(PULSAR_PROXY_PROTOCOL.key())); + builder.proxyServiceUrl(proxyServiceUrl, proxyProtocol); + } + setOptionValue(configuration, PULSAR_ENABLE_TRANSACTION, builder::enableTransaction); + + return sneakyClient(builder::build); + } + + /** Create a pulsar consumer builder by using the given Configuration. */ + public static ConsumerBuilder createConsumerBuilder( + PulsarClient client, Schema schema, BitSailConfiguration configuration) { + ConsumerBuilder builder = client.newConsumer(schema); + + setOptionValue(configuration, PULSAR_SUBSCRIPTION_NAME, builder::subscriptionName); + setOptionValue( + configuration, PULSAR_ACK_TIMEOUT_MILLIS, v -> builder.ackTimeout(v, MILLISECONDS)); + setOptionValue(configuration, PULSAR_ACK_RECEIPT_ENABLED, builder::isAckReceiptEnabled); + setOptionValue( + configuration, + PULSAR_TICK_DURATION_MILLIS, + v -> builder.ackTimeoutTickTime(v, MILLISECONDS)); + setOptionValue( + configuration, + PULSAR_NEGATIVE_ACK_REDELIVERY_DELAY_MICROS, + v -> builder.negativeAckRedeliveryDelay(v, MICROSECONDS)); + setOptionValue(configuration, PULSAR_SUBSCRIPTION_TYPE, e -> builder.subscriptionType(PulsarUtils.getSubscriptionType(e))); + setOptionValue(configuration, PULSAR_SUBSCRIPTION_MODE, e -> builder.subscriptionMode(PulsarUtils.getSubscriptionMode(e))); + setOptionValue(configuration, PULSAR_CRYPTO_FAILURE_ACTION, e -> builder.cryptoFailureAction(PulsarUtils.getCryptoFailureAction(e))); + setOptionValue(configuration, PULSAR_RECEIVER_QUEUE_SIZE, builder::receiverQueueSize); + setOptionValue( + configuration, + PULSAR_ACKNOWLEDGEMENTS_GROUP_TIME_MICROS, + v -> builder.acknowledgmentGroupTime(v, MICROSECONDS)); + setOptionValue( + configuration, + PULSAR_REPLICATE_SUBSCRIPTION_STATE, + builder::replicateSubscriptionState); + setOptionValue( + configuration, + PULSAR_MAX_TOTAL_RECEIVER_QUEUE_SIZE_ACROSS_PARTITIONS, + builder::maxTotalReceiverQueueSizeAcrossPartitions); + setOptionValue(configuration, PULSAR_CONSUMER_NAME, builder::consumerName); + setOptionValue(configuration, PULSAR_READ_COMPACTED, builder::readCompacted); + setOptionValue(configuration, PULSAR_PRIORITY_LEVEL, builder::priorityLevel); + setOptionValue(configuration, PULSAR_CONSUMER_PROPERTIES, builder::properties); + setOptionValue( + configuration, + PULSAR_SUBSCRIPTION_INITIAL_POSITION, + e -> builder.subscriptionInitialPosition(PulsarUtils.getSubscriptionInitialPosition(e))); + createDeadLetterPolicy(configuration).ifPresent(builder::deadLetterPolicy); + setOptionValue( + configuration, + PULSAR_AUTO_UPDATE_PARTITIONS_INTERVAL_SECONDS, + v -> builder.autoUpdatePartitionsInterval(v, SECONDS)); + setOptionValue(configuration, PULSAR_RETRY_ENABLE, builder::enableRetry); + setOptionValue( + configuration, + PULSAR_MAX_PENDING_CHUNKED_MESSAGE, + builder::maxPendingChunkedMessage); + setOptionValue( + configuration, + PULSAR_AUTO_ACK_OLDEST_CHUNKED_MESSAGE_ON_QUEUE_FULL, + builder::autoAckOldestChunkedMessageOnQueueFull); + setOptionValue( + configuration, + PULSAR_EXPIRE_TIME_OF_INCOMPLETE_CHUNKED_MESSAGE_MILLIS, + v -> builder.expireTimeOfIncompleteChunkedMessage(v, MILLISECONDS)); + setOptionValue(configuration, PULSAR_POOL_MESSAGES, builder::poolMessages); + + return builder; + } + + private static SubscriptionInitialPosition getSubscriptionInitialPosition(String value) { + SubscriptionInitialPosition type = SubscriptionInitialPosition.Latest; + for (SubscriptionInitialPosition s : SubscriptionInitialPosition.values()) { + if (s.name().equalsIgnoreCase(value)) { + type = s; + } + } + return type; + } + + private static ConsumerCryptoFailureAction getCryptoFailureAction(String value) { + ConsumerCryptoFailureAction type = ConsumerCryptoFailureAction.FAIL; + for (ConsumerCryptoFailureAction s : ConsumerCryptoFailureAction.values()) { + if (s.name().equalsIgnoreCase(value)) { + type = s; + } + } + return type; + } + + private static Optional createDeadLetterPolicy(BitSailConfiguration configuration) { + if (configuration.fieldExists(PULSAR_MAX_REDELIVER_COUNT.key()) + || configuration.fieldExists(PULSAR_RETRY_LETTER_TOPIC.key()) + || configuration.fieldExists(PULSAR_DEAD_LETTER_TOPIC.key())) { + DeadLetterPolicy.DeadLetterPolicyBuilder builder = DeadLetterPolicy.builder(); + + setOptionValue(configuration, PULSAR_MAX_REDELIVER_COUNT, builder::maxRedeliverCount); + setOptionValue(configuration, PULSAR_RETRY_LETTER_TOPIC, builder::retryLetterTopic); + setOptionValue(configuration, PULSAR_DEAD_LETTER_TOPIC, builder::deadLetterTopic); + + return Optional.of(builder.build()); + } else { + return Optional.empty(); + } + } + + private static ProxyProtocol getProxyProtocol(String value) { + ProxyProtocol res = ProxyProtocol.SNI; + for (ProxyProtocol test : ProxyProtocol.values()) { + if (test.name().equalsIgnoreCase(value)) { + res = test; + } + } + return res; + } + + /** + * Create the {@link Authentication} instance for both {@code PulsarClient} and {@code + * PulsarAdmin}. If the user didn't provide configuration, a {@link AuthenticationDisabled} + * instance would be returned. + * + *

This method behavior is the same as the pulsar command line tools. + */ + private static Authentication createAuthentication(BitSailConfiguration configuration) { + if (configuration.fieldExists(PULSAR_AUTH_PLUGIN_CLASS_NAME.key())) { + String authPluginClassName = configuration.getString(PULSAR_AUTH_PLUGIN_CLASS_NAME.key()); + + if (configuration.fieldExists(PULSAR_AUTH_PARAMS.key())) { + String authParamsString = configuration.getString(PULSAR_AUTH_PARAMS.key()); + return sneakyClient( + () -> AuthenticationFactory.create(authPluginClassName, authParamsString)); + } else if (configuration.fieldExists(PULSAR_AUTH_PARAM_MAP.key())) { + Map paramsMap = (Map) configuration.getMap(PULSAR_AUTH_PARAM_MAP.key()); + return sneakyClient( + () -> AuthenticationFactory.create(authPluginClassName, paramsMap)); + } else { + throw new IllegalArgumentException( + String.format( + "No %s or %s provided", + PULSAR_AUTH_PARAMS.key(), PULSAR_AUTH_PARAM_MAP.key())); + } + } + + return AuthenticationDisabled.INSTANCE; + } + + /** Get the option value str from given config, convert it into the real value instance. */ + public static T getOptionValue( + BitSailConfiguration configuration, ConfigOption option, Function convertor) { + F value = configuration.get(option); + if (value != null) { + return convertor.apply(value); + } else { + return null; + } + } + + /** Set the config option's value to a given builder. */ + public static void setOptionValue( + BitSailConfiguration configuration, ConfigOption option, Consumer setter) { + setOptionValue(configuration, option, identity(), setter); + } + + /** + * Query the config option's value, convert it into a required type, set it to a given builder. + */ + public static void setOptionValue( + BitSailConfiguration configuration, + ConfigOption option, + Function convertor, + Consumer setter) { + if (configuration.fieldExists(option.key())) { + V value = getOptionValue(configuration, option, convertor); + setter.accept(value); + } + } + + + public static boolean matchesRegexSubscriptionMode(String topic, RegexSubscriptionMode subscriptionMode) { + TopicName topicName = TopicName.get(topic); + // Filter the topic persistence. + switch (subscriptionMode) { + case PersistentOnly: + return topicName.isPersistent(); + case NonPersistentOnly: + return !topicName.isPersistent(); + default: + // RegexSubscriptionMode.AllTopics + return true; + } + } + + public static TopicMetadata queryTopicMetadata(PulsarAdmin pulsarAdmin, String topicName) { + // Drop the complete topic name for a clean partitioned topic name. + String completeTopicName = TopicNameUtils.topicName(topicName); + try { + PartitionedTopicMetadata metadata = + pulsarAdmin.topics().getPartitionedTopicMetadata(completeTopicName); + return new TopicMetadata(topicName, metadata.partitions); + } catch (PulsarAdminException e) { + if (e.getStatusCode() == 404) { + // Return null for skipping the topic metadata query. + return null; + } else { + // This method would cause the failure for subscriber. + throw new IllegalStateException(e); + } + } + } + + public static SubscriptionMode getSubscriptionMode(String value) { + SubscriptionMode subscriptionMode = SubscriptionMode.Durable; + for (SubscriptionMode mode : SubscriptionMode.values()) { + if (mode.name().equalsIgnoreCase(value)) { + subscriptionMode = mode; + } + } + return subscriptionMode; + } + public static RegexSubscriptionMode getRegexSubscriptionMode(String value) { + RegexSubscriptionMode subscriptionMode = RegexSubscriptionMode.AllTopics; + for (RegexSubscriptionMode mode : RegexSubscriptionMode.values()) { + if (mode.name().equalsIgnoreCase(value)) { + subscriptionMode = mode; + } + } + return subscriptionMode; + } + + public static List getTopicList(String value) { + List topicList = new LinkedList<>(); + for (String t: value.split(TOPIC_LIST_DELIMITER)) { + topicList.add(t.trim()); + } + return topicList; + } + + public static SubscriptionType getSubscriptionType(String value) { + SubscriptionType subscriptionType = SubscriptionType.Exclusive; + for (SubscriptionType s : SubscriptionType.values()) { + if (s.name().equalsIgnoreCase(value)) { + subscriptionType = s; + } + } + return subscriptionType; + } + + public static CursorVerification getCursorVerification(String value) { + CursorVerification type = CursorVerification.WARN_ON_MISMATCH; + for (CursorVerification s : CursorVerification.values()) { + if (s.name().equalsIgnoreCase(value)) { + type = s; + } + } + return type; + } + +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchema.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchema.java new file mode 100644 index 000000000..d20a53ace --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchema.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.connector.pulsar.common.schema; + +import org.apache.flink.annotation.Internal; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.schema.SchemaInfoImpl; +import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.common.schema.KeyValueEncodingType; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.createSchema; +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.encodeClassInfo; +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.haveProtobuf; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkState; +import static org.apache.flink.util.ReflectionUtil.getTemplateType1; +import static org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo.decodeKeyValueEncodingType; +import static org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo.decodeKeyValueSchemaInfo; +import static org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo.encodeKeyValueSchemaInfo; + +/** + * A wrapper for Pulsar {@link Schema}, make it serializable and can be created from {@link + * SchemaInfo}. + * + *

General pulsar schema info (avro, json, protobuf and keyvalue) don't contain the require class + * info. We have to urge user to provide the related type class and encoded it into schema info. + */ +@Internal +public final class PulsarSchema implements Serializable { + private static final long serialVersionUID = -2561088131419607555L; + + private transient Schema schema; + private transient SchemaInfo schemaInfo; + + /** Create serializable pulsar schema for primitive types. */ + public PulsarSchema(Schema schema) { + SchemaInfo info = schema.getSchemaInfo(); + SchemaType type = info.getType(); + checkArgument(type != SchemaType.JSON, "Json Schema should provide the type class"); + checkArgument(type != SchemaType.AVRO, "Avro Schema should provide the type class"); + checkArgument(type != SchemaType.PROTOBUF, "Protobuf Schema should provide the type class"); + checkArgument( + type != SchemaType.PROTOBUF_NATIVE, + "Protobuf Native Schema should provide the type class"); + checkArgument( + type != SchemaType.KEY_VALUE, + "Key Value Schema should provide the type class of key and value"); + // Primitive type information could be reflected from the schema class. + Class typeClass = getTemplateType1(schema.getClass()); + + this.schemaInfo = encodeClassInfo(info, typeClass); + this.schema = createSchema(schemaInfo); + } + + /** + * Create serializable pulsar schema for struct type or primitive types. + * + * @param schema The schema instance. + * @param typeClass The type class of this schema. + */ + public PulsarSchema(Schema schema, Class typeClass) { + SchemaInfo info = schema.getSchemaInfo(); + checkArgument( + info.getType() != SchemaType.KEY_VALUE, + "Key Value Schema should provide the type classes of key and value"); + validateSchemaInfo(info); + + this.schemaInfo = encodeClassInfo(info, typeClass); + this.schema = createSchema(schemaInfo); + } + + /** Create serializable pulsar schema for key value type. */ + public PulsarSchema( + Schema> kvSchema, Class keyClass, Class valueClass) { + SchemaInfo info = kvSchema.getSchemaInfo(); + checkArgument( + info.getType() == SchemaType.KEY_VALUE, + "This constructor could only be applied for KeyValueSchema"); + + KeyValue infoKeyValue = decodeKeyValueSchemaInfo(info); + + SchemaInfo infoKey = encodeClassInfo(infoKeyValue.getKey(), keyClass); + validateSchemaInfo(infoKey); + + SchemaInfo infoValue = encodeClassInfo(infoKeyValue.getValue(), valueClass); + validateSchemaInfo(infoValue); + + KeyValueEncodingType encodingType = decodeKeyValueEncodingType(info); + SchemaInfo encodedInfo = + encodeKeyValueSchemaInfo(info.getName(), infoKey, infoValue, encodingType); + + this.schemaInfo = encodeClassInfo(encodedInfo, KeyValue.class); + this.schema = createSchema(this.schemaInfo); + } + + public Schema getPulsarSchema() { + return schema; + } + + public SchemaInfo getSchemaInfo() { + return schemaInfo; + } + + public Class getRecordClass() { + return decodeClassInfo(schemaInfo); + } + + private void writeObject(ObjectOutputStream oos) throws IOException { + // Name + oos.writeUTF(schemaInfo.getName()); + + // Schema + byte[] schemaBytes = schemaInfo.getSchema(); + oos.writeInt(schemaBytes.length); + oos.write(schemaBytes); + + // Type + SchemaType type = schemaInfo.getType(); + oos.writeInt(type.getValue()); + + // Properties + Map properties = schemaInfo.getProperties(); + oos.writeInt(properties.size()); + for (Map.Entry entry : properties.entrySet()) { + oos.writeUTF(entry.getKey()); + oos.writeUTF(entry.getValue()); + } + } + + private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException { + // Name + String name = ois.readUTF(); + + // Schema + int byteLen = ois.readInt(); + byte[] schemaBytes = new byte[byteLen]; + int read = ois.read(schemaBytes); + checkState(read == byteLen); + + // Type + int typeIdx = ois.readInt(); + SchemaType type = SchemaType.valueOf(typeIdx); + + // Properties + int propSize = ois.readInt(); + Map properties = new HashMap<>(propSize); + for (int i = 0; i < propSize; i++) { + properties.put(ois.readUTF(), ois.readUTF()); + } + + this.schemaInfo = new SchemaInfoImpl(name, schemaBytes, type, properties); + this.schema = createSchema(schemaInfo); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SchemaInfo that = ((PulsarSchema) o).getPulsarSchema().getSchemaInfo(); + + return Objects.equals(schemaInfo.getType(), that.getType()) + && Arrays.equals(schemaInfo.getSchema(), that.getSchema()) + && Objects.equals(schemaInfo.getProperties(), that.getProperties()); + } + + @Override + public int hashCode() { + return Objects.hash( + schemaInfo.getType(), + Arrays.hashCode(schemaInfo.getSchema()), + schemaInfo.getProperties()); + } + + @Override + public String toString() { + return schemaInfo.toString(); + } + + /** + * We would throw exception if schema type is protobuf and user don't provide protobuf-java in + * class path. + */ + private void validateSchemaInfo(SchemaInfo info) { + SchemaType type = info.getType(); + if (type == SchemaType.PROTOBUF || type == SchemaType.PROTOBUF_NATIVE) { + checkState( + haveProtobuf(), "protobuf-java should be provided if you use related schema."); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaFactory.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaFactory.java new file mode 100644 index 000000000..4a2d7fc1b --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaFactory.java @@ -0,0 +1,43 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.common.schema; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; + +/** + * The schema factory for a specified {@link SchemaType}. We add this factory because of pulsar + * don't provide a serializable schema and we can't create it directly from {@link SchemaInfo}. So + * we have to implement this creation logic. + */ +@Internal +public interface PulsarSchemaFactory { + + /** The supported schema type for this factory. We would classify the factory by the type. */ + SchemaType type(); + + /** Create the schema by the given info. */ + Schema createSchema(SchemaInfo info); + + /** Create the flink type information by the given schema info. */ + TypeInformation createTypeInfo(SchemaInfo info); +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaTypeInformation.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaTypeInformation.java new file mode 100644 index 000000000..ba91df28d --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaTypeInformation.java @@ -0,0 +1,98 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.common.schema; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +import java.util.Objects; + +/** Wrap the pulsar {@code Schema} into a flink {@code TypeInformation}. */ +@Internal +public class PulsarSchemaTypeInformation extends TypeInformation { + private static final long serialVersionUID = 7284667318651333519L; + + private final PulsarSchema schema; + + public PulsarSchemaTypeInformation(PulsarSchema schema) { + this.schema = schema; + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 1; + } + + @Override + public int getTotalFields() { + return 1; + } + + @Override + public Class getTypeClass() { + return schema.getRecordClass(); + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer createSerializer(ExecutionConfig config) { + return new PulsarSchemaTypeSerializer<>(schema); + } + + @Override + public String toString() { + return schema.toString(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof PulsarSchemaTypeInformation) { + PulsarSchemaTypeInformation that = (PulsarSchemaTypeInformation) obj; + return Objects.equals(schema, that.schema); + } else { + return false; + } + } + + @Override + public int hashCode() { + return schema.hashCode(); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof PulsarSchemaTypeInformation; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaTypeSerializer.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaTypeSerializer.java new file mode 100644 index 000000000..1e98ac7a3 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaTypeSerializer.java @@ -0,0 +1,203 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.common.schema; + +import com.google.protobuf.Message; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.InstantiationUtil; +import org.apache.pulsar.client.api.Schema; + +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Objects; + +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.haveProtobuf; +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.isProtobufTypeClass; +import static org.apache.flink.util.Preconditions.checkState; + +/** Wrap the pulsar {@code Schema} into a flink {@code TypeSerializer}. */ +@Internal +public class PulsarSchemaTypeSerializer extends TypeSerializer { + private static final long serialVersionUID = 7771153330969433085L; + + private final PulsarSchema schema; + + public PulsarSchemaTypeSerializer(PulsarSchema schema) { + this.schema = schema; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return this; + } + + @Override + @SuppressWarnings("unchecked") + public T createInstance() { + Class recordClass = schema.getRecordClass(); + + // No exception wouldn't be thrown here if user don't provide protobuf-java. + if (haveProtobuf() && isProtobufTypeClass(recordClass)) { + try { + Method newBuilderMethod = recordClass.getMethod("newBuilder"); + Message.Builder builder = (Message.Builder) newBuilderMethod.invoke(null); + return (T) builder.build(); + } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { + throw new IllegalStateException(e); + } + } else { + return InstantiationUtil.instantiate(recordClass); + } + } + + @Override + public T copy(T from) { + return from; + } + + @Override + public T copy(T from, T reuse) { + return from; + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(T record, DataOutputView target) throws IOException { + Schema pulsarSchema = schema.getPulsarSchema(); + byte[] bytes = pulsarSchema.encode(record); + + target.writeInt(bytes.length); + target.write(bytes); + } + + @Override + public T deserialize(DataInputView source) throws IOException { + int len = source.readInt(); + byte[] bytes = new byte[len]; + int readLen = source.read(bytes); + checkState(len == readLen); + + Schema pulsarSchema = schema.getPulsarSchema(); + return pulsarSchema.decode(bytes); + } + + @Override + public T deserialize(T reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + int len = source.readInt(); + byte[] bytes = new byte[len]; + int readLen = source.read(bytes); + checkState(len == readLen); + + target.writeInt(bytes.length); + target.write(bytes); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof PulsarSchemaTypeSerializer) { + PulsarSchemaTypeSerializer that = (PulsarSchemaTypeSerializer) obj; + return Objects.equals(schema, that.schema); + } + + return false; + } + + @Override + public int hashCode() { + return schema.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new PulsarSchemaTypeSerializerSnapshot<>(schema); + } + + /** + * Snapshot for PulsarSchemaTypeSerializer, we only snapshot the SerializablePulsarSchema into + * the state. + */ + public static final class PulsarSchemaTypeSerializerSnapshot + implements TypeSerializerSnapshot { + + private PulsarSchema schema; + + public PulsarSchemaTypeSerializerSnapshot(PulsarSchema schema) { + this.schema = schema; + } + + @Override + public int getCurrentVersion() { + return 1; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + byte[] bytes = InstantiationUtil.serializeObject(schema); + out.writeInt(bytes.length); + out.write(bytes); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + int len = in.readInt(); + byte[] bytes = new byte[len]; + int readLen = in.read(bytes); + checkState(readLen == len); + + try { + ClassLoader loader = Thread.currentThread().getContextClassLoader(); + this.schema = InstantiationUtil.deserializeObject(bytes, loader); + } catch (ClassNotFoundException e) { + throw new IOException(e); + } + } + + @Override + public TypeSerializer restoreSerializer() { + return new PulsarSchemaTypeSerializer<>(schema); + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaUtils.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaUtils.java new file mode 100644 index 000000000..2c62799c8 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/PulsarSchemaUtils.java @@ -0,0 +1,201 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.common.schema; + +import com.google.protobuf.Message; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import com.bytedance.bitsail.connector.pulsar.common.schema.factories.AvroSchemaFactory; +import com.bytedance.bitsail.connector.pulsar.common.schema.factories.JSONSchemaFactory; +import com.bytedance.bitsail.connector.pulsar.common.schema.factories.KeyValueSchemaFactory; +import com.bytedance.bitsail.connector.pulsar.common.schema.factories.PrimitiveSchemaFactory; +import com.bytedance.bitsail.connector.pulsar.common.schema.factories.ProtobufNativeSchemaFactory; +import com.bytedance.bitsail.connector.pulsar.common.schema.factories.ProtobufSchemaFactory; +import com.bytedance.bitsail.connector.pulsar.common.schema.factories.StringSchemaFactory; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.schema.BooleanSchema; +import org.apache.pulsar.client.impl.schema.ByteSchema; +import org.apache.pulsar.client.impl.schema.BytesSchema; +import org.apache.pulsar.client.impl.schema.DateSchema; +import org.apache.pulsar.client.impl.schema.DoubleSchema; +import org.apache.pulsar.client.impl.schema.FloatSchema; +import org.apache.pulsar.client.impl.schema.InstantSchema; +import org.apache.pulsar.client.impl.schema.IntSchema; +import org.apache.pulsar.client.impl.schema.LocalDateSchema; +import org.apache.pulsar.client.impl.schema.LocalDateTimeSchema; +import org.apache.pulsar.client.impl.schema.LocalTimeSchema; +import org.apache.pulsar.client.impl.schema.LongSchema; +import org.apache.pulsar.client.impl.schema.SchemaInfoImpl; +import org.apache.pulsar.client.impl.schema.ShortSchema; +import org.apache.pulsar.client.impl.schema.TimeSchema; +import org.apache.pulsar.client.impl.schema.TimestampSchema; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; + +import java.util.EnumMap; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.DATE_TYPE_INFO; +import static org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO; +import static org.apache.flink.api.common.typeinfo.Types.BOOLEAN; +import static org.apache.flink.api.common.typeinfo.Types.BYTE; +import static org.apache.flink.api.common.typeinfo.Types.DOUBLE; +import static org.apache.flink.api.common.typeinfo.Types.FLOAT; +import static org.apache.flink.api.common.typeinfo.Types.INSTANT; +import static org.apache.flink.api.common.typeinfo.Types.INT; +import static org.apache.flink.api.common.typeinfo.Types.LOCAL_DATE; +import static org.apache.flink.api.common.typeinfo.Types.LOCAL_DATE_TIME; +import static org.apache.flink.api.common.typeinfo.Types.LOCAL_TIME; +import static org.apache.flink.api.common.typeinfo.Types.LONG; +import static org.apache.flink.api.common.typeinfo.Types.SHORT; +import static org.apache.flink.api.common.typeinfo.Types.SQL_TIME; +import static org.apache.flink.api.common.typeinfo.Types.SQL_TIMESTAMP; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Util class for pulsar schema. Register all the {@link PulsarSchemaFactory} in this class and + * provide the {@link TypeInformation} or {@link PulsarSchema} conversion. + */ +@Internal +@SuppressWarnings("unchecked") +public final class PulsarSchemaUtils { + + private static final Class PROTOBUF_MESSAGE_CLASS; + // Predefined pulsar schema factory. + private static final Map> FACTORY_REGISTER = + new EnumMap<>(SchemaType.class); + + public static final String CLASS_INFO_PLACEHOLDER = "INTERNAL.pulsar.schema.type.class.name"; + + static { + // Initialize protobuf message class. + Class messageClass; + try { + messageClass = (Class) Class.forName("com.google.protobuf.Message"); + } catch (ClassNotFoundException e) { + messageClass = null; + } + PROTOBUF_MESSAGE_CLASS = messageClass; + + // Struct schemas + registerSchemaFactory(new AvroSchemaFactory<>()); + registerSchemaFactory(new JSONSchemaFactory<>()); + registerSchemaFactory(new KeyValueSchemaFactory<>()); + if (haveProtobuf()) { + // Protobuf type should be supported only when we have protobuf-java. + registerSchemaFactory(new ProtobufNativeSchemaFactory<>()); + registerSchemaFactory(new ProtobufSchemaFactory<>()); + } + + // Primitive schemas + registerSchemaFactory(new StringSchemaFactory()); + registerSchemaFactory( + new PrimitiveSchemaFactory<>( + SchemaType.NONE, BytesSchema.of(), BYTE_PRIMITIVE_ARRAY_TYPE_INFO)); + registerPrimitiveFactory(BooleanSchema.of(), BOOLEAN); + registerPrimitiveFactory(ByteSchema.of(), BYTE); + registerPrimitiveFactory(BytesSchema.of(), BYTE_PRIMITIVE_ARRAY_TYPE_INFO); + registerPrimitiveFactory(DateSchema.of(), DATE_TYPE_INFO); + registerPrimitiveFactory(DoubleSchema.of(), DOUBLE); + registerPrimitiveFactory(FloatSchema.of(), FLOAT); + registerPrimitiveFactory(InstantSchema.of(), INSTANT); + registerPrimitiveFactory(IntSchema.of(), INT); + registerPrimitiveFactory(LocalDateSchema.of(), LOCAL_DATE); + registerPrimitiveFactory(LocalDateTimeSchema.of(), LOCAL_DATE_TIME); + registerPrimitiveFactory(LocalTimeSchema.of(), LOCAL_TIME); + registerPrimitiveFactory(LongSchema.of(), LONG); + registerPrimitiveFactory(ShortSchema.of(), SHORT); + registerPrimitiveFactory(TimeSchema.of(), SQL_TIME); + registerPrimitiveFactory(TimestampSchema.of(), SQL_TIMESTAMP); + } + + private PulsarSchemaUtils() { + // No need to create instance. + } + + /** A boolean value for determine if user have protobuf-java in his class path. */ + public static boolean haveProtobuf() { + return PROTOBUF_MESSAGE_CLASS != null; + } + + /** + * Check if the given class is a protobuf generated class. Since this class would throw NP + * exception when you don't provide protobuf-java, use this method with {@link #haveProtobuf()} + */ + public static boolean isProtobufTypeClass(Class clazz) { + return checkNotNull(PROTOBUF_MESSAGE_CLASS).isAssignableFrom(clazz); + } + + private static void registerPrimitiveFactory( + Schema schema, TypeInformation information) { + registerSchemaFactory(new PrimitiveSchemaFactory<>(schema, information)); + } + + private static void registerSchemaFactory(PulsarSchemaFactory factory) { + FACTORY_REGISTER.put(factory.type(), factory); + } + + /** + * Pulsar has a hugh set of built-in schemas. We can create them by the given {@link + * SchemaInfo}. This schema info is a wrapped info created by {@link PulsarSchema}. + */ + @SuppressWarnings("unchecked") + public static Schema createSchema(SchemaInfo info) { + PulsarSchemaFactory factory = FACTORY_REGISTER.get(info.getType()); + checkNotNull(factory, "This schema info %s is not supported.", info); + + return (Schema) factory.createSchema(info); + } + + /** + * Convert the {@link SchemaInfo} into a flink manageable {@link TypeInformation}. This schema + * info is a wrapped info created by {@link PulsarSchema}. + */ + @SuppressWarnings("unchecked") + public static TypeInformation createTypeInformation(SchemaInfo info) { + PulsarSchemaFactory factory = FACTORY_REGISTER.get(info.getType()); + checkNotNull(factory, "This schema info %s is not supported.", info); + + return (TypeInformation) factory.createTypeInfo(info); + } + + public static SchemaInfo encodeClassInfo(SchemaInfo schemaInfo, Class typeClass) { + Map properties = new HashMap<>(schemaInfo.getProperties()); + properties.put(CLASS_INFO_PLACEHOLDER, typeClass.getName()); + + return new SchemaInfoImpl( + schemaInfo.getName(), schemaInfo.getSchema(), schemaInfo.getType(), properties); + } + + @SuppressWarnings("unchecked") + public static Class decodeClassInfo(SchemaInfo schemaInfo) { + Map properties = schemaInfo.getProperties(); + String className = + checkNotNull( + properties.get(CLASS_INFO_PLACEHOLDER), + "This schema don't contain a class name."); + + try { + return (Class) Class.forName(className); + } catch (ClassNotFoundException e) { + throw new IllegalStateException("Couldn't find the schema class " + className); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/AvroSchemaFactory.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/AvroSchemaFactory.java new file mode 100644 index 000000000..d77fef9c5 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/AvroSchemaFactory.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.connector.pulsar.common.schema.factories; + +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.schema.SchemaDefinition; +import org.apache.pulsar.client.impl.schema.AvroSchema; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; + +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; + +/** The schema factory for pulsar's {@link AvroSchema}. */ +public class AvroSchemaFactory extends BaseStructSchemaFactory { + + @Override + public SchemaType type() { + return SchemaType.AVRO; + } + + @Override + public Schema createSchema(SchemaInfo info) { + Class typeClass = decodeClassInfo(info); + SchemaDefinition definition = + SchemaDefinition.builder() + .withPojo(typeClass) + .withProperties(info.getProperties()) + .build(); + + return AvroSchema.of(definition); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/BaseStructSchemaFactory.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/BaseStructSchemaFactory.java new file mode 100644 index 000000000..d472f7b4a --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/BaseStructSchemaFactory.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 com.bytedance.bitsail.connector.pulsar.common.schema.factories; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchema; +import com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaFactory; +import com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaTypeInformation; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.schema.SchemaInfo; + +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; + +/** Implement the common createTypeInfo method for all struct schema factory. */ +public abstract class BaseStructSchemaFactory implements PulsarSchemaFactory { + + @Override + public TypeInformation createTypeInfo(SchemaInfo info) { + Schema pulsarSchema = createSchema(info); + Class typeClass = decodeClassInfo(info); + PulsarSchema schema = new PulsarSchema<>(pulsarSchema, typeClass); + + return new PulsarSchemaTypeInformation<>(schema); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/JSONSchemaFactory.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/JSONSchemaFactory.java new file mode 100644 index 000000000..1c94500d5 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/JSONSchemaFactory.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 com.bytedance.bitsail.connector.pulsar.common.schema.factories; + +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.schema.JSONSchema; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; + +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; + +/** The schema factory for pulsar's {@link JSONSchema}. */ +public class JSONSchemaFactory extends BaseStructSchemaFactory { + + @Override + public SchemaType type() { + return SchemaType.JSON; + } + + @Override + public Schema createSchema(SchemaInfo info) { + Class typeClass = decodeClassInfo(info); + return JSONSchema.of(typeClass, info.getProperties()); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java new file mode 100644 index 000000000..0ed418ee1 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.connector.pulsar.common.schema.factories; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchema; +import com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaFactory; +import com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaTypeInformation; +import com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.schema.KeyValueSchemaImpl; +import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.common.schema.KeyValueEncodingType; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; + +import java.util.Map; + +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.CLASS_INFO_PLACEHOLDER; +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; +import static org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo.decodeKeyValueEncodingType; +import static org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo.decodeKeyValueSchemaInfo; + +/** The schema factory for pulsar's {@link KeyValueSchemaImpl}. */ +public class KeyValueSchemaFactory implements PulsarSchemaFactory> { + + @Override + public SchemaType type() { + return SchemaType.KEY_VALUE; + } + + @Override + public Schema> createSchema(SchemaInfo info) { + KeyValue kvSchemaInfo = decodeKeyValueSchemaInfo(info); + + Schema keySchema = PulsarSchemaUtils.createSchema(kvSchemaInfo.getKey()); + Schema valueSchema = PulsarSchemaUtils.createSchema(kvSchemaInfo.getValue()); + KeyValueEncodingType encodingType = decodeKeyValueEncodingType(info); + + Schema> schema = KeyValueSchemaImpl.of(keySchema, valueSchema, encodingType); + + // Append extra class name into schema info properties. + // KeyValueSchema don't have custom properties builder method, we have to use side effects. + SchemaInfo schemaInfo = schema.getSchemaInfo(); + Map properties = schemaInfo.getProperties(); + properties.put(CLASS_INFO_PLACEHOLDER, KeyValue.class.getName()); + + return schema; + } + + @Override + public TypeInformation> createTypeInfo(SchemaInfo info) { + KeyValue kvSchemaInfo = decodeKeyValueSchemaInfo(info); + + Schema keySchema = PulsarSchemaUtils.createSchema(kvSchemaInfo.getKey()); + Class keyClass = decodeClassInfo(keySchema.getSchemaInfo()); + + Schema valueSchema = PulsarSchemaUtils.createSchema(kvSchemaInfo.getValue()); + Class valueClass = decodeClassInfo(valueSchema.getSchemaInfo()); + + Schema> schema = createSchema(info); + PulsarSchema> pulsarSchema = + new PulsarSchema<>(schema, keyClass, valueClass); + + return new PulsarSchemaTypeInformation<>(pulsarSchema); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/PrimitiveSchemaFactory.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/PrimitiveSchemaFactory.java new file mode 100644 index 000000000..b23f4ed52 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/PrimitiveSchemaFactory.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 com.bytedance.bitsail.connector.pulsar.common.schema.factories; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaFactory; +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableSet; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * The schema factory for pulsar's primitive types. Currently, Pulsar supports these primitive + * types. + */ +public class PrimitiveSchemaFactory implements PulsarSchemaFactory { + + private static final ImmutableSet PRIMITIVE_SCHEMA_TYPES = + ImmutableSet.builder() + .add(SchemaType.NONE) + .add(SchemaType.BOOLEAN) + .add(SchemaType.INT8) + .add(SchemaType.INT16) + .add(SchemaType.INT32) + .add(SchemaType.INT64) + .add(SchemaType.FLOAT) + .add(SchemaType.DOUBLE) + .add(SchemaType.BYTES) + .add(SchemaType.STRING) + .add(SchemaType.TIMESTAMP) + .add(SchemaType.TIME) + .add(SchemaType.DATE) + .add(SchemaType.INSTANT) + .add(SchemaType.LOCAL_DATE) + .add(SchemaType.LOCAL_TIME) + .add(SchemaType.LOCAL_DATE_TIME) + .build(); + + private final SchemaType type; + private final Schema schema; + private final TypeInformation typeInformation; + + public PrimitiveSchemaFactory(Schema schema, TypeInformation typeInformation) { + this(schema.getSchemaInfo().getType(), schema, typeInformation); + } + + public PrimitiveSchemaFactory( + SchemaType type, Schema schema, TypeInformation typeInformation) { + checkArgument(PRIMITIVE_SCHEMA_TYPES.contains(type)); + + this.type = type; + this.schema = schema; + this.typeInformation = typeInformation; + } + + @Override + public SchemaType type() { + return type; + } + + @Override + public Schema createSchema(SchemaInfo info) { + return schema; + } + + @Override + public TypeInformation createTypeInfo(SchemaInfo info) { + return typeInformation; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/ProtobufNativeSchemaFactory.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/ProtobufNativeSchemaFactory.java new file mode 100644 index 000000000..5ec56a9cc --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/ProtobufNativeSchemaFactory.java @@ -0,0 +1,43 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.common.schema.factories; + +import com.google.protobuf.GeneratedMessageV3; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.schema.ProtobufNativeSchema; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; + +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; + +/** The schema factory for pulsar's {@link ProtobufNativeSchema}. */ +public class ProtobufNativeSchemaFactory + extends BaseStructSchemaFactory { + + @Override + public SchemaType type() { + return SchemaType.PROTOBUF_NATIVE; + } + + @Override + public Schema createSchema(SchemaInfo info) { + Class typeClass = decodeClassInfo(info); + return ProtobufNativeSchema.of(typeClass, info.getProperties()); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/ProtobufSchemaFactory.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/ProtobufSchemaFactory.java new file mode 100644 index 000000000..1d39f6387 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/ProtobufSchemaFactory.java @@ -0,0 +1,43 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.common.schema.factories; + +import com.google.protobuf.GeneratedMessageV3; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.schema.ProtobufSchema; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; + +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; + +/** The schema factory for pulsar's {@link ProtobufSchema}. */ +public class ProtobufSchemaFactory + extends BaseStructSchemaFactory { + + @Override + public SchemaType type() { + return SchemaType.PROTOBUF; + } + + @Override + public Schema createSchema(SchemaInfo info) { + Class typeClass = decodeClassInfo(info); + return ProtobufSchema.of(typeClass, info.getProperties()); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/StringSchemaFactory.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/StringSchemaFactory.java new file mode 100644 index 000000000..408dcb27d --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/schema/factories/StringSchemaFactory.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.connector.pulsar.common.schema.factories; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaFactory; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.schema.StringSchema; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; + +/** The schema factory for pulsar's {@link StringSchema}. */ +public class StringSchemaFactory implements PulsarSchemaFactory { + + @Override + public SchemaType type() { + return SchemaType.STRING; + } + + @Override + public Schema createSchema(SchemaInfo info) { + // SchemaInfo contains the string encode type. + return StringSchema.fromSchemaInfo(info); + } + + @Override + public TypeInformation createTypeInfo(SchemaInfo info) { + return Types.STRING; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/utils/PulsarExceptionUtils.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/utils/PulsarExceptionUtils.java new file mode 100644 index 000000000..5026fb02d --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/utils/PulsarExceptionUtils.java @@ -0,0 +1,82 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.common.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.function.SupplierWithException; +import org.apache.flink.util.function.ThrowingRunnable; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.PulsarClientException; + +/** + * Util class for pulsar checked exceptions. Sneaky throw {@link PulsarAdminException} and {@link + * PulsarClientException}. + */ +@Internal +public final class PulsarExceptionUtils { + + private PulsarExceptionUtils() { + // No public constructor. + } + + public static void sneakyClient( + ThrowingRunnable runnable) { + sneaky(runnable); + } + + public static T sneakyClient( + SupplierWithException supplier) { + return sneaky(supplier); + } + + public static void sneakyAdmin(ThrowingRunnable runnable) { + sneaky(runnable); + } + + public static T sneakyAdmin( + SupplierWithException supplier) { + return sneaky(supplier); + } + + private static void sneaky(ThrowingRunnable runnable) { + try { + runnable.run(); + } catch (Exception r) { + sneakyThrow(r); + } + } + + /** Catch the throwable exception and rethrow it without try catch. */ + private static T sneaky(SupplierWithException supplier) { + try { + return supplier.get(); + } catch (Exception r) { + sneakyThrow(r); + } + + // This method wouldn't be executed. + throw new RuntimeException("Never throw here."); + } + + /** javac hack for unchecking the checked exception. */ + @SuppressWarnings("unchecked") + public static void sneakyThrow(Exception t) throws T { + throw (T) t; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/utils/PulsarSerdeUtils.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/utils/PulsarSerdeUtils.java new file mode 100644 index 000000000..f308a98d9 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/utils/PulsarSerdeUtils.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 com.bytedance.bitsail.connector.pulsar.common.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.function.BiConsumerWithException; +import org.apache.flink.util.function.FunctionWithException; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** Util for serialize and deserialize. */ +@Internal +public final class PulsarSerdeUtils { + + private PulsarSerdeUtils() { + // No public constructor. + } + + // Bytes serialization. + + public static void serializeBytes(DataOutputStream out, byte[] bytes) throws IOException { + out.writeInt(bytes.length); + out.write(bytes); + } + + public static byte[] deserializeBytes(DataInputStream in) throws IOException { + int size = in.readInt(); + byte[] bytes = new byte[size]; + int result = in.read(bytes); + if (result < 0) { + throw new IOException("Couldn't deserialize the object, wrong byte buffer."); + } + + return bytes; + } + + // Common Object serialization. + + public static void serializeObject(DataOutputStream out, Object obj) throws IOException { + Preconditions.checkNotNull(obj); + + byte[] objectBytes = InstantiationUtil.serializeObject(obj); + serializeBytes(out, objectBytes); + } + + public static T deserializeObject(DataInputStream in) throws IOException { + byte[] objectBytes = deserializeBytes(in); + ClassLoader loader = Thread.currentThread().getContextClassLoader(); + + try { + return InstantiationUtil.deserializeObject(objectBytes, loader); + } catch (ClassNotFoundException e) { + throw new IOException(e); + } + } + + // Common Set serialization. + + public static void serializeSet( + DataOutputStream out, + Set set, + BiConsumerWithException serializer) + throws IOException { + out.writeInt(set.size()); + for (T t : set) { + serializer.accept(out, t); + } + } + + public static Set deserializeSet( + DataInputStream in, FunctionWithException deserializer) + throws IOException { + int size = in.readInt(); + Set set = new HashSet<>(size); + for (int i = 0; i < size; i++) { + T t = deserializer.apply(in); + set.add(t); + } + + return set; + } + + // Common Map serialization. + + public static void serializeMap( + DataOutputStream out, + Map map, + BiConsumerWithException keySerializer, + BiConsumerWithException valueSerializer) + throws IOException { + out.writeInt(map.size()); + for (Map.Entry entry : map.entrySet()) { + keySerializer.accept(out, entry.getKey()); + valueSerializer.accept(out, entry.getValue()); + } + } + + public static Map deserializeMap( + DataInputStream in, + FunctionWithException keyDeserializer, + FunctionWithException valueDeserializer) + throws IOException { + int size = in.readInt(); + Map result = new HashMap<>(size); + for (int i = 0; i < size; i++) { + K key = keyDeserializer.apply(in); + V value = valueDeserializer.apply(in); + result.put(key, value); + } + return result; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/utils/PulsarTransactionUtils.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/utils/PulsarTransactionUtils.java new file mode 100644 index 000000000..26913456a --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/common/utils/PulsarTransactionUtils.java @@ -0,0 +1,117 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.common.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.client.api.transaction.TxnID; +import org.apache.pulsar.client.impl.transaction.TransactionImpl; + +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Transaction was introduced into pulsar since 2.7.0, but the interface {@link Transaction} didn't + * provide a id method until 2.8.1. We have to add this util for acquiring the {@link TxnID} for + * compatible consideration. + * + *

TODO Remove this hack after pulsar 2.8.1 release. + */ +@Internal +@SuppressWarnings("java:S3011") +public final class PulsarTransactionUtils { + + private static volatile Field mostBitsField; + private static volatile Field leastBitsField; + + private PulsarTransactionUtils() { + // No public constructor + } + + public static TxnID getId(Transaction transaction) { + // 2.8.1 and after. + try { + Method getId = Transaction.class.getDeclaredMethod("getTxnID"); + return (TxnID) getId.invoke(transaction); + } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { + // 2.8.0 and before. + TransactionImpl impl = (TransactionImpl) transaction; + Long txnIdMostBits = getTxnIdMostBits(impl); + Long txnIdLeastBits = getTxnIdLeastBits(impl); + + checkNotNull(txnIdMostBits, "Failed to get txnIdMostBits"); + checkNotNull(txnIdLeastBits, "Failed to get txnIdLeastBits"); + + return new TxnID(txnIdMostBits, txnIdLeastBits); + } + } + + private static Long getTxnIdMostBits(TransactionImpl transaction) { + if (mostBitsField == null) { + synchronized (PulsarTransactionUtils.class) { + if (mostBitsField == null) { + try { + mostBitsField = TransactionImpl.class.getDeclaredField("txnIdMostBits"); + mostBitsField.setAccessible(true); + } catch (NoSuchFieldException e) { + // Nothing to do for this exception. + } + } + } + } + + if (mostBitsField != null) { + try { + return (Long) mostBitsField.get(transaction); + } catch (IllegalAccessException e) { + // Nothing to do for this exception. + } + } + + return null; + } + + private static Long getTxnIdLeastBits(TransactionImpl transaction) { + if (leastBitsField == null) { + synchronized (PulsarTransactionUtils.class) { + if (leastBitsField == null) { + try { + leastBitsField = TransactionImpl.class.getDeclaredField("txnIdLeastBits"); + leastBitsField.setAccessible(true); + } catch (NoSuchFieldException e) { + // Nothing to do for this exception. + } + } + } + } + + if (leastBitsField != null) { + try { + return (Long) leastBitsField.get(transaction); + } catch (IllegalAccessException e) { + // Nothing to do for this exception. + } + } + + return null; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSource.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSource.java new file mode 100644 index 000000000..96d743491 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSource.java @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.connector.pulsar.source; + +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.PulsarSourceEnumStateSerializer; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.PulsarSourceEnumerator; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.SplitsAssignmentState; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StartCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range.RangeGenerator; +import com.bytedance.bitsail.connector.pulsar.source.reader.PulsarSourceReaderFactory; +import com.bytedance.bitsail.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import com.bytedance.bitsail.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchemaInitializationContext; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplitSerializer; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +/** + * The Source implementation of Pulsar. Please use a {@link PulsarSourceBuilder} to construct a + * {@link com.bytedance.bitsail.connector.pulsar.source.PulsarSource}. The following example shows how to create a PulsarSource emitting records + * of String type. + * + *

{@code
+ * PulsarSource source = PulsarSource
+ *     .builder()
+ *     .setTopics(TOPIC1, TOPIC2)
+ *     .setServiceUrl(getServiceUrl())
+ *     .setAdminUrl(getAdminUrl())
+ *     .setSubscriptionName("test")
+ *     .setDeserializationSchema(PulsarDeserializationSchema.flinkSchema(new SimpleStringSchema()))
+ *     .setBounded(StopCursor::defaultStopCursor)
+ *     .build();
+ * }
+ * + *

See {@link PulsarSourceBuilder} for more details. + * + * @param The output type of the source. + */ +@PublicEvolving +public final class PulsarSource + implements Source, + ResultTypeQueryable { + private static final long serialVersionUID = 7773108631275567433L; + + /** + * The common configuration for pulsar source, we don't support the pulsar's configuration class + * directly. + */ + private final Configuration configuration; + + private final SourceConfiguration sourceConfiguration; + + private final PulsarSubscriber subscriber; + + private final RangeGenerator rangeGenerator; + + private final StartCursor startCursor; + + private final StopCursor stopCursor; + + private final Boundedness boundedness; + + /** The pulsar deserialization schema used for deserializing message. */ + private final PulsarDeserializationSchema deserializationSchema; + + /** + * The constructor for PulsarSource, it's package protected for forcing using {@link + * PulsarSourceBuilder}. + */ + PulsarSource( + Configuration configuration, + PulsarSubscriber subscriber, + RangeGenerator rangeGenerator, + StartCursor startCursor, + StopCursor stopCursor, + Boundedness boundedness, + PulsarDeserializationSchema deserializationSchema) { + + this.configuration = configuration; + this.sourceConfiguration = new SourceConfiguration(configuration); + this.subscriber = subscriber; + this.rangeGenerator = rangeGenerator; + this.startCursor = startCursor; + this.stopCursor = stopCursor; + this.boundedness = boundedness; + this.deserializationSchema = deserializationSchema; + } + + /** + * Get a PulsarSourceBuilder to builder a {@link com.bytedance.bitsail.connector.pulsar.source.PulsarSource}. + * + * @return a Pulsar source builder. + */ + @SuppressWarnings("java:S4977") + public static PulsarSourceBuilder builder() { + return new PulsarSourceBuilder<>(); + } + + @Override + public Boundedness getBoundedness() { + return boundedness; + } + + @Override + public SourceReader createReader(SourceReaderContext readerContext) + throws Exception { + // Initialize the deserialization schema before creating the pulsar reader. + deserializationSchema.open( + new PulsarDeserializationSchemaInitializationContext(readerContext)); + + return PulsarSourceReaderFactory.create( + readerContext, deserializationSchema, configuration, sourceConfiguration); + } + + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + SplitsAssignmentState assignmentState = + new SplitsAssignmentState(stopCursor, sourceConfiguration); + return new PulsarSourceEnumerator( + subscriber, + startCursor, + rangeGenerator, + configuration, + sourceConfiguration, + enumContext, + assignmentState); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, + PulsarSourceEnumState checkpoint) { + SplitsAssignmentState assignmentState = + new SplitsAssignmentState(stopCursor, sourceConfiguration, checkpoint); + return new PulsarSourceEnumerator( + subscriber, + startCursor, + rangeGenerator, + configuration, + sourceConfiguration, + enumContext, + assignmentState); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return PulsarPartitionSplitSerializer.INSTANCE; + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return PulsarSourceEnumStateSerializer.INSTANCE; + } + + @Override + public TypeInformation getProducedType() { + return deserializationSchema.getProducedType(); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceBuilder.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceBuilder.java new file mode 100644 index 000000000..4d7a25379 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceBuilder.java @@ -0,0 +1,518 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source; + +import com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StartCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range.FullRangeGenerator; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range.RangeGenerator; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range.UniformRangeGenerator; +import com.bytedance.bitsail.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.UnmodifiableConfiguration; +import org.apache.pulsar.client.api.RegexSubscriptionMode; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.regex.Pattern; + +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_ENABLE_TRANSACTION; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS; +import static com.bytedance.bitsail.connector.pulsar.source.config.PulsarSourceConfigUtils.checkConfigurations; +import static java.lang.Boolean.FALSE; +import static org.apache.flink.util.InstantiationUtil.isSerializable; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * The builder class for {@link PulsarSource} to make it easier for the users to construct a {@link + * PulsarSource}. + * + *

The following example shows the minimum setup to create a PulsarSource that reads the String + * values from a Pulsar topic. + * + *

{@code
+ * PulsarSource source = PulsarSource
+ *     .builder()
+ *     .setServiceUrl(PULSAR_BROKER_URL)
+ *     .setAdminUrl(PULSAR_BROKER_HTTP_URL)
+ *     .setSubscriptionName("flink-source-1")
+ *     .setTopics(Arrays.asList(TOPIC1, TOPIC2))
+ *     .setDeserializationSchema(PulsarDeserializationSchema.flinkSchema(new SimpleStringSchema()))
+ *     .build();
+ * }
+ * + *

The service url, admin url, subscription name, topics to consume, and the record deserializer + * are required fields that must be set. + * + *

To specify the starting position of PulsarSource, one can call {@link + * #setStartCursor(StartCursor)}. + * + *

By default, the PulsarSource runs in an {@link Boundedness#CONTINUOUS_UNBOUNDED} mode and + * never stop until the Flink job is canceled or fails. To let the PulsarSource run in {@link + * Boundedness#CONTINUOUS_UNBOUNDED} but stops at some given offsets, one can call {@link + * #setUnboundedStopCursor(StopCursor)} and disable auto partition discovery as described below. For + * example the following PulsarSource stops after it consumes up to a event time when the Flink + * started. + * + *

To stop the connector user has to disable the auto partition discovery. As auto partition + * discovery always expected new splits to come and not exiting. To disable auto partition + * discovery, use builder.setConfig({@link + * PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS}, -1). + * + *

{@code
+ * PulsarSource source = PulsarSource
+ *     .builder()
+ *     .setServiceUrl(PULSAR_BROKER_URL)
+ *     .setAdminUrl(PULSAR_BROKER_HTTP_URL)
+ *     .setSubscriptionName("flink-source-1")
+ *     .setTopics(Arrays.asList(TOPIC1, TOPIC2))
+ *     .setDeserializationSchema(PulsarDeserializationSchema.flinkSchema(new SimpleStringSchema()))
+ *     .setUnboundedStopCursor(StopCursor.atEventTime(System.currentTimeMillis()))
+ *     .build();
+ * }
+ * + * @param The output type of the source. + */ +@PublicEvolving +public final class PulsarSourceBuilder { + private static final Logger LOG = LoggerFactory.getLogger(com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder.class); + + private final Configuration configuration; + private PulsarSubscriber subscriber; + private RangeGenerator rangeGenerator; + private StartCursor startCursor; + private StopCursor stopCursor; + private Boundedness boundedness; + private PulsarDeserializationSchema deserializationSchema; + + // private builder constructor. + PulsarSourceBuilder() { + this.configuration = new Configuration(); + this.startCursor = StartCursor.defaultStartCursor(); + this.stopCursor = StopCursor.defaultStopCursor(); + } + + /** + * Sets the admin endpoint for the PulsarAdmin of the PulsarSource. + * + * @param adminUrl the url for the PulsarAdmin. + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setAdminUrl(String adminUrl) { + return setConfig(PULSAR_ADMIN_URL, adminUrl); + } + + /** + * Sets the server's link for the PulsarConsumer of the PulsarSource. + * + * @param serviceUrl the server url of the Pulsar cluster. + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setServiceUrl(String serviceUrl) { + return setConfig(PULSAR_SERVICE_URL, serviceUrl); + } + + /** + * Sets the name for this pulsar subscription. + * + * @param subscriptionName the server url of the Pulsar cluster. + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setSubscriptionName(String subscriptionName) { + return setConfig(PULSAR_SUBSCRIPTION_NAME, subscriptionName); + } + + /** + * {@link SubscriptionType} is the consuming behavior for pulsar, we would generator different + * split by the given subscription type. Please take some time to consider which subscription + * type matches your application best. Default is {@link SubscriptionType#Shared}. + * + * @param subscriptionType The type of subscription. + * @return this PulsarSourceBuilder. + * @see Pulsar + * Subscriptions + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setSubscriptionType(SubscriptionType subscriptionType) { + if (configuration.contains(PULSAR_SUBSCRIPTION_TYPE)) { + SubscriptionType existedType = configuration.get(PULSAR_SUBSCRIPTION_TYPE); + checkArgument( + existedType == subscriptionType, + "Can't override the subscription type %s with a new type %s", + existedType, + subscriptionType); + } else { + configuration.set(PULSAR_SUBSCRIPTION_TYPE, subscriptionType); + } + return this; + } + + /** + * Set a pulsar topic list for flink source. Some topic may not exist currently, consuming this + * non-existed topic wouldn't throw any exception. But the best solution is just consuming by + * using a topic regex. You can set topics once either with {@link #setTopics} or {@link + * #setTopicPattern} in this builder. + * + * @param topics The topic list you would like to consume message. + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setTopics(String... topics) { + return setTopics(Arrays.asList(topics)); + } + + /** + * Set a pulsar topic list for flink source. Some topic may not exist currently, consuming this + * non-existed topic wouldn't throw any exception. But the best solution is just consuming by + * using a topic regex. You can set topics once either with {@link #setTopics} or {@link + * #setTopicPattern} in this builder. + * + * @param topics The topic list you would like to consume message. + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setTopics(List topics) { + ensureSubscriberIsNull("topics"); + this.subscriber = PulsarSubscriber.getTopicListSubscriber(topics); + return this; + } + + /** + * Set a topic pattern to consume from the java regex str. You can set topics once either with + * {@link #setTopics} or {@link #setTopicPattern} in this builder. + * + * @param topicsPattern the pattern of the topic name to consume from. + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setTopicPattern(String topicsPattern) { + return setTopicPattern(Pattern.compile(topicsPattern)); + } + + /** + * Set a topic pattern to consume from the java {@link Pattern}. You can set topics once either + * with {@link #setTopics} or {@link #setTopicPattern} in this builder. + * + * @param topicsPattern the pattern of the topic name to consume from. + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setTopicPattern(Pattern topicsPattern) { + return setTopicPattern(topicsPattern, RegexSubscriptionMode.AllTopics); + } + + /** + * Set a topic pattern to consume from the java regex str. You can set topics once either with + * {@link #setTopics} or {@link #setTopicPattern} in this builder. + * + * @param topicsPattern the pattern of the topic name to consume from. + * @param regexSubscriptionMode The topic filter for regex subscription. + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setTopicPattern( + String topicsPattern, RegexSubscriptionMode regexSubscriptionMode) { + return setTopicPattern(Pattern.compile(topicsPattern), regexSubscriptionMode); + } + + /** + * Set a topic pattern to consume from the java {@link Pattern}. You can set topics once either + * with {@link #setTopics} or {@link #setTopicPattern} in this builder. + * + * @param topicsPattern the pattern of the topic name to consume from. + * @param regexSubscriptionMode When subscribing to a topic using a regular expression, you can + * pick a certain type of topics. + *
    + *
  • PersistentOnly: only subscribe to persistent topics. + *
  • NonPersistentOnly: only sustartCursorbscribe to non-persistent topics. + *
  • AllTopics: subscribe to both persistent and non-persistent topics. + *
+ * + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setTopicPattern( + Pattern topicsPattern, RegexSubscriptionMode regexSubscriptionMode) { + ensureSubscriberIsNull("topic pattern"); + this.subscriber = + PulsarSubscriber.getTopicPatternSubscriber(topicsPattern, regexSubscriptionMode); + return this; + } + + /** + * Set a topic range generator for Key_Shared subscription. + * + * @param rangeGenerator A generator which would generate a set of {@link TopicRange} for given + * topic. + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setRangeGenerator(RangeGenerator rangeGenerator) { + if (configuration.contains(PULSAR_SUBSCRIPTION_TYPE)) { + SubscriptionType subscriptionType = configuration.get(PULSAR_SUBSCRIPTION_TYPE); + checkArgument( + subscriptionType == SubscriptionType.Key_Shared, + "Key_Shared subscription should be used for custom rangeGenerator instead of %s", + subscriptionType); + } else { + LOG.warn("No subscription type provided, set it to Key_Shared."); + setSubscriptionType(SubscriptionType.Key_Shared); + } + this.rangeGenerator = checkNotNull(rangeGenerator); + return this; + } + + /** + * Specify from which offsets the PulsarSource should start consume from by providing an {@link + * StartCursor}. + * + * @param startCursor set the starting offsets for the Source. + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setStartCursor(StartCursor startCursor) { + this.startCursor = checkNotNull(startCursor); + return this; + } + + /** + * By default, the PulsarSource runs in an {@link Boundedness#CONTINUOUS_UNBOUNDED} mode and + * never stop until the Flink job is canceled or fails. To let the PulsarSource run in {@link + * Boundedness#CONTINUOUS_UNBOUNDED} but stops at some given offsets, one can call {@link + * #setUnboundedStopCursor(StopCursor)} and disable auto partition discovery as described below. + * + *

This method is different from {@link #setBoundedStopCursor(StopCursor)} that after setting + * the stopping offsets with this method, {@link PulsarSource#getBoundedness()} will still + * return {@link Boundedness#CONTINUOUS_UNBOUNDED} even though it will stop at the stopping + * offsets specified by the stopping offsets {@link StopCursor}. + * + *

To stop the connector user has to disable the auto partition discovery. As auto partition + * discovery always expected new splits to come and not exiting. To disable auto partition + * discovery, use builder.setConfig({@link + * PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS}, -1). + * + * @param stopCursor The {@link StopCursor} to specify the stopping offset. + * @return this PulsarSourceBuilder. + * @see #setBoundedStopCursor(StopCursor) + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setUnboundedStopCursor(StopCursor stopCursor) { + this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED; + this.stopCursor = checkNotNull(stopCursor); + return this; + } + + /** + * By default the PulsarSource is set to run in {@link Boundedness#CONTINUOUS_UNBOUNDED} manner + * and thus never stops until the Flink job fails or is canceled. To let the PulsarSource run in + * {@link Boundedness#BOUNDED} manner and stops at some point, one can set an {@link StopCursor} + * to specify the stopping offsets for each partition. When all the partitions have reached + * their stopping offsets, the PulsarSource will then exit. + * + *

This method is different from {@link #setUnboundedStopCursor(StopCursor)} that after + * setting the stopping offsets with this method, {@link PulsarSource#getBoundedness()} will + * return {@link Boundedness#BOUNDED} instead of {@link Boundedness#CONTINUOUS_UNBOUNDED}. + * + * @param stopCursor the {@link StopCursor} to specify the stopping offsets. + * @return this PulsarSourceBuilder. + * @see #setUnboundedStopCursor(StopCursor) + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setBoundedStopCursor(StopCursor stopCursor) { + this.boundedness = Boundedness.BOUNDED; + this.stopCursor = checkNotNull(stopCursor); + return this; + } + + /** + * DeserializationSchema is required for getting the {@link Schema} for deserialize message from + * pulsar and getting the {@link TypeInformation} for message serialization in flink. + * + *

We have defined a set of implementations, using {@code + * PulsarDeserializationSchema#pulsarSchema} or {@code PulsarDeserializationSchema#flinkSchema} + * for creating the desired schema. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setDeserializationSchema( + PulsarDeserializationSchema deserializationSchema) { + com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder self = specialized(); + self.deserializationSchema = deserializationSchema; + return self; + } + + /** + * Set an arbitrary property for the PulsarSource and PulsarConsumer. The valid keys can be + * found in {@link PulsarSourceOptions} and {@link PulsarOptions}. + * + *

Make sure the option could be set only once or with same value. + * + * @param key the key of the property. + * @param value the value of the property. + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setConfig(ConfigOption key, T value) { + checkNotNull(key); + checkNotNull(value); + if (configuration.contains(key)) { + T oldValue = configuration.get(key); + checkArgument( + Objects.equals(oldValue, value), + "This option %s has been already set to value %s.", + key.key(), + oldValue); + } else { + configuration.set(key, value); + } + return this; + } + + /** + * Set arbitrary properties for the PulsarSource and PulsarConsumer. The valid keys can be found + * in {@link PulsarSourceOptions} and {@link PulsarOptions}. + * + * @param config the config to set for the PulsarSource. + * @return this PulsarSourceBuilder. + */ + public com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder setConfig(Configuration config) { + Map existedConfigs = configuration.toMap(); + List duplicatedKeys = new ArrayList<>(); + for (Map.Entry entry : config.toMap().entrySet()) { + String key = entry.getKey(); + String value2 = existedConfigs.get(key); + if (value2 != null && !value2.equals(entry.getValue())) { + duplicatedKeys.add(key); + } + } + checkArgument( + duplicatedKeys.isEmpty(), + "Invalid configuration, these keys %s are already exist with different config value.", + duplicatedKeys); + configuration.addAll(config); + return this; + } + + /** + * Build the {@link PulsarSource}. + * + * @return a PulsarSource with the settings made for this builder. + */ + @SuppressWarnings("java:S3776") + public PulsarSource build() { + // Check builder configuration. + checkConfigurations(configuration); + + // Ensure the topic subscriber for pulsar. + checkNotNull(subscriber, "No topic names or topic pattern are provided."); + + SubscriptionType subscriptionType = configuration.get(PULSAR_SUBSCRIPTION_TYPE); + if (subscriptionType == SubscriptionType.Key_Shared) { + if (rangeGenerator == null) { + LOG.warn( + "No range generator provided for key_shared subscription," + + " we would use the DivideRangeGenerator as the default range generator."); + this.rangeGenerator = new UniformRangeGenerator(); + } + } else { + // Override the range generator. + this.rangeGenerator = new FullRangeGenerator(); + } + + if (boundedness == null) { + LOG.warn("No boundedness was set, mark it as a endless stream."); + this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED; + } + if (boundedness == Boundedness.BOUNDED + && configuration.get(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS) >= 0) { + LOG.warn( + "{} property is overridden to -1 because the source is bounded.", + PULSAR_PARTITION_DISCOVERY_INTERVAL_MS); + configuration.set(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, -1L); + } + + checkNotNull(deserializationSchema, "deserializationSchema should be set."); + + // Enable transaction if the cursor auto commit is disabled for Key_Shared & Shared. + if (FALSE.equals(configuration.get(PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE)) + && (subscriptionType == SubscriptionType.Key_Shared + || subscriptionType == SubscriptionType.Shared)) { + LOG.info( + "Pulsar cursor auto commit is disabled, make sure checkpoint is enabled " + + "and your pulsar cluster is support the transaction."); + configuration.set(PULSAR_ENABLE_TRANSACTION, true); + + if (!configuration.contains(PULSAR_TRANSACTION_TIMEOUT_MILLIS)) { + LOG.warn( + "The default pulsar transaction timeout is 3 hours, " + + "make sure it was greater than your checkpoint interval."); + } else { + Long timeout = configuration.get(PULSAR_TRANSACTION_TIMEOUT_MILLIS); + LOG.warn( + "The configured transaction timeout is {} mille seconds, " + + "make sure it was greater than your checkpoint interval.", + timeout); + } + } + + // Since these implementation could be a lambda, make sure they are serializable. + checkState(isSerializable(startCursor), "StartCursor isn't serializable"); + checkState(isSerializable(stopCursor), "StopCursor isn't serializable"); + checkState(isSerializable(rangeGenerator), "RangeGenerator isn't serializable"); + + // Make the configuration unmodifiable. + Configuration config = new UnmodifiableConfiguration(configuration); + + return new PulsarSource<>( + config, + subscriber, + rangeGenerator, + startCursor, + stopCursor, + boundedness, + deserializationSchema); + } + + // ------------- private helpers -------------- + + /** Helper method for java compiler recognize the generic type. */ + @SuppressWarnings("unchecked") + private com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder specialized() { + return (com.bytedance.bitsail.connector.pulsar.source.PulsarSourceBuilder) this; + } + + /** Topic name and topic pattern is conflict, make sure they are set only once. */ + private void ensureSubscriberIsNull(String attemptingSubscribeMode) { + if (subscriber != null) { + throw new IllegalStateException( + String.format( + "Cannot use %s for consumption because a %s is already set for consumption.", + attemptingSubscribeMode, subscriber.getClass().getSimpleName())); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceOptions.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceOptions.java new file mode 100644 index 000000000..abef70190 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceOptions.java @@ -0,0 +1,610 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.docs.ConfigGroup; +import org.apache.flink.annotation.docs.ConfigGroups; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.description.Description; +import com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions; +import com.bytedance.bitsail.connector.pulsar.source.config.CursorVerification; +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionMode; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static java.util.Collections.emptyMap; +import static org.apache.flink.configuration.description.TextElement.code; +import static org.apache.flink.configuration.description.TextElement.text; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.CONSUMER_CONFIG_PREFIX; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.SOURCE_CONFIG_PREFIX; + +/** + * Configurations for PulsarSource. All the options list here could be configured in {@link + * PulsarSourceBuilder#setConfig(ConfigOption, Object)}. The {@link PulsarOptions} is also required + * for pulsar source. + * + * @see PulsarOptions + */ +@PublicEvolving +@ConfigGroups( + groups = { + @ConfigGroup(name = "PulsarSource", keyPrefix = SOURCE_CONFIG_PREFIX), + @ConfigGroup(name = "PulsarConsumer", keyPrefix = CONSUMER_CONFIG_PREFIX) + }) +@SuppressWarnings("java:S1192") +public final class PulsarSourceOptions { + + // Pulsar source connector config prefix. + public static final String SOURCE_CONFIG_PREFIX = "pulsar.source."; + // Pulsar consumer API config prefix. + public static final String CONSUMER_CONFIG_PREFIX = "pulsar.consumer."; + + private PulsarSourceOptions() { + // This is a constant class + } + + /////////////////////////////////////////////////////////////////////////////// + // + // The configuration for pulsar source part. + // All the configuration listed below should have the pulsar.source prefix. + // + /////////////////////////////////////////////////////////////////////////////// + + public static final ConfigOption PULSAR_PARTITION_DISCOVERY_INTERVAL_MS = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "partitionDiscoveryIntervalMs") + .longType() + .defaultValue(Duration.ofSeconds(30).toMillis()) + .withDescription( + Description.builder() + .text( + "The interval (in ms) for the Pulsar source to discover the new partitions.") + .text(" A non-positive value disables the partition discovery.") + .build()); + + public static final ConfigOption PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "enableAutoAcknowledgeMessage") + .booleanType() + .defaultValue(false) + .withDescription( + Description.builder() + .text( + "Flink commits the consuming position with pulsar transactions on checkpoint.") + .text( + " However, if you have disabled the Flink checkpoint or disabled transaction for your Pulsar cluster,") + .text( + " ensure that you have set this option to %s.", + code("true")) + .linebreak() + .text( + "The source would use pulsar client's internal mechanism and commit cursor in two ways.") + .list( + text( + "For %s and %s subscription, the cursor would be committed once the message is consumed.", + code("Key_Shared"), code("Shared")), + text( + "For %s and %s subscription, the cursor would be committed in a given interval.", + code("Exclusive"), code("Failover"))) + .build()); + + public static final ConfigOption PULSAR_AUTO_COMMIT_CURSOR_INTERVAL = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "autoCommitCursorInterval") + .longType() + .defaultValue(Duration.ofSeconds(5).toMillis()) + .withDescription( + Description.builder() + .text( + "This option is used only when the user disables the checkpoint and uses Exclusive or Failover subscription.") + .text( + " We would automatically commit the cursor using the given period (in ms).") + .build()); + + public static final ConfigOption PULSAR_TRANSACTION_TIMEOUT_MILLIS = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "transactionTimeoutMillis") + .longType() + .defaultValue(Duration.ofHours(3).toMillis()) + .withDescription( + Description.builder() + .text( + "This option is used in %s or %s subscription.", + code("Shared"), code("Key_Shared")) + .text( + " You should configure this option when you do not enable the %s option.", + code("pulsar.source.enableAutoAcknowledgeMessage")) + .linebreak() + .text( + "The value (in ms) should be greater than the checkpoint interval.") + .build()); + + public static final ConfigOption PULSAR_MAX_FETCH_TIME = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "maxFetchTime") + .longType() + .defaultValue(Duration.ofSeconds(10).toMillis()) + .withDescription( + Description.builder() + .text("The maximum time (in ms) to wait when fetching records.") + .text(" A longer time increases throughput but also latency.") + .text( + " A fetch batch might be finished earlier because of %s.", + code("pulsar.source.maxFetchRecords")) + .build()); + + public static final ConfigOption PULSAR_MAX_FETCH_RECORDS = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "maxFetchRecords") + .intType() + .defaultValue(100) + .withDescription( + Description.builder() + .text( + "The maximum number of records to fetch to wait when polling.") + .text(" A longer time increases throughput but also latency.") + .text( + " A fetch batch might be finished earlier because of %s.", + code("pulsar.source.maxFetchTime")) + .build()); + + public static final ConfigOption PULSAR_VERIFY_INITIAL_OFFSETS = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "verifyInitialOffsets") + .enumType(CursorVerification.class) + .defaultValue(CursorVerification.WARN_ON_MISMATCH) + .withDescription( + Description.builder() + .text( + "Upon (re)starting the source, check whether the expected message can be read.") + .text( + " If failure is enabled, the application fails. Otherwise, it logs a warning.") + .text( + " A possible solution is to adjust the retention settings in Pulsar or ignoring the check result.") + .build()); + + /////////////////////////////////////////////////////////////////////////////// + // + // The configuration for ConsumerConfigurationData part. + // All the configuration listed below should have the pulsar.consumer prefix. + // + /////////////////////////////////////////////////////////////////////////////// + + public static final ConfigOption PULSAR_SUBSCRIPTION_NAME = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "subscriptionName") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text("Specify the subscription name for this consumer.") + .text( + " This argument is required when constructing the consumer.") + .build()); + + public static final ConfigOption PULSAR_SUBSCRIPTION_TYPE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "subscriptionType") + .enumType(SubscriptionType.class) + .defaultValue(SubscriptionType.Shared) + .withDescription( + Description.builder() + .text("Subscription type.") + .linebreak() + .linebreak() + .text("Four subscription types are available:") + .list( + text("Exclusive"), + text("Failover"), + text("Shared"), + text("Key_Shared")) + .build()); + + public static final ConfigOption PULSAR_SUBSCRIPTION_MODE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "subscriptionMode") + .enumType(SubscriptionMode.class) + .defaultValue(SubscriptionMode.Durable) + .withDescription( + Description.builder() + .text( + "Select the subscription mode to be used when subscribing to the topic.") + .list( + text( + "%s: Make the subscription to be backed by a durable cursor that will retain messages and persist the current position.", + code("Durable")), + text( + "%s: Lightweight subscription mode that doesn't have a durable cursor associated", + code("NonDurable"))) + .build()); + + public static final ConfigOption PULSAR_RECEIVER_QUEUE_SIZE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "receiverQueueSize") + .intType() + .defaultValue(1000) + .withDescription( + Description.builder() + .text("Size of a consumer's receiver queue.") + .linebreak() + .text( + "For example, the number of messages accumulated by a consumer before an application calls %s.", + code("Receive")) + .linebreak() + .text( + "A value higher than the default value increases consumer throughput, though at the expense of more memory utilization.") + .build()); + + public static final ConfigOption PULSAR_ACKNOWLEDGEMENTS_GROUP_TIME_MICROS = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "acknowledgementsGroupTimeMicros") + .longType() + .defaultValue(TimeUnit.MILLISECONDS.toMicros(100)) + .withDescription( + Description.builder() + .text( + "Group a consumer acknowledgment for a specified time (in μs).") + .text( + " By default, a consumer uses %s grouping time to send out acknowledgments to a broker.", + code("100μs")) + .text( + " If the group time is set to %s, acknowledgments are sent out immediately.", + code("0")) + .text( + " A longer ack group time is more efficient at the expense of a slight increase in message re-deliveries after a failure.") + .build()); + + public static final ConfigOption PULSAR_NEGATIVE_ACK_REDELIVERY_DELAY_MICROS = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "negativeAckRedeliveryDelayMicros") + .longType() + .defaultValue(TimeUnit.MINUTES.toMicros(1)) + .withDescription( + Description.builder() + .text( + "Delay (in μs) to wait before redelivering messages that failed to be processed.") + .linebreak() + .text( + "When an application uses %s, failed messages are redelivered after a fixed timeout.", + code("Consumer.negativeAcknowledge(Message)")) + .build()); + + public static final ConfigOption + PULSAR_MAX_TOTAL_RECEIVER_QUEUE_SIZE_ACROSS_PARTITIONS = + ConfigOptions.key( + CONSUMER_CONFIG_PREFIX + + "maxTotalReceiverQueueSizeAcrossPartitions") + .intType() + .defaultValue(50000) + .withDescription( + Description.builder() + .text( + "The maximum total receiver queue size across partitions.") + .linebreak() + .text( + "This setting reduces the receiver queue size for individual partitions if the total receiver queue size exceeds this value.") + .build()); + + public static final ConfigOption PULSAR_CONSUMER_NAME = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "consumerName") + .stringType() + .noDefaultValue() + .withDescription( + "The consumer name is informative and it can be used to identify a particular consumer instance from the topic stats."); + + public static final ConfigOption PULSAR_ACK_TIMEOUT_MILLIS = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "ackTimeoutMillis") + .longType() + .defaultValue(0L) + .withDescription( + Description.builder() + .text( + "The timeout (in ms) for unacknowledged messages, truncated to the nearest millisecond. The timeout needs to be greater than 1 second.") + .linebreak() + .text( + "By default, the acknowledge timeout is disabled and that means that messages delivered to a consumer will not be re-delivered unless the consumer crashes.") + .linebreak() + .text( + "When acknowledgement timeout being enabled, if a message is not acknowledged within the specified timeout it will be re-delivered to the consumer (possibly to a different consumer in case of a shared subscription).") + .build()); + + public static final ConfigOption PULSAR_TICK_DURATION_MILLIS = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "tickDurationMillis") + .longType() + .defaultValue(1000L) + .withDescription( + Description.builder() + .text("Granularity (in ms) of the ack-timeout redelivery.") + .linebreak() + .text( + "A greater (for example, 1 hour) %s reduces the memory overhead to track messages.", + code("tickDurationMillis")) + .build()); + + public static final ConfigOption PULSAR_PRIORITY_LEVEL = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "priorityLevel") + .intType() + .defaultValue(0) + .withDescription( + Description.builder() + .text( + "Priority level for a consumer to which a broker gives more priorities while dispatching messages in the shared subscription type.") + .linebreak() + .text( + "The broker follows descending priorities. For example, 0=max-priority, 1, 2,...") + .linebreak() + .text( + "In shared subscription mode, the broker first dispatches messages to the consumers on the highest priority level if they have permits.") + .text( + " Otherwise, the broker considers consumers on the next priority level.") + .linebreak() + .linebreak() + .text("Example 1") + .linebreak() + .text( + "If a subscription has consumer A with %s 0 and consumer B with %s 1, then the broker only dispatches messages to consumer A until it runs out permits and then starts dispatching messages to consumer B.", + code("priorityLevel"), code("priorityLevel")) + .linebreak() + .text("Example 2") + .linebreak() + .text( + "Consumer Priority, Level, Permits\n" + + "C1, 0, 2\n" + + "C2, 0, 1\n" + + "C3, 0, 1\n" + + "C4, 1, 2\n" + + "C5, 1, 1\n") + .linebreak() + .text( + "The order in which a broker dispatches messages to consumers is: C1, C2, C3, C1, C4, C5, C4.") + .build()); + + public static final ConfigOption PULSAR_MAX_PENDING_CHUNKED_MESSAGE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "maxPendingChunkedMessage") + .intType() + .defaultValue(10) + .withDescription( + Description.builder() + .text( + "The consumer buffers chunk messages into memory until it receives all the chunks of the original message.") + .text( + " While consuming chunk-messages, chunks from the same message might not be contiguous in the stream and they might be mixed with other messages' chunks.") + .text( + " So, consumer has to maintain multiple buffers to manage chunks coming from different messages.") + .text( + " This mainly happens when multiple publishers are publishing messages on the topic concurrently or publishers failed to publish all chunks of the messages.") + .linebreak() + .text( + "For example, there are M1-C1, M2-C1, M1-C2, M2-C2 messages.") + .text( + "Messages M1-C1 and M1-C2 belong to the M1 original message while M2-C1 and M2-C2 belong to the M2 message.") + .linebreak() + .text( + "Buffering a large number of outstanding uncompleted chunked messages can bring memory pressure and it can be guarded by providing this %s threshold.", + code("pulsar.consumer.maxPendingChunkedMessage")) + .text( + " Once, a consumer reaches this threshold, it drops the outstanding unchunked messages by silently acknowledging or asking the broker to redeliver messages later by marking it unacknowledged.") + .text( + " This behavior can be controlled by the %s option.", + code( + "pulsar.consumer.autoAckOldestChunkedMessageOnQueueFull")) + .build()); + + public static final ConfigOption PULSAR_AUTO_ACK_OLDEST_CHUNKED_MESSAGE_ON_QUEUE_FULL = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "autoAckOldestChunkedMessageOnQueueFull") + .booleanType() + .defaultValue(false) + .withDescription( + Description.builder() + .text( + "Buffering a large number of outstanding uncompleted chunked messages can bring memory pressure and it can be guarded by providing this %s threshold.", + code("pulsar.consumer.maxPendingChunkedMessage")) + .text( + " Once a consumer reaches this threshold, it drops the outstanding unchunked-messages by silently acknowledging if %s is true. Otherwise, it marks them for redelivery.", + code( + "pulsar.consumer.autoAckOldestChunkedMessageOnQueueFull")) + .build()); + + public static final ConfigOption PULSAR_EXPIRE_TIME_OF_INCOMPLETE_CHUNKED_MESSAGE_MILLIS = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "expireTimeOfIncompleteChunkedMessageMillis") + .longType() + .defaultValue(60 * 1000L) + .withDescription( + Description.builder() + .text( + "If a producer fails to publish all the chunks of a message,") + .text( + " the consumer can expire incomplete chunks if the consumer cannot receive all chunks in expire times (default 1 hour, in ms).") + .build()); + + public static final ConfigOption PULSAR_CRYPTO_FAILURE_ACTION = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "cryptoFailureAction") + .enumType(ConsumerCryptoFailureAction.class) + .defaultValue(ConsumerCryptoFailureAction.FAIL) + .withDescription( + Description.builder() + .text( + "The consumer should take action when it receives a message that can not be decrypted.") + .list( + text( + "%s: this is the default option to fail messages until crypto succeeds.", + code("FAIL")), + text( + "%s: silently acknowledge but do not deliver messages to an application.", + code("DISCARD")), + text( + "%s: deliver encrypted messages to applications. It is the application's responsibility to decrypt the message.", + code("CONSUME"))) + .linebreak() + .text("Fail to decompress the messages.") + .linebreak() + .text( + "If messages contain batch messages, a client is not be able to retrieve individual messages in batch.") + .linebreak() + .text( + "The delivered encrypted message contains %s which contains encryption and compression information in.", + code("EncryptionContext")) + .text( + " You can use an application to decrypt the consumed message payload.") + .build()); + + public static final ConfigOption> PULSAR_CONSUMER_PROPERTIES = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "properties") + .mapType() + .defaultValue(emptyMap()) + .withDescription( + Description.builder() + .text("A name or value property of this consumer.") + .text( + " %s is application defined metadata attached to a consumer.", + code("properties")) + .text( + " When getting a topic stats, associate this metadata with the consumer stats for easier identification.") + .build()); + + public static final ConfigOption PULSAR_READ_COMPACTED = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "readCompacted") + .booleanType() + .defaultValue(false) + .withDescription( + Description.builder() + .text( + "If enabling %s, a consumer reads messages from a compacted topic rather than reading a full message backlog of a topic.", + code("readCompacted")) + .linebreak() + .text( + "A consumer only sees the latest value for each key in the compacted topic, up until reaching the point in the topic message when compacting backlog. Beyond that point, send messages as normal.") + .linebreak() + .text( + "Only enabling %s on subscriptions to persistent topics, which have a single active consumer (like failure or exclusive subscriptions).", + code("readCompacted")) + .linebreak() + .text( + "Attempting to enable it on subscriptions to non-persistent topics or on shared subscriptions leads to a subscription call throwing a %s.", + code("PulsarClientException")) + .build()); + + public static final ConfigOption + PULSAR_SUBSCRIPTION_INITIAL_POSITION = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "subscriptionInitialPosition") + .enumType(SubscriptionInitialPosition.class) + .defaultValue(SubscriptionInitialPosition.Latest) + .withDescription( + "Initial position at which to set cursor when subscribing to a topic at first time."); + + // The config set for DeadLetterPolicy + + /** + * Dead letter policy for consumers. + * + *

By default, some messages are probably redelivered many times, even to the extent that it + * never stops. + * + *

By using the dead letter mechanism, messages have the max redelivery count. When exceeding + * the maximum number of redeliveries, messages are sent to the Dead Letter Topic and + * acknowledged automatically. + * + *

You can enable the dead letter mechanism by setting deadLetterPolicy. + * + *

Example pulsar.consumer.deadLetterPolicy.maxRedeliverCount = 10 Default dead + * letter topic name is {TopicName}-{Subscription}-DLQ. + * + *

To set a custom dead letter topic name: + * + *


+     * pulsar.consumer.deadLetterPolicy.maxRedeliverCount = 10
+     * pulsar.consumer.deadLetterPolicy.deadLetterTopic = your-topic-name
+     * 
+ * + *

When specifying the dead letter policy while not specifying ackTimeoutMillis, you can set + * the ack timeout to 30000 millisecond. + */ + public static final ConfigOption PULSAR_MAX_REDELIVER_COUNT = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "deadLetterPolicy.maxRedeliverCount") + .intType() + .defaultValue(0) + .withDescription( + "The maximum number of times that a message are redelivered before being sent to the dead letter queue."); + + public static final ConfigOption PULSAR_RETRY_LETTER_TOPIC = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "deadLetterPolicy.retryLetterTopic") + .stringType() + .noDefaultValue() + .withDescription("Name of the retry topic where the failed messages are sent."); + public static final ConfigOption PULSAR_DEAD_LETTER_TOPIC = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "deadLetterPolicy.deadLetterTopic") + .stringType() + .noDefaultValue() + .withDescription("Name of the dead topic where the failed messages are sent."); + + public static final ConfigOption PULSAR_RETRY_ENABLE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "retryEnable") + .booleanType() + .defaultValue(false) + .withDescription("If enabled, the consumer will automatically retry messages."); + + public static final ConfigOption PULSAR_AUTO_UPDATE_PARTITIONS_INTERVAL_SECONDS = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "autoUpdatePartitionsIntervalSeconds") + .intType() + .defaultValue(60) + .withDescription( + "The interval (in seconds) of updating partitions. This only works if autoUpdatePartitions is enabled."); + + public static final ConfigOption PULSAR_REPLICATE_SUBSCRIPTION_STATE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "replicateSubscriptionState") + .booleanType() + .defaultValue(false) + .withDescription( + Description.builder() + .text( + "If %s is enabled, a subscription state is replicated to geo-replicated clusters.", + code("replicateSubscriptionState")) + .build()); + + public static final ConfigOption PULSAR_ACK_RECEIPT_ENABLED = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "ackReceiptEnabled") + .booleanType() + .defaultValue(false) + .withDescription( + "Acknowledgement will return a receipt but this does not mean that the message will not be resent after getting the receipt."); + + public static final ConfigOption PULSAR_POOL_MESSAGES = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "poolMessages") + .booleanType() + .defaultValue(false) + .withDescription("Enable pooling of messages and the underlying data buffers."); + + public static final String PULSAR_START_CURSOR_MODE_LATEST = "latest"; + public static final String PULSAR_START_CURSOR_MODE_EARLIEST = "earliest"; + public static final String PULSAR_START_CURSOR_MODE_TIMESTAMP = "timestamp"; + public static final ConfigOption PULSAR_START_CURSOR_MODE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "startCursorMode") + .stringType() + .defaultValue(PULSAR_START_CURSOR_MODE_LATEST); + public static final ConfigOption PULSAR_START_CURSOR_TIMESTAMP = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "startCursorTimestamp") + .longType() + .noDefaultValue(); + public static final String PULSAR_STOP_CURSOR_MODE_LATEST = "latest"; + public static final String PULSAR_STOP_CURSOR_MODE_NEVER = "never"; + public static final String PULSAR_STOP_CURSOR_MODE_TIMESTAMP = "timestamp"; + public static final ConfigOption PULSAR_STOP_CURSOR_MODE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "stopCursorMode") + .stringType() + .defaultValue(PULSAR_STOP_CURSOR_MODE_NEVER); + public static final ConfigOption PULSAR_STOP_CURSOR_TIMESTAMP = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "stopCursorTimestamp") + .longType() + .noDefaultValue(); + +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceOptionsV1.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceOptionsV1.java new file mode 100644 index 000000000..d93cfc6b6 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceOptionsV1.java @@ -0,0 +1,245 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source; + +import com.bytedance.bitsail.common.option.ConfigOption; +import com.bytedance.bitsail.common.option.ConfigOptions; +import com.bytedance.bitsail.connector.pulsar.source.config.CursorVerification; + +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionMode; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static java.util.Collections.emptyMap; + +@SuppressWarnings("java:S1192") +public final class PulsarSourceOptionsV1 { + + public static final String READER_PREFIX = "job.reader."; + // Pulsar source connector config prefix. + public static final String SOURCE_CONFIG_PREFIX = READER_PREFIX + "pulsar.source."; + // Pulsar consumer API config prefix. + public static final String CONSUMER_CONFIG_PREFIX = READER_PREFIX + "pulsar.consumer."; + + private PulsarSourceOptionsV1() { + // This is a constant class + } + + /////////////////////////////////////////////////////////////////////////////// + // + // The configuration for pulsar source part. + // All the configuration listed below should have the pulsar.source prefix. + // + /////////////////////////////////////////////////////////////////////////////// + + public static final ConfigOption PULSAR_PARTITION_DISCOVERY_INTERVAL_MS = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "partitionDiscoveryIntervalMs") + .defaultValue(Duration.ofSeconds(30).toMillis()); + + public static final ConfigOption PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "enableAutoAcknowledgeMessage") + .defaultValue(false); + + public static final ConfigOption PULSAR_AUTO_COMMIT_CURSOR_INTERVAL = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "autoCommitCursorInterval") + .defaultValue(Duration.ofSeconds(5).toMillis()); + + public static final ConfigOption PULSAR_TRANSACTION_TIMEOUT_MILLIS = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "transactionTimeoutMillis") + .defaultValue(Duration.ofHours(3).toMillis()); + + public static final ConfigOption PULSAR_MAX_FETCH_TIME = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "maxFetchTime") + .defaultValue(Duration.ofSeconds(100).toMillis()); + + public static final ConfigOption PULSAR_MAX_FETCH_RECORDS = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "maxFetchRecords") + .defaultValue(100); + + public static final ConfigOption PULSAR_VERIFY_INITIAL_OFFSETS = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "verifyInitialOffsets") + .defaultValue(CursorVerification.WARN_ON_MISMATCH.name()); + + /////////////////////////////////////////////////////////////////////////////// + // + // The configuration for ConsumerConfigurationData part. + // All the configuration listed below should have the pulsar.consumer prefix. + // + /////////////////////////////////////////////////////////////////////////////// + + public static final ConfigOption PULSAR_SUBSCRIPTION_NAME = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "subscriptionName") + .noDefaultValue(String.class); + + public static final ConfigOption PULSAR_SUBSCRIPTION_TYPE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "subscriptionType") + .defaultValue(SubscriptionType.Shared.name()); + + public static final ConfigOption PULSAR_SUBSCRIPTION_MODE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "subscriptionMode") + .defaultValue(SubscriptionMode.Durable.name()); + + public static final ConfigOption PULSAR_RECEIVER_QUEUE_SIZE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "receiverQueueSize") + .defaultValue(1000); + + public static final ConfigOption PULSAR_ACKNOWLEDGEMENTS_GROUP_TIME_MICROS = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "acknowledgementsGroupTimeMicros") + .defaultValue(TimeUnit.MILLISECONDS.toMicros(100)); + + public static final ConfigOption PULSAR_NEGATIVE_ACK_REDELIVERY_DELAY_MICROS = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "negativeAckRedeliveryDelayMicros") + .defaultValue(TimeUnit.MINUTES.toMicros(1)); + + public static final ConfigOption + PULSAR_MAX_TOTAL_RECEIVER_QUEUE_SIZE_ACROSS_PARTITIONS = + ConfigOptions.key( + CONSUMER_CONFIG_PREFIX + + "maxTotalReceiverQueueSizeAcrossPartitions") + .defaultValue(50000); + + public static final ConfigOption PULSAR_CONSUMER_NAME = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "consumerName") + .noDefaultValue(String.class); + + public static final ConfigOption PULSAR_ACK_TIMEOUT_MILLIS = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "ackTimeoutMillis") + .defaultValue(0L); + + public static final ConfigOption PULSAR_TICK_DURATION_MILLIS = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "tickDurationMillis") + .defaultValue(1000L); + + public static final ConfigOption PULSAR_PRIORITY_LEVEL = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "priorityLevel") + .defaultValue(0); + + public static final ConfigOption PULSAR_MAX_PENDING_CHUNKED_MESSAGE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "maxPendingChunkedMessage") + .defaultValue(10); + + public static final ConfigOption PULSAR_AUTO_ACK_OLDEST_CHUNKED_MESSAGE_ON_QUEUE_FULL = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "autoAckOldestChunkedMessageOnQueueFull") + .defaultValue(false); + + public static final ConfigOption PULSAR_EXPIRE_TIME_OF_INCOMPLETE_CHUNKED_MESSAGE_MILLIS = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "expireTimeOfIncompleteChunkedMessageMillis") + .defaultValue(60 * 1000L); + + public static final ConfigOption PULSAR_CRYPTO_FAILURE_ACTION = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "cryptoFailureAction") + .defaultValue(ConsumerCryptoFailureAction.FAIL.name()); + + // TODO (Dian): adapt to json format + public static final ConfigOption> PULSAR_CONSUMER_PROPERTIES = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "properties") + .defaultValue(emptyMap()); + + public static final ConfigOption PULSAR_READ_COMPACTED = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "readCompacted") + .defaultValue(false); + + public static final ConfigOption + PULSAR_SUBSCRIPTION_INITIAL_POSITION = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "subscriptionInitialPosition") + .defaultValue(SubscriptionInitialPosition.Latest.name()); + + // The config set for DeadLetterPolicy + + /** + * Dead letter policy for consumers. + * + *

By default, some messages are probably redelivered many times, even to the extent that it + * never stops. + * + *

By using the dead letter mechanism, messages have the max redelivery count. When exceeding + * the maximum number of redeliveries, messages are sent to the Dead Letter Topic and + * acknowledged automatically. + * + *

You can enable the dead letter mechanism by setting deadLetterPolicy. + * + *

Example pulsar.consumer.deadLetterPolicy.maxRedeliverCount = 10 Default dead + * letter topic name is {TopicName}-{Subscription}-DLQ. + * + *

To set a custom dead letter topic name: + * + *


+     * pulsar.consumer.deadLetterPolicy.maxRedeliverCount = 10
+     * pulsar.consumer.deadLetterPolicy.deadLetterTopic = your-topic-name
+     * 
+ * + *

When specifying the dead letter policy while not specifying ackTimeoutMillis, you can set + * the ack timeout to 30000 millisecond. + */ + public static final ConfigOption PULSAR_MAX_REDELIVER_COUNT = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "deadLetterPolicy.maxRedeliverCount") + .defaultValue(0); + + public static final ConfigOption PULSAR_RETRY_LETTER_TOPIC = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "deadLetterPolicy.retryLetterTopic") + .noDefaultValue(String.class); + public static final ConfigOption PULSAR_DEAD_LETTER_TOPIC = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "deadLetterPolicy.deadLetterTopic") + .noDefaultValue(String.class); + + public static final ConfigOption PULSAR_RETRY_ENABLE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "retryEnable") + .defaultValue(false); + + public static final ConfigOption PULSAR_AUTO_UPDATE_PARTITIONS_INTERVAL_SECONDS = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "autoUpdatePartitionsIntervalSeconds") + .defaultValue(60); + + public static final ConfigOption PULSAR_REPLICATE_SUBSCRIPTION_STATE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "replicateSubscriptionState") + .defaultValue(false); + + public static final ConfigOption PULSAR_ACK_RECEIPT_ENABLED = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "ackReceiptEnabled") + .defaultValue(false); + + public static final ConfigOption PULSAR_POOL_MESSAGES = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "poolMessages") + .defaultValue(false); + + public static final String PULSAR_START_CURSOR_MODE_LATEST = "latest"; + public static final String PULSAR_START_CURSOR_MODE_EARLIEST = "earliest"; + public static final String PULSAR_START_CURSOR_MODE_TIMESTAMP = "timestamp"; + public static final ConfigOption PULSAR_START_CURSOR_MODE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "startCursorMode") + .defaultValue(PULSAR_START_CURSOR_MODE_LATEST); + public static final ConfigOption PULSAR_START_CURSOR_TIMESTAMP = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "startCursorTimestamp") + .noDefaultValue(Long.class); + public static final String PULSAR_STOP_CURSOR_MODE_LATEST = "latest"; + public static final String PULSAR_STOP_CURSOR_MODE_NEVER = "never"; + public static final String PULSAR_STOP_CURSOR_MODE_TIMESTAMP = "timestamp"; + public static final ConfigOption PULSAR_STOP_CURSOR_MODE = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "stopCursorMode") + .defaultValue(PULSAR_STOP_CURSOR_MODE_NEVER); + public static final ConfigOption PULSAR_STOP_CURSOR_TIMESTAMP = + ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "stopCursorTimestamp") + .noDefaultValue(Long.class); + +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceV1.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceV1.java new file mode 100644 index 000000000..04a1fc718 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/PulsarSourceV1.java @@ -0,0 +1,313 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source; + +import com.bytedance.bitsail.base.connector.reader.v1.Boundedness; +import com.bytedance.bitsail.base.connector.reader.v1.Source; +import com.bytedance.bitsail.base.connector.reader.v1.SourceReader; +import com.bytedance.bitsail.base.connector.reader.v1.SourceSplitCoordinator; +import com.bytedance.bitsail.base.execution.ExecutionEnviron; +import com.bytedance.bitsail.base.execution.Mode; +import com.bytedance.bitsail.base.extension.ParallelismComputable; +import com.bytedance.bitsail.base.parallelism.ParallelismAdvice; +import com.bytedance.bitsail.base.serializer.BinarySerializer; +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.common.option.CommonOptions; +import com.bytedance.bitsail.common.row.Row; +import com.bytedance.bitsail.connector.pulsar.common.config.v1.PulsarOptionsV1; +import com.bytedance.bitsail.connector.pulsar.common.config.v1.PulsarUtils; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.coordinator.PulsarSourceEnumStateV1; +import com.bytedance.bitsail.connector.pulsar.source.coordinator.PulsarSourceSplitCoordinator; +import com.bytedance.bitsail.connector.pulsar.source.coordinator.SplitsAssignmentStateV1; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StartCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicMetadata; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range.FullRangeGenerator; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range.RangeGenerator; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range.UniformRangeGenerator; +import com.bytedance.bitsail.connector.pulsar.source.reader.source.v1.PulsarOrderedSourceReaderV1; +import com.bytedance.bitsail.connector.pulsar.source.reader.source.v1.PulsarUnorderedSourceReaderV1; +import com.bytedance.bitsail.connector.pulsar.source.reader.split.v1.PulsarOrderedPartitionSplitReader; +import com.bytedance.bitsail.connector.pulsar.source.reader.split.v1.PulsarUnorderedPartitionSplitReader; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplit; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplitSerializer; + +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.RegexSubscriptionMode; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; + +import java.util.List; +import java.util.Objects; +import java.util.function.Supplier; +import java.util.regex.Pattern; + +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptionsV1.PULSAR_START_CURSOR_MODE_EARLIEST; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptionsV1.PULSAR_START_CURSOR_MODE_LATEST; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptionsV1.PULSAR_START_CURSOR_MODE_TIMESTAMP; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptionsV1.PULSAR_STOP_CURSOR_MODE_LATEST; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptionsV1.PULSAR_STOP_CURSOR_MODE_NEVER; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptionsV1.PULSAR_STOP_CURSOR_MODE_TIMESTAMP; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptionsV1.PULSAR_SUBSCRIPTION_TYPE; + +@PublicEvolving +@Slf4j +public final class PulsarSourceV1 + implements Source, ParallelismComputable { + private static final long serialVersionUID = 7773108631275567433L; + private BitSailConfiguration readerConfiguration; + + private BitSailConfiguration commonConfiguration; + private static final int DEFAULT_PULSAR_PARALLELISM_THRESHOLD = 4; + private RangeGenerator rangeGenerator; + private PulsarSubscriber subscriber; + private StartCursor startCursor; + private SourceConfiguration sourceConfiguration; + private StopCursor stopCursor; + private SplitsAssignmentStateV1 assignmentState; + + @Override + public void configure(ExecutionEnviron execution, BitSailConfiguration readerConfiguration) { + this.readerConfiguration = readerConfiguration; + this.commonConfiguration = execution.getCommonConfiguration(); + SubscriptionType subscriptionType = + PulsarUtils.getSubscriptionType(readerConfiguration.getString(PULSAR_SUBSCRIPTION_TYPE.key())); + if (subscriptionType == SubscriptionType.Key_Shared) { + if (rangeGenerator == null) { + log.warn( + "No range generator provided for key_shared subscription," + + " we would use the DivideRangeGenerator as the default range generator."); + this.rangeGenerator = new UniformRangeGenerator(); + } + } else { + // Override the range generator. + this.rangeGenerator = new FullRangeGenerator(); + } + String topics = readerConfiguration.getString(PulsarOptionsV1.PULSAR_TOPICS.key()); + String topicMode = readerConfiguration.getString(PulsarOptionsV1.PULSAR_TOPIC_MODE.key()); + String topicSubscriptionMode = readerConfiguration.getString(PulsarSourceOptionsV1.PULSAR_SUBSCRIPTION_MODE.key()); + if (PulsarUtils.TOPIC_MODE_PATTERN.equalsIgnoreCase(topicMode)) { + this.subscriber = + PulsarSubscriber.getTopicPatternSubscriber(Pattern.compile(topics), PulsarUtils.getRegexSubscriptionMode(topicSubscriptionMode)); + + } else { + this.subscriber = PulsarSubscriber.getTopicListSubscriber(PulsarUtils.getTopicList(topics)); + } + setStartCursor(readerConfiguration); + setStopCursor(readerConfiguration); + + sourceConfiguration = new SourceConfiguration(readerConfiguration); + + assignmentState = + new SplitsAssignmentStateV1(stopCursor, sourceConfiguration); + } + + private void setStartCursor(BitSailConfiguration readerConfiguration) { + String startCursorMode = readerConfiguration.getString(PulsarSourceOptionsV1.PULSAR_START_CURSOR_MODE.key(), PULSAR_START_CURSOR_MODE_LATEST); + switch (startCursorMode) { + case PULSAR_START_CURSOR_MODE_LATEST: + this.startCursor = StartCursor.latest(); + break; + case PULSAR_START_CURSOR_MODE_EARLIEST: + this.startCursor = StartCursor.earliest(); + break; + case PULSAR_START_CURSOR_MODE_TIMESTAMP: + Long cursorTimestamp = readerConfiguration.getLong(PulsarSourceOptionsV1.PULSAR_START_CURSOR_TIMESTAMP.key()); + if (cursorTimestamp == null) { + cursorTimestamp = System.currentTimeMillis(); + } + this.startCursor = StartCursor.fromMessageTime(cursorTimestamp); + break; + default: + this.startCursor = StartCursor.latest(); + break; + } + } + + private void setStopCursor(BitSailConfiguration readerConfiguration) { + String stopCursorMode = readerConfiguration.get(PulsarSourceOptionsV1.PULSAR_STOP_CURSOR_MODE); + switch (stopCursorMode) { + case PULSAR_STOP_CURSOR_MODE_LATEST: + this.stopCursor = StopCursor.latest(); + break; + case PULSAR_STOP_CURSOR_MODE_NEVER: + this.stopCursor = StopCursor.never(); + break; + case PULSAR_STOP_CURSOR_MODE_TIMESTAMP: + Long cursorTimestamp = readerConfiguration.getLong(PulsarSourceOptionsV1.PULSAR_STOP_CURSOR_TIMESTAMP.key()); + if (cursorTimestamp == null) { + cursorTimestamp = System.currentTimeMillis(); + } + this.stopCursor = StopCursor.atEventTime(cursorTimestamp); + break; + default: + this.stopCursor = StopCursor.never(); + break; + } + } + + @Override + public Boundedness getSourceBoundedness() { + return Mode.BATCH.equals(Mode.getJobRunMode(commonConfiguration.get(CommonOptions.JOB_TYPE))) ? + Boundedness.BOUNDEDNESS : + Boundedness.UNBOUNDEDNESS; + } + + + @Override + public SourceSplitCoordinator createSplitCoordinator( + SourceSplitCoordinator.Context coordinatorContext) { + return new PulsarSourceSplitCoordinator( + subscriber, + startCursor, + rangeGenerator, + readerConfiguration, + sourceConfiguration, + coordinatorContext, + assignmentState, + getSourceBoundedness()); + } + + @Override + public String getReaderName() { + return "pulsar"; + } + + @Override + public ParallelismAdvice getParallelismAdvice(BitSailConfiguration commonConfiguration, + BitSailConfiguration pulsarConfiguration, + ParallelismAdvice upstreamAdvice) throws Exception { + String topics = pulsarConfiguration.getString(PulsarOptionsV1.PULSAR_TOPICS.key()); + String topicMode = pulsarConfiguration.getString(PulsarOptionsV1.PULSAR_TOPIC_MODE.key()); + String topicSubscriptionMode = pulsarConfiguration.getString(PulsarSourceOptionsV1.PULSAR_SUBSCRIPTION_MODE.key()); + PulsarAdmin pulsarAdmin = PulsarUtils.createAdmin(pulsarConfiguration); + Integer partitions; + if (PulsarUtils.TOPIC_MODE_PATTERN.equalsIgnoreCase(topicMode)) { + RegexSubscriptionMode subscriptionMode = PulsarUtils.getRegexSubscriptionMode(topicSubscriptionMode); + TopicName destination = TopicName.get(topics); + NamespaceName namespaceName = destination.getNamespaceObject(); + String namespace = namespaceName.toString(); + Pattern topicPattern = Pattern.compile(topics); + partitions = pulsarAdmin + .namespaces() + .getTopics(namespace) + .parallelStream() + .filter(t -> PulsarUtils.matchesRegexSubscriptionMode(t, subscriptionMode)) + .filter(topic -> topicPattern.matcher(topic).find()) + .map(topic -> PulsarUtils.queryTopicMetadata(pulsarAdmin, topic)) + .filter(Objects::nonNull) + .map(TopicMetadata::getPartitionSize) + .reduce(0, Integer::sum); + } else { + List topicList = PulsarUtils.getTopicList(topics); + partitions = topicList.parallelStream() + .map(topic -> PulsarUtils.queryTopicMetadata(pulsarAdmin, topic)) + .filter(Objects::nonNull) + .map(TopicMetadata::getPartitionSize) + .reduce(0, Integer::sum); + } + + + try { + int adviceParallelism = Math.max(partitions / DEFAULT_PULSAR_PARALLELISM_THRESHOLD, 1); + + return ParallelismAdvice.builder() + .adviceParallelism(adviceParallelism) + .enforceDownStreamChain(true) + .build(); + } finally { + } + } + + + @Override + public SourceReader createReader(SourceReader.Context readerContext) { + PulsarClient pulsarClient = PulsarUtils.createClient(readerConfiguration); + PulsarAdmin pulsarAdmin = PulsarUtils.createAdmin(readerConfiguration); + + // Create different pulsar source reader by subscription type. + SubscriptionType subscriptionType = sourceConfiguration.getSubscriptionType(); + if (subscriptionType == SubscriptionType.Failover + || subscriptionType == SubscriptionType.Exclusive) { + // Create a ordered split reader supplier. + Supplier splitReaderSupplier = + () -> + new PulsarOrderedPartitionSplitReader( + pulsarClient, + pulsarAdmin, + readerConfiguration, + sourceConfiguration); + + return new PulsarOrderedSourceReaderV1( + new FutureCompletingBlockingQueue<>(2), + splitReaderSupplier, + readerConfiguration, + readerContext, + sourceConfiguration, + pulsarClient, + pulsarAdmin); + } else if (subscriptionType == SubscriptionType.Shared + || subscriptionType == SubscriptionType.Key_Shared) { + TransactionCoordinatorClient coordinatorClient = + ((PulsarClientImpl) pulsarClient).getTcClient(); + if (coordinatorClient == null + && !sourceConfiguration.isEnableAutoAcknowledgeMessage()) { + throw new IllegalStateException("Transaction is required but didn't enabled"); + } + + Supplier splitReaderSupplier = + () -> + new PulsarUnorderedPartitionSplitReader( + pulsarClient, + pulsarAdmin, + readerConfiguration, + sourceConfiguration, + coordinatorClient); + + return new PulsarUnorderedSourceReaderV1( + new FutureCompletingBlockingQueue<>(2), + splitReaderSupplier, + readerConfiguration, + readerContext, + sourceConfiguration, + pulsarClient, + pulsarAdmin, + coordinatorClient); + } else { + throw new UnsupportedOperationException( + "This subscription type is not " + subscriptionType + " supported currently."); + } + } + + + @Override + public BinarySerializer getSplitSerializer() { + return PulsarPartitionSplitSerializer.INSTANCE; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/config/CursorVerification.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/config/CursorVerification.java new file mode 100644 index 000000000..4a3b93c4a --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/config/CursorVerification.java @@ -0,0 +1,32 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.config; + +import org.apache.flink.annotation.PublicEvolving; + +/** The enum class for defining the cursor verify behavior. */ +@PublicEvolving +public enum CursorVerification { + + /** We would just fail the consuming. */ + FAIL_ON_MISMATCH, + + /** Print a warn message and start consuming from the valid offset. */ + WARN_ON_MISMATCH, +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/config/PulsarSourceConfigUtils.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/config/PulsarSourceConfigUtils.java new file mode 100644 index 000000000..67e2bf2d1 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/config/PulsarSourceConfigUtils.java @@ -0,0 +1,197 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.config; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableList; +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableSet; +import org.apache.flink.util.Preconditions; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.DeadLetterPolicy; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; + +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static java.util.concurrent.TimeUnit.MICROSECONDS; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarConfigUtils.setOptionValue; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAMS; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAM_MAP; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_ACKNOWLEDGEMENTS_GROUP_TIME_MICROS; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_ACK_RECEIPT_ENABLED; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_ACK_TIMEOUT_MILLIS; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_AUTO_ACK_OLDEST_CHUNKED_MESSAGE_ON_QUEUE_FULL; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_AUTO_UPDATE_PARTITIONS_INTERVAL_SECONDS; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_CONSUMER_NAME; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_CONSUMER_PROPERTIES; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_CRYPTO_FAILURE_ACTION; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_DEAD_LETTER_TOPIC; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_EXPIRE_TIME_OF_INCOMPLETE_CHUNKED_MESSAGE_MILLIS; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_PENDING_CHUNKED_MESSAGE; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_REDELIVER_COUNT; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_TOTAL_RECEIVER_QUEUE_SIZE_ACROSS_PARTITIONS; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_NEGATIVE_ACK_REDELIVERY_DELAY_MICROS; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_POOL_MESSAGES; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_PRIORITY_LEVEL; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_COMPACTED; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_RECEIVER_QUEUE_SIZE; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_REPLICATE_SUBSCRIPTION_STATE; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_RETRY_ENABLE; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_RETRY_LETTER_TOPIC; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_INITIAL_POSITION; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_MODE; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; +import static com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions.PULSAR_TICK_DURATION_MILLIS; + +/** Create source related {@link Consumer} and validate config. */ +@Internal +public final class PulsarSourceConfigUtils { + + private PulsarSourceConfigUtils() { + // No need to create instance. + } + + private static final List>> CONFLICT_SOURCE_OPTIONS = + ImmutableList.>>builder() + .add(ImmutableSet.of(PULSAR_AUTH_PARAMS, PULSAR_AUTH_PARAM_MAP)) + .build(); + + private static final Set> REQUIRED_SOURCE_OPTIONS = + ImmutableSet.>builder() + .add(PULSAR_SERVICE_URL) + .add(PULSAR_ADMIN_URL) + .add(PULSAR_SUBSCRIPTION_NAME) + .build(); + + /** + * Helper method for checking client related config options. We would validate: + * + *

    + *
  • If user have provided the required client config options. + *
  • If user have provided some conflict options. + *
+ */ + public static void checkConfigurations(Configuration configuration) { + REQUIRED_SOURCE_OPTIONS.forEach( + option -> + Preconditions.checkArgument( + configuration.contains(option), + "Config option %s is not provided for pulsar source.", + option)); + + CONFLICT_SOURCE_OPTIONS.forEach( + options -> { + long nums = options.stream().filter(configuration::contains).count(); + Preconditions.checkArgument( + nums <= 1, + "Conflict config options %s were provided, we only support one of them for creating pulsar source.", + options); + }); + } + + /** Create a pulsar consumer builder by using the given Configuration. */ + public static ConsumerBuilder createConsumerBuilder( + PulsarClient client, Schema schema, Configuration configuration) { + ConsumerBuilder builder = client.newConsumer(schema); + + setOptionValue(configuration, PULSAR_SUBSCRIPTION_NAME, builder::subscriptionName); + setOptionValue( + configuration, PULSAR_ACK_TIMEOUT_MILLIS, v -> builder.ackTimeout(v, MILLISECONDS)); + setOptionValue(configuration, PULSAR_ACK_RECEIPT_ENABLED, builder::isAckReceiptEnabled); + setOptionValue( + configuration, + PULSAR_TICK_DURATION_MILLIS, + v -> builder.ackTimeoutTickTime(v, MILLISECONDS)); + setOptionValue( + configuration, + PULSAR_NEGATIVE_ACK_REDELIVERY_DELAY_MICROS, + v -> builder.negativeAckRedeliveryDelay(v, MICROSECONDS)); + setOptionValue(configuration, PULSAR_SUBSCRIPTION_TYPE, builder::subscriptionType); + setOptionValue(configuration, PULSAR_SUBSCRIPTION_MODE, builder::subscriptionMode); + setOptionValue(configuration, PULSAR_CRYPTO_FAILURE_ACTION, builder::cryptoFailureAction); + setOptionValue(configuration, PULSAR_RECEIVER_QUEUE_SIZE, builder::receiverQueueSize); + setOptionValue( + configuration, + PULSAR_ACKNOWLEDGEMENTS_GROUP_TIME_MICROS, + v -> builder.acknowledgmentGroupTime(v, MICROSECONDS)); + setOptionValue( + configuration, + PULSAR_REPLICATE_SUBSCRIPTION_STATE, + builder::replicateSubscriptionState); + setOptionValue( + configuration, + PULSAR_MAX_TOTAL_RECEIVER_QUEUE_SIZE_ACROSS_PARTITIONS, + builder::maxTotalReceiverQueueSizeAcrossPartitions); + setOptionValue(configuration, PULSAR_CONSUMER_NAME, builder::consumerName); + setOptionValue(configuration, PULSAR_READ_COMPACTED, builder::readCompacted); + setOptionValue(configuration, PULSAR_PRIORITY_LEVEL, builder::priorityLevel); + setOptionValue(configuration, PULSAR_CONSUMER_PROPERTIES, builder::properties); + setOptionValue( + configuration, + PULSAR_SUBSCRIPTION_INITIAL_POSITION, + builder::subscriptionInitialPosition); + createDeadLetterPolicy(configuration).ifPresent(builder::deadLetterPolicy); + setOptionValue( + configuration, + PULSAR_AUTO_UPDATE_PARTITIONS_INTERVAL_SECONDS, + v -> builder.autoUpdatePartitionsInterval(v, SECONDS)); + setOptionValue(configuration, PULSAR_RETRY_ENABLE, builder::enableRetry); + setOptionValue( + configuration, + PULSAR_MAX_PENDING_CHUNKED_MESSAGE, + builder::maxPendingChunkedMessage); + setOptionValue( + configuration, + PULSAR_AUTO_ACK_OLDEST_CHUNKED_MESSAGE_ON_QUEUE_FULL, + builder::autoAckOldestChunkedMessageOnQueueFull); + setOptionValue( + configuration, + PULSAR_EXPIRE_TIME_OF_INCOMPLETE_CHUNKED_MESSAGE_MILLIS, + v -> builder.expireTimeOfIncompleteChunkedMessage(v, MILLISECONDS)); + setOptionValue(configuration, PULSAR_POOL_MESSAGES, builder::poolMessages); + + return builder; + } + + private static Optional createDeadLetterPolicy(Configuration configuration) { + if (configuration.contains(PULSAR_MAX_REDELIVER_COUNT) + || configuration.contains(PULSAR_RETRY_LETTER_TOPIC) + || configuration.contains(PULSAR_DEAD_LETTER_TOPIC)) { + DeadLetterPolicy.DeadLetterPolicyBuilder builder = DeadLetterPolicy.builder(); + + setOptionValue(configuration, PULSAR_MAX_REDELIVER_COUNT, builder::maxRedeliverCount); + setOptionValue(configuration, PULSAR_RETRY_LETTER_TOPIC, builder::retryLetterTopic); + setOptionValue(configuration, PULSAR_DEAD_LETTER_TOPIC, builder::deadLetterTopic); + + return Optional.of(builder.build()); + } else { + return Optional.empty(); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/config/SourceConfiguration.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/config/SourceConfiguration.java new file mode 100644 index 000000000..136f986c2 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/config/SourceConfiguration.java @@ -0,0 +1,203 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; + +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.connector.pulsar.common.config.v1.PulsarUtils; +import com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptions; +import com.bytedance.bitsail.connector.pulsar.source.PulsarSourceOptionsV1; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.CursorPosition; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.SubscriptionMode; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.io.Serializable; +import java.time.Duration; + +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarConfigUtils.getOptionValue; + +/** The configure class for pulsar source. */ +@PublicEvolving +public class SourceConfiguration implements Serializable { + private static final long serialVersionUID = 8488507275800787580L; + + /** The interval in millis for flink querying topic partition information. */ + private final long partitionDiscoveryIntervalMs; + + /** + * This is used for all subscription type. But the behavior may not be the same among them. If + * you don't enable the flink checkpoint, make sure this option is set to true. + * + *
    + *
  • {@link SubscriptionType#Shared} and {@link SubscriptionType#Key_Shared} would + * immediately acknowledge the message after consuming it. + *
  • {@link SubscriptionType#Failover} and {@link SubscriptionType#Exclusive} would perform + * a incremental acknowledge in a fixed {@link #autoCommitCursorInterval}. + *
+ */ + private final boolean enableAutoAcknowledgeMessage; + + /** + * The interval in millis for acknowledge message when you enable {@link + * #enableAutoAcknowledgeMessage} and use {@link SubscriptionType#Failover} or {@link + * SubscriptionType#Exclusive} as your consuming subscription type. + */ + private final long autoCommitCursorInterval; + + /** + * Pulsar's transaction have a timeout mechanism for uncommitted transaction. We use transaction + * for {@link SubscriptionType#Shared} and {@link SubscriptionType#Key_Shared} when user disable + * {@link #enableAutoAcknowledgeMessage} and enable flink checkpoint. Since the checkpoint + * interval couldn't be acquired from {@link SourceReaderContext#getConfiguration()}, we have to + * expose this option. Make sure this value is greater than the checkpoint interval. + */ + private final long transactionTimeoutMillis; + + /** + * The fetch time for flink split reader polling message. We would stop polling message and + * return the message in {@link RecordsWithSplitIds} when timeout or exceed the {@link + * #maxFetchRecords}. + */ + private final Duration maxFetchTime; + + /** + * The fetch counts for a split reader. We would stop polling message and return the message in + * {@link RecordsWithSplitIds} when timeout {@link #maxFetchTime} or exceed this value. + */ + private final int maxFetchRecords; + + /** Validate the {@link CursorPosition} generated by {@link StartCursor}. */ + private final CursorVerification verifyInitialOffsets; + + /** + * The pulsar's subscription name for this flink source. All the readers would share this + * subscription name. + * + * @see ConsumerBuilder#subscriptionName + */ + private final String subscriptionName; + + /** + * The pulsar's subscription type for this flink source. All the readers would share this + * subscription type. + * + * @see SubscriptionType + */ + private final SubscriptionType subscriptionType; + + /** + * The pulsar's subscription mode for this flink source. All the readers would share this + * subscription mode. + * + * @see SubscriptionMode + */ + private final SubscriptionMode subscriptionMode; + + public SourceConfiguration(Configuration configuration) { + this.partitionDiscoveryIntervalMs = + configuration.get(PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS); + this.enableAutoAcknowledgeMessage = + configuration.get(PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE); + this.autoCommitCursorInterval = configuration.get(PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL); + this.transactionTimeoutMillis = configuration.get(PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS); + this.maxFetchTime = + getOptionValue(configuration, PulsarSourceOptions.PULSAR_MAX_FETCH_TIME, Duration::ofMillis); + this.maxFetchRecords = configuration.get(PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS); + this.verifyInitialOffsets = configuration.get(PulsarSourceOptions.PULSAR_VERIFY_INITIAL_OFFSETS); + this.subscriptionName = configuration.get(PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME); + this.subscriptionType = configuration.get(PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE); + this.subscriptionMode = configuration.get(PulsarSourceOptions.PULSAR_SUBSCRIPTION_MODE); + } + + public SourceConfiguration(BitSailConfiguration configuration) { + this.partitionDiscoveryIntervalMs = + configuration.get(PulsarSourceOptionsV1.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS); + this.enableAutoAcknowledgeMessage = + configuration.get(PulsarSourceOptionsV1.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE); + this.autoCommitCursorInterval = configuration.get(PulsarSourceOptionsV1.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL); + this.transactionTimeoutMillis = configuration.get(PulsarSourceOptionsV1.PULSAR_TRANSACTION_TIMEOUT_MILLIS); + this.maxFetchTime = + PulsarUtils.getOptionValue(configuration, PulsarSourceOptionsV1.PULSAR_MAX_FETCH_TIME, Duration::ofMillis); + this.maxFetchRecords = configuration.get(PulsarSourceOptionsV1.PULSAR_MAX_FETCH_RECORDS); + this.verifyInitialOffsets = PulsarUtils.getCursorVerification(configuration.get(PulsarSourceOptionsV1.PULSAR_VERIFY_INITIAL_OFFSETS)); + this.subscriptionName = configuration.get(PulsarSourceOptionsV1.PULSAR_SUBSCRIPTION_NAME); + this.subscriptionType = PulsarUtils.getSubscriptionType(configuration.get(PulsarSourceOptionsV1.PULSAR_SUBSCRIPTION_TYPE)); + this.subscriptionMode = PulsarUtils.getSubscriptionMode(configuration.get(PulsarSourceOptionsV1.PULSAR_SUBSCRIPTION_MODE)); + } + + public boolean enablePartitionDiscovery() { + return partitionDiscoveryIntervalMs > 0; + } + + public long getPartitionDiscoveryIntervalMs() { + return partitionDiscoveryIntervalMs; + } + + public boolean isEnableAutoAcknowledgeMessage() { + return enableAutoAcknowledgeMessage; + } + + public long getAutoCommitCursorInterval() { + return autoCommitCursorInterval; + } + + public long getTransactionTimeoutMillis() { + return transactionTimeoutMillis; + } + + public Duration getMaxFetchTime() { + return maxFetchTime; + } + + public int getMaxFetchRecords() { + return maxFetchRecords; + } + + public CursorVerification getVerifyInitialOffsets() { + return verifyInitialOffsets; + } + + public String getSubscriptionName() { + return subscriptionName; + } + + public SubscriptionType getSubscriptionType() { + return subscriptionType; + } + + public SubscriptionMode getSubscriptionMode() { + return subscriptionMode; + } + + /** Convert the subscription into a readable str. */ + public String getSubscriptionDesc() { + return getSubscriptionName() + + "(" + + getSubscriptionType() + + "," + + getSubscriptionMode() + + ")"; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/coordinator/PulsarSourceEnumStateV1.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/coordinator/PulsarSourceEnumStateV1.java new file mode 100644 index 000000000..b9aa4139c --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/coordinator/PulsarSourceEnumStateV1.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.connector.pulsar.source.coordinator; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.SplitsAssignmentState; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplit; + +import java.io.Serializable; +import java.util.Map; +import java.util.Set; + +/** + * The state class for pulsar source enumerator, used for storing the split state. This class is + * managed and controlled by {@link SplitsAssignmentState}. + */ +public class PulsarSourceEnumStateV1 implements Serializable { + + /** The topic partitions that have been appended to this source. */ + private final Set appendedPartitions; + + /** + * We convert the topic partition into a split and add to this pending list for assigning to a + * reader. It is used for Key_Shared, Failover, Exclusive subscription. + */ + private final Set pendingPartitionSplits; + + /** + * It is used for Shared subscription. When a reader is crashed in Shared subscription, its + * splits would be put in here. + */ + private final Map> sharedPendingPartitionSplits; + + /** + * A {@link PulsarPartitionSplit} should be assigned for all flink readers. Using this map for + * recording assign status. + */ + private final Map> readerAssignedSplits; + + private final boolean initialized; + + public PulsarSourceEnumStateV1( + Set appendedPartitions, + Set pendingPartitionSplits, + Map> pendingSharedPartitionSplits, + Map> readerAssignedSplits, + boolean initialized) { + this.appendedPartitions = appendedPartitions; + this.pendingPartitionSplits = pendingPartitionSplits; + this.sharedPendingPartitionSplits = pendingSharedPartitionSplits; + this.readerAssignedSplits = readerAssignedSplits; + this.initialized = initialized; + } + + public Set getAppendedPartitions() { + return appendedPartitions; + } + + public Set getPendingPartitionSplits() { + return pendingPartitionSplits; + } + + public Map> getSharedPendingPartitionSplits() { + return sharedPendingPartitionSplits; + } + + public Map> getReaderAssignedSplits() { + return readerAssignedSplits; + } + + public boolean isInitialized() { + return initialized; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/coordinator/PulsarSourceSplitCoordinator.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/coordinator/PulsarSourceSplitCoordinator.java new file mode 100644 index 000000000..973677dc4 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/coordinator/PulsarSourceSplitCoordinator.java @@ -0,0 +1,265 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.coordinator; + +import com.bytedance.bitsail.base.connector.reader.v1.Boundedness; +import com.bytedance.bitsail.base.connector.reader.v1.SourceSplitCoordinator; +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.connector.pulsar.common.config.v1.PulsarUtils; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StartCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range.RangeGenerator; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplit; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.KeySharedPolicy; +import org.apache.pulsar.client.api.KeySharedPolicy.KeySharedPolicySticky; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Range; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; +import static com.bytedance.bitsail.connector.pulsar.source.config.CursorVerification.FAIL_ON_MISMATCH; +import static java.util.Collections.singletonList; + +/** The enumerator class for pulsar source. */ +@Internal +public class PulsarSourceSplitCoordinator + implements SourceSplitCoordinator { + + private static final Logger LOG = LoggerFactory.getLogger(PulsarSourceSplitCoordinator.class); + + private PulsarAdmin pulsarAdmin; + private PulsarClient pulsarClient; + private final PulsarSubscriber subscriber; + private final StartCursor startCursor; + private final RangeGenerator rangeGenerator; + private final BitSailConfiguration readerConfiguration; + private final SourceConfiguration sourceConfiguration; + private final Context context; + private final SplitsAssignmentStateV1 assignmentState; + + + public PulsarSourceSplitCoordinator(PulsarSubscriber subscriber, StartCursor startCursor, RangeGenerator rangeGenerator, BitSailConfiguration readerConfiguration, + SourceConfiguration sourceConfiguration, Context context, + SplitsAssignmentStateV1 assignmentState, Boundedness sourceBoundedness) { + this.subscriber = subscriber; + this.startCursor = startCursor; + this.rangeGenerator = rangeGenerator; + this.readerConfiguration = readerConfiguration; + this.sourceConfiguration = sourceConfiguration; + this.context = context; + this.assignmentState = assignmentState; + + } + + @Override + public void start() { + this.pulsarAdmin = PulsarUtils.createAdmin(readerConfiguration); + this.pulsarClient = PulsarUtils.createClient(readerConfiguration); + rangeGenerator.open(readerConfiguration, sourceConfiguration); + + // Check the pulsar topic information and convert it into source split. + if (sourceConfiguration.enablePartitionDiscovery()) { + LOG.info( + "Starting the PulsarSourceEnumerator for subscription {} " + + "with partition discovery interval of {} ms.", + sourceConfiguration.getSubscriptionDesc(), + sourceConfiguration.getPartitionDiscoveryIntervalMs()); + context.runAsync( + this::getSubscribedTopicPartitions, + this::checkPartitionChanges, + 0, + sourceConfiguration.getPartitionDiscoveryIntervalMs()); + } else { + LOG.info( + "Starting the PulsarSourceEnumerator for subscription {} " + + "without periodic partition discovery.", + sourceConfiguration.getSubscriptionDesc()); + context.runAsyncOnce(this::getSubscribedTopicPartitions, this::checkPartitionChanges); + } + } + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { + // the pulsar source pushes splits eagerly, rather than act upon split requests. + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + // Put the split back to current pending splits. + assignmentState.putSplitsBackToPendingList(splits, subtaskId); + + // If the failed subtask has already restarted, we need to assign pending splits to it + if (context.registeredReaders().contains(subtaskId)) { + assignPendingPartitionSplits(singletonList(subtaskId)); + } + } + + @Override + public void addReader(int subtaskId) { + LOG.debug( + "Adding reader {} to PulsarSourceEnumerator for subscription {}.", + subtaskId, + sourceConfiguration.getSubscriptionDesc()); + assignPendingPartitionSplits(singletonList(subtaskId)); + } + + @Override + public PulsarSourceEnumStateV1 snapshotState() { + return assignmentState.snapshotState(); + } + + @Override + public void close() { + if (pulsarAdmin != null) { + pulsarAdmin.close(); + } + } + + // ----------------- private methods ------------------- + + /** + * List subscribed topic partitions on Pulsar cluster. + * + *

NOTE: This method should only be invoked in the worker executor thread, because it + * requires network I/O with Pulsar cluster. + * + * @return Set of subscribed {@link TopicPartition}s + */ + private Set getSubscribedTopicPartitions() { + int parallelism = context.totalParallelism(); + Set partitions = + subscriber.getSubscribedTopicPartitions(pulsarAdmin, rangeGenerator, parallelism); + + // Seek start position for given partitions. + seekStartPosition(partitions); + + return partitions; + } + + private void seekStartPosition(Set partitions) { + ConsumerBuilder consumerBuilder = consumerBuilder(); + Set seekedTopics = new HashSet<>(); + + for (TopicPartition partition : partitions) { + String topicName = partition.getFullTopicName(); + if (!assignmentState.containsTopic(topicName) && seekedTopics.add(topicName)) { + try (Consumer consumer = + sneakyClient(() -> consumerBuilder.clone().topic(topicName).subscribe())) { + startCursor.seekPosition( + partition.getTopic(), partition.getPartitionId(), consumer); + } catch (PulsarClientException e) { + if (sourceConfiguration.getVerifyInitialOffsets() == FAIL_ON_MISMATCH) { + throw new IllegalArgumentException(e); + } else { + // WARN_ON_MISMATCH would just print this warning message. + // No need to print the stacktrace. + LOG.warn( + "Failed to set initial consuming position for partition {}", + partition, + e); + } + } + } + } + } + + private ConsumerBuilder consumerBuilder() { + ConsumerBuilder builder = + PulsarUtils.createConsumerBuilder(pulsarClient, Schema.BYTES, readerConfiguration); + if (sourceConfiguration.getSubscriptionType() == SubscriptionType.Key_Shared) { + Range range = TopicRange.createFullRange().toPulsarRange(); + KeySharedPolicySticky keySharedPolicy = KeySharedPolicy.stickyHashRange().ranges(range); + // Force this consume use sticky hash range in Key_Shared subscription. + // Pulsar won't remove old message dispatcher before 2.8.2 release. + builder.keySharedPolicy(keySharedPolicy); + } + + return builder; + } + + /** + * Check if there's any partition changes within subscribed topic partitions fetched by worker + * thread, and convert them to splits the assign them to pulsar readers. + * + *

NOTE: This method should only be invoked in the coordinator executor thread. + * + * @param fetchedPartitions Map from topic name to its description + * @param throwable Exception in worker thread + */ + private void checkPartitionChanges(Set fetchedPartitions, Throwable throwable) { + if (throwable != null) { + throw new FlinkRuntimeException( + "Failed to list subscribed topic partitions due to ", throwable); + } + + // Append the partitions into current assignment state. + assignmentState.appendTopicPartitions(fetchedPartitions); + List registeredReaders = new ArrayList<>(context.registeredReaders()); + + // Assign the new readers. + assignPendingPartitionSplits(registeredReaders); + } + + private void assignPendingPartitionSplits(List pendingReaders) { + // Validate the reader. + pendingReaders.forEach( + reader -> { + if (!context.registeredReaders().contains(reader)) { + throw new IllegalStateException( + "Reader " + reader + " is not registered to source coordinator"); + } + }); + + // Assign splits to downstream readers. + assignmentState.assignSplits(pendingReaders).ifPresent(assignment -> { + assignment.forEach(context::assignSplit); + }); + + // If periodically partition discovery is disabled and the initializing discovery has done, + // signal NoMoreSplitsEvent to pending readers + if (assignmentState.noMoreNewPartitionSplits()) { + LOG.debug( + "No more PulsarPartitionSplits to assign." + + " Sending NoMoreSplitsEvent to reader {} in subscription {}.", + pendingReaders, + sourceConfiguration.getSubscriptionDesc()); + pendingReaders.forEach(this.context::signalNoMoreSplits); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/coordinator/SplitsAssignmentStateV1.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/coordinator/SplitsAssignmentStateV1.java new file mode 100644 index 000000000..6e5ab92f2 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/coordinator/SplitsAssignmentStateV1.java @@ -0,0 +1,239 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.coordinator; + +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplit; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.util.InstantiationUtil; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +/** The state class for recording the split assignment. */ +@Internal +public class SplitsAssignmentStateV1 implements Serializable { + + private final StopCursor stopCursor; + private final SourceConfiguration sourceConfiguration; + + // The dynamic states for checkpoint. + private final Set appendedPartitions; + // This pending splits is used for Key_Shared, Failover, Exclusive subscription. + private final Set pendingPartitionSplits; + // These two fields are used for Shared subscription. + private final Map> sharedPendingPartitionSplits; + private final Map> readerAssignedSplits; + private boolean initialized; + + public SplitsAssignmentStateV1(StopCursor stopCursor, SourceConfiguration sourceConfiguration) { + this.stopCursor = stopCursor; + this.sourceConfiguration = sourceConfiguration; + this.appendedPartitions = new HashSet<>(); + this.pendingPartitionSplits = new HashSet<>(); + this.sharedPendingPartitionSplits = new HashMap<>(); + this.readerAssignedSplits = new HashMap<>(); + this.initialized = false; + } + + public SplitsAssignmentStateV1( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumStateV1 sourceEnumState) { + this.stopCursor = stopCursor; + this.sourceConfiguration = sourceConfiguration; + this.appendedPartitions = sourceEnumState.getAppendedPartitions(); + this.pendingPartitionSplits = sourceEnumState.getPendingPartitionSplits(); + this.sharedPendingPartitionSplits = sourceEnumState.getSharedPendingPartitionSplits(); + this.readerAssignedSplits = sourceEnumState.getReaderAssignedSplits(); + this.initialized = sourceEnumState.isInitialized(); + } + + public PulsarSourceEnumStateV1 snapshotState() { + return new PulsarSourceEnumStateV1( + appendedPartitions, + pendingPartitionSplits, + sharedPendingPartitionSplits, + readerAssignedSplits, + initialized); + } + + /** + * Append the new fetched partitions to current state. We would generate pending source split + * for downstream pulsar readers. Since the {@link SplitEnumeratorContext} don't support put the + * split back to enumerator, we don't support partition deletion. + * + * @param fetchedPartitions The partitions from the {@link PulsarSubscriber}. + */ + public void appendTopicPartitions(Set fetchedPartitions) { + for (TopicPartition partition : fetchedPartitions) { + // If this partition is a new partition. + if (!appendedPartitions.contains(partition)) { + if (!sharePartition()) { + // Create a split and add it to pending list. + pendingPartitionSplits.add(createSplit(partition)); + } + + // Shared subscription don't create splits, we just register partitions. + appendedPartitions.add(partition); + } + } + + // Update this initialize flag. + if (!initialized) { + this.initialized = true; + } + } + + public boolean containsTopic(String topicName) { + return appendedPartitions.stream() + .anyMatch(partition -> Objects.equals(partition.getFullTopicName(), topicName)); + } + + /** Put these splits back to pending list. */ + public void putSplitsBackToPendingList(List splits, int readerId) { + if (!sharePartition()) { + // Put these splits back to normal pending list. + pendingPartitionSplits.addAll(splits); + } else { + // Put the splits back to shared pending list. + Set pending = + sharedPendingPartitionSplits.computeIfAbsent(readerId, id -> new HashSet<>()); + pending.addAll(splits); + } + } + + public Optional>> assignSplits( + List pendingReaders) { + // Avoid empty readers assign. + if (pendingReaders.isEmpty()) { + return Optional.empty(); + } + + Map> assignMap; + + // We extract the assign logic into two method for better readability. + if (!sharePartition()) { + assignMap = assignNormalSplits(pendingReaders); + } else { + assignMap = assignSharedSplits(pendingReaders); + } + + if (assignMap.isEmpty()) { + return Optional.empty(); + } else { + return Optional.of(assignMap); + } + } + + /** + * @return It would return true only if periodically partition discovery is disabled, the + * initializing partition discovery has finished AND there is no pending splits for + * assignment. + */ + public boolean noMoreNewPartitionSplits() { + return !sourceConfiguration.enablePartitionDiscovery() + && initialized + && pendingPartitionSplits.isEmpty(); + } + + // ----------------- private methods ------------------- + + /** The splits don't shared for all the readers. */ + private Map> assignNormalSplits( + List pendingReaders) { + Map> assignMap = new HashMap<>(); + + // Drain a list of splits. + List pendingSplits = drainPendingPartitionsSplits(); + for (int i = 0; i < pendingSplits.size(); i++) { + PulsarPartitionSplit split = pendingSplits.get(i); + int readerId = pendingReaders.get(i % pendingReaders.size()); + assignMap.computeIfAbsent(readerId, id -> new ArrayList<>()).add(split); + } + + return assignMap; + } + + /** Every split would be shared among available readers. */ + private Map> assignSharedSplits( + List pendingReaders) { + Map> assignMap = new HashMap<>(); + + // Drain the splits from share pending list. + for (Integer reader : pendingReaders) { + Set pendingSplits = sharedPendingPartitionSplits.remove(reader); + if (pendingSplits == null) { + pendingSplits = new HashSet<>(); + } + + Set assignedSplits = + readerAssignedSplits.computeIfAbsent(reader, r -> new HashSet<>()); + + for (TopicPartition partition : appendedPartitions) { + String partitionName = partition.toString(); + if (!assignedSplits.contains(partitionName)) { + pendingSplits.add(createSplit(partition)); + assignedSplits.add(partitionName); + } + } + + if (!pendingSplits.isEmpty()) { + assignMap.put(reader, new ArrayList<>(pendingSplits)); + } + } + + return assignMap; + } + + private PulsarPartitionSplit createSplit(TopicPartition partition) { + try { + StopCursor stop = InstantiationUtil.clone(stopCursor); + return new PulsarPartitionSplit(partition, stop); + } catch (IOException | ClassNotFoundException e) { + throw new IllegalStateException(e); + } + } + + private List drainPendingPartitionsSplits() { + List splits = new ArrayList<>(pendingPartitionSplits); + pendingPartitionSplits.clear(); + + return splits; + } + + /** {@link SubscriptionType#Shared} mode should share a same split for all the readers. */ + private boolean sharePartition() { + return sourceConfiguration.getSubscriptionType() == SubscriptionType.Shared; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/PulsarSourceEnumState.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/PulsarSourceEnumState.java new file mode 100644 index 000000000..67c9c402d --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/PulsarSourceEnumState.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 com.bytedance.bitsail.connector.pulsar.source.enumerator; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; + +import java.io.Serializable; +import java.util.Map; +import java.util.Set; + +/** + * The state class for pulsar source enumerator, used for storing the split state. This class is + * managed and controlled by {@link SplitsAssignmentState}. + */ +public class PulsarSourceEnumState implements Serializable { + + /** The topic partitions that have been appended to this source. */ + private final Set appendedPartitions; + + /** + * We convert the topic partition into a split and add to this pending list for assigning to a + * reader. It is used for Key_Shared, Failover, Exclusive subscription. + */ + private final Set pendingPartitionSplits; + + /** + * It is used for Shared subscription. When a reader is crashed in Shared subscription, its + * splits would be put in here. + */ + private final Map> sharedPendingPartitionSplits; + + /** + * A {@link PulsarPartitionSplit} should be assigned for all flink readers. Using this map for + * recording assign status. + */ + private final Map> readerAssignedSplits; + + private final boolean initialized; + + public PulsarSourceEnumState( + Set appendedPartitions, + Set pendingPartitionSplits, + Map> pendingSharedPartitionSplits, + Map> readerAssignedSplits, + boolean initialized) { + this.appendedPartitions = appendedPartitions; + this.pendingPartitionSplits = pendingPartitionSplits; + this.sharedPendingPartitionSplits = pendingSharedPartitionSplits; + this.readerAssignedSplits = readerAssignedSplits; + this.initialized = initialized; + } + + public Set getAppendedPartitions() { + return appendedPartitions; + } + + public Set getPendingPartitionSplits() { + return pendingPartitionSplits; + } + + public Map> getSharedPendingPartitionSplits() { + return sharedPendingPartitionSplits; + } + + public Map> getReaderAssignedSplits() { + return readerAssignedSplits; + } + + public boolean isInitialized() { + return initialized; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/PulsarSourceEnumStateSerializer.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/PulsarSourceEnumStateSerializer.java new file mode 100644 index 000000000..5404fe1b6 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/PulsarSourceEnumStateSerializer.java @@ -0,0 +1,122 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplitSerializer; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.function.FunctionWithException; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.Map; +import java.util.Set; + +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarSerdeUtils.deserializeMap; +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarSerdeUtils.deserializeSet; +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarSerdeUtils.serializeMap; +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarSerdeUtils.serializeSet; + +/** The {@link SimpleVersionedSerializer Serializer} for the enumerator state of Pulsar source. */ +public class PulsarSourceEnumStateSerializer + implements SimpleVersionedSerializer { + + public static final PulsarSourceEnumStateSerializer INSTANCE = + new PulsarSourceEnumStateSerializer(); + + private static final PulsarPartitionSplitSerializer SPLIT_SERIALIZER = + PulsarPartitionSplitSerializer.INSTANCE; + + private PulsarSourceEnumStateSerializer() { + // Singleton instance. + } + + @Override + public int getVersion() { + // We use PulsarPartitionSplitSerializer's version because we use reuse this class. + return PulsarPartitionSplitSerializer.CURRENT_VERSION; + } + + @Override + public byte[] serialize(PulsarSourceEnumState obj) throws IOException { + // VERSION 0 serialization + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + serializeSet( + out, obj.getAppendedPartitions(), SPLIT_SERIALIZER::serializeTopicPartition); + serializeSet( + out, + obj.getPendingPartitionSplits(), + SPLIT_SERIALIZER::serializePulsarPartitionSplit); + serializeMap( + out, + obj.getSharedPendingPartitionSplits(), + DataOutputStream::writeInt, + (o, v) -> serializeSet(o, v, SPLIT_SERIALIZER::serializePulsarPartitionSplit)); + serializeMap( + out, + obj.getReaderAssignedSplits(), + DataOutputStream::writeInt, + (o, v) -> serializeSet(o, v, DataOutputStream::writeUTF)); + out.writeBoolean(obj.isInitialized()); + + out.flush(); + return baos.toByteArray(); + } + } + + @Override + public PulsarSourceEnumState deserialize(int version, byte[] serialized) throws IOException { + // VERSION 0 deserialization + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + Set partitions = deserializeSet(in, deserializePartition(version)); + Set splits = deserializeSet(in, deserializeSplit(version)); + Map> sharedSplits = + deserializeMap( + in, + DataInput::readInt, + i -> deserializeSet(i, deserializeSplit(version))); + Map> mapping = + deserializeMap( + in, DataInput::readInt, i -> deserializeSet(i, DataInput::readUTF)); + boolean initialized = in.readBoolean(); + + return new PulsarSourceEnumState( + partitions, splits, sharedSplits, mapping, initialized); + } + } + + // ----------------- private methods ------------------- + + private FunctionWithException + deserializePartition(int version) { + return in -> SPLIT_SERIALIZER.deserializeTopicPartition(version, in); + } + + private FunctionWithException + deserializeSplit(int version) { + return in -> SPLIT_SERIALIZER.deserializePulsarPartitionSplit(version, in); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java new file mode 100644 index 000000000..34d5a72d0 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java @@ -0,0 +1,269 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; + +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StartCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range.RangeGenerator; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.KeySharedPolicy; +import org.apache.pulsar.client.api.KeySharedPolicy.KeySharedPolicySticky; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Range; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static java.util.Collections.singletonList; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarConfigUtils.createAdmin; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarConfigUtils.createClient; +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; +import static com.bytedance.bitsail.connector.pulsar.source.config.CursorVerification.FAIL_ON_MISMATCH; +import static com.bytedance.bitsail.connector.pulsar.source.config.PulsarSourceConfigUtils.createConsumerBuilder; + +/** The enumerator class for pulsar source. */ +@Internal +public class PulsarSourceEnumerator + implements SplitEnumerator { + + private static final Logger LOG = LoggerFactory.getLogger(org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumerator.class); + + private final PulsarAdmin pulsarAdmin; + private final PulsarClient pulsarClient; + private final PulsarSubscriber subscriber; + private final StartCursor startCursor; + private final RangeGenerator rangeGenerator; + private final Configuration configuration; + private final SourceConfiguration sourceConfiguration; + private final SplitEnumeratorContext context; + private final SplitsAssignmentState assignmentState; + + public PulsarSourceEnumerator( + PulsarSubscriber subscriber, + StartCursor startCursor, + RangeGenerator rangeGenerator, + Configuration configuration, + SourceConfiguration sourceConfiguration, + SplitEnumeratorContext context, + SplitsAssignmentState assignmentState) { + this.pulsarAdmin = createAdmin(configuration); + this.pulsarClient = createClient(configuration); + this.subscriber = subscriber; + this.startCursor = startCursor; + this.rangeGenerator = rangeGenerator; + this.configuration = configuration; + this.sourceConfiguration = sourceConfiguration; + this.context = context; + this.assignmentState = assignmentState; + } + + @Override + public void start() { + rangeGenerator.open(configuration, sourceConfiguration); + + // Check the pulsar topic information and convert it into source split. + if (sourceConfiguration.enablePartitionDiscovery()) { + LOG.info( + "Starting the PulsarSourceEnumerator for subscription {} " + + "with partition discovery interval of {} ms.", + sourceConfiguration.getSubscriptionDesc(), + sourceConfiguration.getPartitionDiscoveryIntervalMs()); + context.callAsync( + this::getSubscribedTopicPartitions, + this::checkPartitionChanges, + 0, + sourceConfiguration.getPartitionDiscoveryIntervalMs()); + } else { + LOG.info( + "Starting the PulsarSourceEnumerator for subscription {} " + + "without periodic partition discovery.", + sourceConfiguration.getSubscriptionDesc()); + context.callAsync(this::getSubscribedTopicPartitions, this::checkPartitionChanges); + } + } + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { + // the pulsar source pushes splits eagerly, rather than act upon split requests. + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + // Put the split back to current pending splits. + assignmentState.putSplitsBackToPendingList(splits, subtaskId); + + // If the failed subtask has already restarted, we need to assign pending splits to it + if (context.registeredReaders().containsKey(subtaskId)) { + assignPendingPartitionSplits(singletonList(subtaskId)); + } + } + + @Override + public void addReader(int subtaskId) { + LOG.debug( + "Adding reader {} to PulsarSourceEnumerator for subscription {}.", + subtaskId, + sourceConfiguration.getSubscriptionDesc()); + assignPendingPartitionSplits(singletonList(subtaskId)); + } + + @Override + public PulsarSourceEnumState snapshotState() { + return assignmentState.snapshotState(); + } + + @Override + public void close() { + if (pulsarAdmin != null) { + pulsarAdmin.close(); + } + } + + // ----------------- private methods ------------------- + + /** + * List subscribed topic partitions on Pulsar cluster. + * + *

NOTE: This method should only be invoked in the worker executor thread, because it + * requires network I/O with Pulsar cluster. + * + * @return Set of subscribed {@link TopicPartition}s + */ + private Set getSubscribedTopicPartitions() { + int parallelism = context.currentParallelism(); + Set partitions = + subscriber.getSubscribedTopicPartitions(pulsarAdmin, rangeGenerator, parallelism); + + // Seek start position for given partitions. + seekStartPosition(partitions); + + return partitions; + } + + private void seekStartPosition(Set partitions) { + ConsumerBuilder consumerBuilder = consumerBuilder(); + Set seekedTopics = new HashSet<>(); + + for (TopicPartition partition : partitions) { + String topicName = partition.getFullTopicName(); + if (!assignmentState.containsTopic(topicName) && seekedTopics.add(topicName)) { + try (Consumer consumer = + sneakyClient(() -> consumerBuilder.clone().topic(topicName).subscribe())) { + startCursor.seekPosition( + partition.getTopic(), partition.getPartitionId(), consumer); + } catch (PulsarClientException e) { + if (sourceConfiguration.getVerifyInitialOffsets() == FAIL_ON_MISMATCH) { + throw new IllegalArgumentException(e); + } else { + // WARN_ON_MISMATCH would just print this warning message. + // No need to print the stacktrace. + LOG.warn( + "Failed to set initial consuming position for partition {}", + partition, + e); + } + } + } + } + } + + private ConsumerBuilder consumerBuilder() { + ConsumerBuilder builder = + createConsumerBuilder(pulsarClient, Schema.BYTES, configuration); + if (sourceConfiguration.getSubscriptionType() == SubscriptionType.Key_Shared) { + Range range = TopicRange.createFullRange().toPulsarRange(); + KeySharedPolicySticky keySharedPolicy = KeySharedPolicy.stickyHashRange().ranges(range); + // Force this consume use sticky hash range in Key_Shared subscription. + // Pulsar won't remove old message dispatcher before 2.8.2 release. + builder.keySharedPolicy(keySharedPolicy); + } + + return builder; + } + + /** + * Check if there's any partition changes within subscribed topic partitions fetched by worker + * thread, and convert them to splits the assign them to pulsar readers. + * + *

NOTE: This method should only be invoked in the coordinator executor thread. + * + * @param fetchedPartitions Map from topic name to its description + * @param throwable Exception in worker thread + */ + private void checkPartitionChanges(Set fetchedPartitions, Throwable throwable) { + if (throwable != null) { + throw new FlinkRuntimeException( + "Failed to list subscribed topic partitions due to ", throwable); + } + + // Append the partitions into current assignment state. + assignmentState.appendTopicPartitions(fetchedPartitions); + List registeredReaders = new ArrayList<>(context.registeredReaders().keySet()); + + // Assign the new readers. + assignPendingPartitionSplits(registeredReaders); + } + + private void assignPendingPartitionSplits(List pendingReaders) { + // Validate the reader. + pendingReaders.forEach( + reader -> { + if (!context.registeredReaders().containsKey(reader)) { + throw new IllegalStateException( + "Reader " + reader + " is not registered to source coordinator"); + } + }); + + // Assign splits to downstream readers. + assignmentState.assignSplits(pendingReaders).ifPresent(context::assignSplits); + + // If periodically partition discovery is disabled and the initializing discovery has done, + // signal NoMoreSplitsEvent to pending readers + if (assignmentState.noMoreNewPartitionSplits()) { + LOG.debug( + "No more PulsarPartitionSplits to assign." + + " Sending NoMoreSplitsEvent to reader {} in subscription {}.", + pendingReaders, + sourceConfiguration.getSubscriptionDesc()); + pendingReaders.forEach(this.context::signalNoMoreSplits); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/SplitsAssignmentState.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/SplitsAssignmentState.java new file mode 100644 index 000000000..f4768a77b --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/SplitsAssignmentState.java @@ -0,0 +1,238 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.flink.util.InstantiationUtil; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +/** The state class for recording the split assignment. */ +@Internal +public class SplitsAssignmentState { + + private final StopCursor stopCursor; + private final SourceConfiguration sourceConfiguration; + + // The dynamic states for checkpoint. + private final Set appendedPartitions; + // This pending splits is used for Key_Shared, Failover, Exclusive subscription. + private final Set pendingPartitionSplits; + // These two fields are used for Shared subscription. + private final Map> sharedPendingPartitionSplits; + private final Map> readerAssignedSplits; + private boolean initialized; + + public SplitsAssignmentState(StopCursor stopCursor, SourceConfiguration sourceConfiguration) { + this.stopCursor = stopCursor; + this.sourceConfiguration = sourceConfiguration; + this.appendedPartitions = new HashSet<>(); + this.pendingPartitionSplits = new HashSet<>(); + this.sharedPendingPartitionSplits = new HashMap<>(); + this.readerAssignedSplits = new HashMap<>(); + this.initialized = false; + } + + public SplitsAssignmentState( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + this.stopCursor = stopCursor; + this.sourceConfiguration = sourceConfiguration; + this.appendedPartitions = sourceEnumState.getAppendedPartitions(); + this.pendingPartitionSplits = sourceEnumState.getPendingPartitionSplits(); + this.sharedPendingPartitionSplits = sourceEnumState.getSharedPendingPartitionSplits(); + this.readerAssignedSplits = sourceEnumState.getReaderAssignedSplits(); + this.initialized = sourceEnumState.isInitialized(); + } + + public PulsarSourceEnumState snapshotState() { + return new PulsarSourceEnumState( + appendedPartitions, + pendingPartitionSplits, + sharedPendingPartitionSplits, + readerAssignedSplits, + initialized); + } + + /** + * Append the new fetched partitions to current state. We would generate pending source split + * for downstream pulsar readers. Since the {@link SplitEnumeratorContext} don't support put the + * split back to enumerator, we don't support partition deletion. + * + * @param fetchedPartitions The partitions from the {@link PulsarSubscriber}. + */ + public void appendTopicPartitions(Set fetchedPartitions) { + for (TopicPartition partition : fetchedPartitions) { + // If this partition is a new partition. + if (!appendedPartitions.contains(partition)) { + if (!sharePartition()) { + // Create a split and add it to pending list. + pendingPartitionSplits.add(createSplit(partition)); + } + + // Shared subscription don't create splits, we just register partitions. + appendedPartitions.add(partition); + } + } + + // Update this initialize flag. + if (!initialized) { + this.initialized = true; + } + } + + public boolean containsTopic(String topicName) { + return appendedPartitions.stream() + .anyMatch(partition -> Objects.equals(partition.getFullTopicName(), topicName)); + } + + /** Put these splits back to pending list. */ + public void putSplitsBackToPendingList(List splits, int readerId) { + if (!sharePartition()) { + // Put these splits back to normal pending list. + pendingPartitionSplits.addAll(splits); + } else { + // Put the splits back to shared pending list. + Set pending = + sharedPendingPartitionSplits.computeIfAbsent(readerId, id -> new HashSet<>()); + pending.addAll(splits); + } + } + + public Optional> assignSplits( + List pendingReaders) { + // Avoid empty readers assign. + if (pendingReaders.isEmpty()) { + return Optional.empty(); + } + + Map> assignMap; + + // We extract the assign logic into two method for better readability. + if (!sharePartition()) { + assignMap = assignNormalSplits(pendingReaders); + } else { + assignMap = assignSharedSplits(pendingReaders); + } + + if (assignMap.isEmpty()) { + return Optional.empty(); + } else { + return Optional.of(new SplitsAssignment<>(assignMap)); + } + } + + /** + * @return It would return true only if periodically partition discovery is disabled, the + * initializing partition discovery has finished AND there is no pending splits for + * assignment. + */ + public boolean noMoreNewPartitionSplits() { + return !sourceConfiguration.enablePartitionDiscovery() + && initialized + && pendingPartitionSplits.isEmpty(); + } + + // ----------------- private methods ------------------- + + /** The splits don't shared for all the readers. */ + private Map> assignNormalSplits( + List pendingReaders) { + Map> assignMap = new HashMap<>(); + + // Drain a list of splits. + List pendingSplits = drainPendingPartitionsSplits(); + for (int i = 0; i < pendingSplits.size(); i++) { + PulsarPartitionSplit split = pendingSplits.get(i); + int readerId = pendingReaders.get(i % pendingReaders.size()); + assignMap.computeIfAbsent(readerId, id -> new ArrayList<>()).add(split); + } + + return assignMap; + } + + /** Every split would be shared among available readers. */ + private Map> assignSharedSplits( + List pendingReaders) { + Map> assignMap = new HashMap<>(); + + // Drain the splits from share pending list. + for (Integer reader : pendingReaders) { + Set pendingSplits = sharedPendingPartitionSplits.remove(reader); + if (pendingSplits == null) { + pendingSplits = new HashSet<>(); + } + + Set assignedSplits = + readerAssignedSplits.computeIfAbsent(reader, r -> new HashSet<>()); + + for (TopicPartition partition : appendedPartitions) { + String partitionName = partition.toString(); + if (!assignedSplits.contains(partitionName)) { + pendingSplits.add(createSplit(partition)); + assignedSplits.add(partitionName); + } + } + + if (!pendingSplits.isEmpty()) { + assignMap.put(reader, new ArrayList<>(pendingSplits)); + } + } + + return assignMap; + } + + private PulsarPartitionSplit createSplit(TopicPartition partition) { + try { + StopCursor stop = InstantiationUtil.clone(stopCursor); + return new PulsarPartitionSplit(partition, stop); + } catch (IOException | ClassNotFoundException e) { + throw new IllegalStateException(e); + } + } + + private List drainPendingPartitionsSplits() { + List splits = new ArrayList<>(pendingPartitionSplits); + pendingPartitionSplits.clear(); + + return splits; + } + + /** {@link SubscriptionType#Shared} mode should share a same split for all the readers. */ + private boolean sharePartition() { + return sourceConfiguration.getSubscriptionType() == SubscriptionType.Shared; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/CursorPosition.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/CursorPosition.java new file mode 100644 index 000000000..31eabf21d --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/CursorPosition.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 com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClientException; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +/** The class for defining the start or stop position. */ +@PublicEvolving +public final class CursorPosition implements Serializable { + private static final long serialVersionUID = -802405183307684549L; + + private final Type type; + + private final MessageId messageId; + + private final Long timestamp; + + public CursorPosition(@Nullable MessageId messageId) { + this.type = Type.MESSAGE_ID; + this.messageId = messageId; + this.timestamp = null; + } + + public CursorPosition(@Nullable Long timestamp) { + this.type = Type.TIMESTAMP; + this.messageId = null; + this.timestamp = timestamp; + } + + @VisibleForTesting + public MessageId getMessageId() { + return messageId; + } + + /** Pulsar consumer could be subscribed by the position. */ + public void seekPosition(Consumer consumer) throws PulsarClientException { + if (type == Type.MESSAGE_ID) { + consumer.seek(messageId); + } else { + if (timestamp != null) { + consumer.seek(timestamp); + } else { + consumer.seek(System.currentTimeMillis()); + } + } + } + + @Override + public String toString() { + if (type == Type.TIMESTAMP) { + return "timestamp: " + timestamp; + } else { + return "message id: " + messageId; + } + } + + /** + * The position type for reader to choose whether timestamp or message id as the start position. + */ + public enum Type { + TIMESTAMP, + + MESSAGE_ID + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/StartCursor.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/StartCursor.java new file mode 100644 index 000000000..fedbd46a6 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/StartCursor.java @@ -0,0 +1,81 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor; + +import org.apache.flink.annotation.PublicEvolving; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.start.TimestampStartCursor; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.io.Serializable; + +/** + * A interface for users to specify the start position of a pulsar subscription. Since it would be + * serialized into split. The implementation for this interface should be well considered. I don't + * recommend adding extra internal state for this implementation. + * + *

This class would be used only for {@link SubscriptionType#Exclusive} and {@link + * SubscriptionType#Failover}. + */ +@PublicEvolving +@FunctionalInterface +public interface StartCursor extends Serializable { + + CursorPosition position(String topic, int partitionId); + + /** Helper method for seek the right position for given pulsar consumer. */ + default void seekPosition(String topic, int partitionId, Consumer consumer) + throws PulsarClientException { + CursorPosition position = position(topic, partitionId); + position.seekPosition(consumer); + } + + // --------------------------- Static Factory Methods ----------------------------- + + static StartCursor defaultStartCursor() { + return earliest(); + } + + static StartCursor earliest() { + return fromMessageId(MessageId.earliest); + } + + static StartCursor latest() { + return fromMessageId(MessageId.latest); + } + + static StartCursor fromMessageId(MessageId messageId) { + return fromMessageId(messageId, true); + } + + /** + * @param messageId Find the available message id and start consuming from it. + * @param inclusive {@code ture} would include the given message id. + */ + static StartCursor fromMessageId(MessageId messageId, boolean inclusive) { + return new MessageIdStartCursor(messageId, inclusive); + } + + static StartCursor fromMessageTime(long timestamp) { + return new TimestampStartCursor(timestamp); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/StopCursor.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/StopCursor.java new file mode 100644 index 000000000..0f89d6ec8 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/StopCursor.java @@ -0,0 +1,76 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor; + +import org.apache.flink.annotation.PublicEvolving; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.stop.LatestMessageStopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.stop.MessageIdStopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.stop.NeverStopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.stop.TimestampStopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; + +import java.io.Serializable; + +/** + * A interface for users to specify the stop position of a pulsar subscription. Since it would be + * serialized into split. The implementation for this interface should be well considered. I don't + * recommend adding extra internal state for this implementation. + */ +@PublicEvolving +@FunctionalInterface +public interface StopCursor extends Serializable { + + /** The open method for the cursor initializer. This method could be executed multiple times. */ + default void open(PulsarAdmin admin, TopicPartition partition) {} + + /** + * Determine whether to pause consumption on the current message by the returned boolean value. + * The message presented in method argument wouldn't be consumed if the return result is true. + */ + boolean shouldStop(Message message); + + // --------------------------- Static Factory Methods ----------------------------- + + static StopCursor defaultStopCursor() { + return never(); + } + + static StopCursor never() { + return new NeverStopCursor(); + } + + static StopCursor latest() { + return new LatestMessageStopCursor(); + } + + static StopCursor atMessageId(MessageId messageId) { + return new MessageIdStopCursor(messageId); + } + + static StopCursor afterMessageId(MessageId messageId) { + return new MessageIdStopCursor(messageId, false); + } + + static StopCursor atEventTime(long timestamp) { + return new TimestampStopCursor(timestamp); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java new file mode 100644 index 000000000..1a515fdd2 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java @@ -0,0 +1,65 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.start; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.CursorPosition; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.impl.MessageIdImpl; + +import static org.apache.flink.util.Preconditions.checkState; + +/** This cursor would left pulsar start consuming from a specific message id. */ +public class MessageIdStartCursor implements StartCursor { + private static final long serialVersionUID = -8057345435887170111L; + + private final MessageId messageId; + + /** + * The default {@code inclusive} behavior should be controlled in {@link + * ConsumerBuilder#startMessageIdInclusive}. But pulsar has a bug and don't support this + * currently. We have to use {@code entry + 1} policy for consuming the next available message. + * If the message id entry is not valid. Pulsar would automatically find next valid message id. + * Please referer this + * code for understanding pulsar internal logic. + * + * @param messageId The message id for start position. + * @param inclusive Should we include the start message id in consuming result. + */ + public MessageIdStartCursor(MessageId messageId, boolean inclusive) { + if (inclusive) { + this.messageId = messageId; + } else { + checkState( + messageId instanceof MessageIdImpl, + "We only support normal message id and batch message id."); + MessageIdImpl id = (MessageIdImpl) messageId; + this.messageId = + new MessageIdImpl( + id.getLedgerId(), id.getEntryId() + 1, id.getPartitionIndex()); + } + } + + @Override + public CursorPosition position(String topic, int partitionId) { + return new CursorPosition(messageId); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java new file mode 100644 index 000000000..29bcbe60a --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java @@ -0,0 +1,38 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.start; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.CursorPosition; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StartCursor; + +/** This cursor would left pulsar start consuming from a specific timestamp. */ +public class TimestampStartCursor implements StartCursor { + private static final long serialVersionUID = 5170578885838095320L; + + private final long timestamp; + + public TimestampStartCursor(long timestamp) { + this.timestamp = timestamp; + } + + @Override + public CursorPosition position(String topic, int partitionId) { + return new CursorPosition(timestamp); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java new file mode 100644 index 000000000..db8118269 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java @@ -0,0 +1,51 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.stop; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; + +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; + +/** + * A stop cursor that initialize the position to the latest message id. The offsets initialization + * are taken care of by the {@code PulsarPartitionSplitReaderBase} instead of by the {@code + * PulsarSourceEnumerator}. + */ +public class LatestMessageStopCursor implements StopCursor { + + private MessageId messageId; + + @Override + public void open(PulsarAdmin admin, TopicPartition partition) { + if (messageId == null) { + String topic = partition.getFullTopicName(); + messageId = sneakyAdmin(() -> admin.topics().getLastMessageId(topic)); + } + } + + @Override + public boolean shouldStop(Message message) { + MessageId id = message.getMessageId(); + return id.compareTo(messageId) >= 0; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java new file mode 100644 index 000000000..76b597d76 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java @@ -0,0 +1,54 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.stop; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; + +/** + * Stop consuming message at a given message id. We use the {@link MessageId#compareTo(Object)} for + * compare the consuming message with the given message id. + */ +public class MessageIdStopCursor implements StopCursor { + private static final long serialVersionUID = -3990454110809274542L; + + private final MessageId messageId; + + private final boolean exclusive; + + public MessageIdStopCursor(MessageId messageId) { + this(messageId, true); + } + + public MessageIdStopCursor(MessageId messageId, boolean exclusive) { + this.messageId = messageId; + this.exclusive = exclusive; + } + + @Override + public boolean shouldStop(Message message) { + MessageId id = message.getMessageId(); + if (exclusive) { + return id.compareTo(messageId) > 0; + } else { + return id.compareTo(messageId) >= 0; + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java new file mode 100644 index 000000000..2e6958bcb --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java @@ -0,0 +1,32 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.stop; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.pulsar.client.api.Message; + +/** A implementation which wouldn't stop forever. */ +public class NeverStopCursor implements StopCursor { + private static final long serialVersionUID = -3113601090292771786L; + + @Override + public boolean shouldStop(Message message) { + return false; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/TimestampStopCursor.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/TimestampStopCursor.java new file mode 100644 index 000000000..78abf601d --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/cursor/stop/TimestampStopCursor.java @@ -0,0 +1,38 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.stop; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.pulsar.client.api.Message; + +/** Stop consuming message at the given event time. */ +public class TimestampStopCursor implements StopCursor { + private static final long serialVersionUID = 3381576769339353027L; + + private final long timestamp; + + public TimestampStopCursor(long timestamp) { + this.timestamp = timestamp; + } + + @Override + public boolean shouldStop(Message message) { + return message.getEventTime() >= timestamp; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java new file mode 100644 index 000000000..a57cfc443 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java @@ -0,0 +1,70 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber; + +import org.apache.flink.annotation.Internal; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.impl.TopicListSubscriber; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.impl.TopicPatternSubscriber; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range.RangeGenerator; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.RegexSubscriptionMode; + +import java.io.Serializable; +import java.util.List; +import java.util.Set; +import java.util.regex.Pattern; + +/** + * Pulsar consumer allows a few different ways to consume from the topics, including: + * + *

    + *
  1. Subscribe from a collection of topics. + *
  2. Subscribe to a topic pattern using Java {@code Regex}. + *
+ * + *

The PulsarSubscriber provides a unified interface for the Pulsar source to support all these + * two types of subscribing mode. + */ +@Internal +public interface PulsarSubscriber extends Serializable { + + /** + * Get a set of subscribed {@link TopicPartition}s. The method could throw {@link + * IllegalStateException}, a extra try catch is required. + * + * @param pulsarAdmin The admin interface used to retrieve subscribed topic partitions. + * @param rangeGenerator The range for different partitions. + * @param parallelism The parallelism of flink source. + * @return A subscribed {@link TopicPartition} for each pulsar topic partition. + */ + Set getSubscribedTopicPartitions( + PulsarAdmin pulsarAdmin, RangeGenerator rangeGenerator, int parallelism); + + // ----------------- factory methods -------------- + + static PulsarSubscriber getTopicListSubscriber(List topics) { + return new TopicListSubscriber(topics); + } + + static PulsarSubscriber getTopicPatternSubscriber( + Pattern topicPattern, RegexSubscriptionMode subscriptionMode) { + return new TopicPatternSubscriber(topicPattern, subscriptionMode); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java new file mode 100644 index 000000000..8d54f9321 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java @@ -0,0 +1,79 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.impl; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicMetadata; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicNameUtils; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; + +import java.util.List; +import java.util.stream.IntStream; + +import static java.util.stream.Collectors.toList; + +/** PulsarSubscriber abstract class to simplify Pulsar admin related operations. */ +public abstract class BasePulsarSubscriber implements PulsarSubscriber { + private static final long serialVersionUID = 2053021503331058888L; + + protected TopicMetadata queryTopicMetadata(PulsarAdmin pulsarAdmin, String topicName) { + // Drop the complete topic name for a clean partitioned topic name. + String completeTopicName = TopicNameUtils.topicName(topicName); + try { + PartitionedTopicMetadata metadata = + pulsarAdmin.topics().getPartitionedTopicMetadata(completeTopicName); + return new TopicMetadata(topicName, metadata.partitions); + } catch (PulsarAdminException e) { + if (e.getStatusCode() == 404) { + // Return null for skipping the topic metadata query. + return null; + } else { + // This method would cause the failure for subscriber. + throw new IllegalStateException(e); + } + } + } + + protected List toTopicPartitions( + TopicMetadata metadata, List ranges) { + if (!metadata.isPartitioned()) { + // For non-partitioned topic. + return ranges.stream() + .map(range -> new TopicPartition(metadata.getName(), -1, range)) + .collect(toList()); + } else { + return IntStream.range(0, metadata.getPartitionSize()) + .boxed() + .flatMap( + partitionId -> + ranges.stream() + .map( + range -> + new TopicPartition( + metadata.getName(), + partitionId, + range))) + .collect(toList()); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/impl/TopicListSubscriber.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/impl/TopicListSubscriber.java new file mode 100644 index 000000000..d1f39bf67 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/impl/TopicListSubscriber.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.impl; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range.RangeGenerator; +import org.apache.pulsar.client.admin.PulsarAdmin; + +import java.util.List; +import java.util.Objects; +import java.util.Set; + +import static java.util.stream.Collectors.toSet; + +/** the implements of consuming multiple topics. */ +public class TopicListSubscriber extends BasePulsarSubscriber { + private static final long serialVersionUID = 6473918213832993116L; + + private final List topics; + + public TopicListSubscriber(List topics) { + this.topics = topics; + } + + @Override + public Set getSubscribedTopicPartitions( + PulsarAdmin pulsarAdmin, RangeGenerator rangeGenerator, int parallelism) { + + return topics.parallelStream() + .map(topic -> queryTopicMetadata(pulsarAdmin, topic)) + .filter(Objects::nonNull) + .flatMap( + metadata -> { + List ranges = rangeGenerator.range(metadata, parallelism); + return toTopicPartitions(metadata, ranges).stream(); + }) + .collect(toSet()); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java new file mode 100644 index 000000000..a1ed80e62 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java @@ -0,0 +1,104 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.subscriber.impl; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range.RangeGenerator; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.RegexSubscriptionMode; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; + +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.regex.Pattern; + +import static java.util.stream.Collectors.toSet; + +/** Subscribe to matching topics based on topic pattern. */ +public class TopicPatternSubscriber extends BasePulsarSubscriber { + private static final long serialVersionUID = 3307710093243745104L; + + private final Pattern topicPattern; + private final RegexSubscriptionMode subscriptionMode; + private final String namespace; + + public TopicPatternSubscriber(Pattern topicPattern, RegexSubscriptionMode subscriptionMode) { + this.topicPattern = topicPattern; + this.subscriptionMode = subscriptionMode; + + // Extract the namespace from topic pattern regex. + // If no namespace provided in the regex, we would directly use "default" as the namespace. + TopicName destination = TopicName.get(topicPattern.toString()); + NamespaceName namespaceName = destination.getNamespaceObject(); + this.namespace = namespaceName.toString(); + } + + @Override + public Set getSubscribedTopicPartitions( + PulsarAdmin pulsarAdmin, RangeGenerator rangeGenerator, int parallelism) { + try { + return pulsarAdmin + .namespaces() + .getTopics(namespace) + .parallelStream() + .filter(this::matchesSubscriptionMode) + .filter(topic -> topicPattern.matcher(topic).find()) + .map(topic -> queryTopicMetadata(pulsarAdmin, topic)) + .filter(Objects::nonNull) + .flatMap( + metadata -> { + List ranges = + rangeGenerator.range(metadata, parallelism); + return toTopicPartitions(metadata, ranges).stream(); + }) + .collect(toSet()); + } catch (PulsarAdminException e) { + if (e.getStatusCode() == 404) { + // Skip the topic metadata query. + return Collections.emptySet(); + } else { + // This method would cause the failure for subscriber. + throw new IllegalStateException(e); + } + } + } + + /** + * Filter the topic by regex subscription mode. This logic is the same as pulsar consumer's + * regex subscription. + */ + private boolean matchesSubscriptionMode(String topic) { + TopicName topicName = TopicName.get(topic); + // Filter the topic persistence. + switch (subscriptionMode) { + case PersistentOnly: + return topicName.isPersistent(); + case NonPersistentOnly: + return !topicName.isPersistent(); + default: + // RegexSubscriptionMode.AllTopics + return true; + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicMetadata.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicMetadata.java new file mode 100644 index 000000000..81b67fa36 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicMetadata.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 com.bytedance.bitsail.connector.pulsar.source.enumerator.topic; + +import org.apache.flink.annotation.PublicEvolving; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.pulsar.common.partition.PartitionedTopicMetadata.NON_PARTITIONED; + +/** The pojo class for pulsar topic metadata information. */ +@PublicEvolving +public final class TopicMetadata { + + /** + * The name of the topic, it would be a {@link TopicNameUtils#topicName(String)} which don't + * contain partition information. + */ + private final String name; + + /** The size for a partitioned topic. It would be zero for non-partitioned topic. */ + private final int partitionSize; + + public TopicMetadata(String name, int partitionSize) { + checkArgument(partitionSize >= 0); + + this.name = name; + this.partitionSize = partitionSize; + } + + public String getName() { + return name; + } + + public boolean isPartitioned() { + return partitionSize != NON_PARTITIONED; + } + + public int getPartitionSize() { + return partitionSize; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicNameUtils.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicNameUtils.java new file mode 100644 index 000000000..d0815d4c1 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicNameUtils.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.connector.pulsar.source.enumerator.topic; + +import org.apache.flink.annotation.Internal; +import org.apache.pulsar.common.naming.TopicName; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** util for topic name. */ +@Internal +public final class TopicNameUtils { + + private TopicNameUtils() { + // No public constructor. + } + + /** Ensure the given topic name should be a topic without partition information. */ + public static String topicName(String topic) { + return TopicName.get(topic).getPartitionedTopicName(); + } + + /** Create a topic name with partition information. */ + public static String topicNameWithPartition(String topic, int partitionId) { + checkArgument(partitionId >= 0, "Illegal partition id %s", partitionId); + return TopicName.get(topic).getPartition(partitionId).toString(); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicPartition.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicPartition.java new file mode 100644 index 000000000..7b1b3a998 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicPartition.java @@ -0,0 +1,117 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.topic; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.pulsar.client.api.Range; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.io.Serializable; +import java.util.Objects; + +import static com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; +import static com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Topic partition is the basic topic information used by {@link SplitReader}, we create this topic + * metas for a specified topic by subscription type and convert it into a partition split. + */ +@Internal +public class TopicPartition implements Serializable { + private static final long serialVersionUID = -1474354741550810953L; + + /** + * The topic name of the pulsar. It would be a full topic name, if your don't provide the tenant + * and namespace, we would add them automatically. + */ + private final String topic; + + /** + * Index of partition for the topic. It would be natural number for partitioned topic with a + * non-key_shared subscription. + */ + private final int partitionId; + + /** + * The ranges for this topic, used for limiting consume scope. It would be a {@link + * TopicRange#createFullRange()} full range for all the subscription type except {@link + * SubscriptionType#Key_Shared}. + */ + private final TopicRange range; + + public TopicPartition(String topic, int partitionId, TopicRange range) { + this.topic = topicName(checkNotNull(topic)); + this.partitionId = partitionId; + this.range = checkNotNull(range); + } + + public String getTopic() { + return topic; + } + + public int getPartitionId() { + return partitionId; + } + + /** + * Pulsar split the topic partition into a bunch of small topics, we would get the real topic + * name by using this method. + */ + public String getFullTopicName() { + if (partitionId >= 0) { + return topicNameWithPartition(topic, partitionId); + } else { + return topic; + } + } + + public TopicRange getRange() { + return range; + } + + public Range getPulsarRange() { + return range.toPulsarRange(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TopicPartition that = (TopicPartition) o; + return partitionId == that.partitionId + && topic.equals(that.topic) + && range.equals(that.range); + } + + @Override + public int hashCode() { + return Objects.hash(topic, partitionId, range); + } + + @Override + public String toString() { + return getFullTopicName() + "|" + range; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicRange.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicRange.java new file mode 100644 index 000000000..b03e4b089 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/TopicRange.java @@ -0,0 +1,104 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.topic; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.pulsar.client.api.KeySharedPolicy; +import org.apache.pulsar.client.api.Range; + +import java.io.Serializable; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * This class is used to define the range for KeyShared subscription. + * + *

{@link Range} is a hash scope for pulsar message key. The total hash range size is 65536, so + * the end of the range should be 65535, the start the range should be 0. + * + * @see KeySharedPolicy.KeySharedPolicySticky + */ +@PublicEvolving +public class TopicRange implements Serializable { + private static final long serialVersionUID = 3176938692775594400L; + + public static final int RANGE_SIZE = KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; + + /** The start position for hash range. */ + public static final int MIN_RANGE = 0; + + /** The end position for hash range, it's 65535. */ + public static final int MAX_RANGE = RANGE_SIZE - 1; + + /** The start of the range, default is zero. */ + private final int start; + + /** The end of the range, included. */ + private final int end; + + public TopicRange(int start, int end) { + checkArgument(start >= MIN_RANGE, "Start range %s shouldn't below zero.", start); + checkArgument(end <= MAX_RANGE, "End range %s shouldn't exceed 65535.", end); + checkArgument(start <= end, "Range end must >= range start."); + + this.start = start; + this.end = end; + } + + /** Convert to pulsar's {@link Range} API for consuming in client. */ + public Range toPulsarRange() { + return new Range(start, end); + } + + /** Create a topic range which contains the fully hash range. */ + public static TopicRange createFullRange() { + return new TopicRange(MIN_RANGE, MAX_RANGE); + } + + public int getStart() { + return start; + } + + public int getEnd() { + return end; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TopicRange that = (TopicRange) o; + return start == that.start && end == that.end; + } + + @Override + public int hashCode() { + return Objects.hash(start, end); + } + + @Override + public String toString() { + return getStart() + "-" + getEnd(); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/FixedRangeGenerator.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/FixedRangeGenerator.java new file mode 100644 index 000000000..2c8c15a6b --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/FixedRangeGenerator.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicMetadata; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange; + +import java.util.List; + +/** Always return the same range set for all topics. */ +public class FixedRangeGenerator implements RangeGenerator { + private static final long serialVersionUID = -3895203007855538734L; + + private final List ranges; + + public FixedRangeGenerator(List ranges) { + this.ranges = ranges; + } + + @Override + public List range(TopicMetadata metadata, int parallelism) { + return ranges; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/FullRangeGenerator.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/FullRangeGenerator.java new file mode 100644 index 000000000..c28cb538b --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/FullRangeGenerator.java @@ -0,0 +1,39 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicMetadata; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.Collections; +import java.util.List; + +/** + * Default implementation for {@link SubscriptionType#Shared}, {@link SubscriptionType#Failover} and + * {@link SubscriptionType#Exclusive} subscription. + */ +public class FullRangeGenerator implements RangeGenerator { + private static final long serialVersionUID = -4571731955155036216L; + + @Override + public List range(TopicMetadata metadata, int parallelism) { + return Collections.singletonList(TopicRange.createFullRange()); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/RangeGenerator.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/RangeGenerator.java new file mode 100644 index 000000000..0ffa39d91 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/RangeGenerator.java @@ -0,0 +1,62 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range; + +import org.apache.flink.annotation.PublicEvolving; + +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicMetadata; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange; + +import org.apache.flink.configuration.Configuration; +import org.apache.pulsar.client.api.KeySharedPolicy; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.io.Serializable; +import java.util.List; + +/** + * A generator for generating the {@link TopicRange} for given topic. It was used for pulsar's + * {@link SubscriptionType#Key_Shared} mode. {@link TopicRange} would be used in {@link + * KeySharedPolicy} for different pulsar source readers. + * + *

If you implement this interface, make sure that each {@link TopicRange} would be assigned to a + * specified source reader. Since flink parallelism is provided, make sure the pulsar message key's + * hashcode is evenly distributed among these topic ranges. + */ +@PublicEvolving +@FunctionalInterface +public interface RangeGenerator extends Serializable { + + /** + * Generate range for the given topic. + * + * @param metadata The metadata of the topic. + * @param parallelism The reader size for this topic. + */ + List range(TopicMetadata metadata, int parallelism); + + default void open(Configuration configuration, SourceConfiguration sourceConfiguration) { + // This method is used for user implementation. + } + default void open(BitSailConfiguration configuration, SourceConfiguration sourceConfiguration) { + // This method is used for user implementation. + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/UniformRangeGenerator.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/UniformRangeGenerator.java new file mode 100644 index 000000000..88903a4c4 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/enumerator/topic/range/UniformRangeGenerator.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.range; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicMetadata; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.ArrayList; +import java.util.List; + +import static com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange.MAX_RANGE; +import static com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange.RANGE_SIZE; + +/** + * This range generator would divide the range by the flink source parallelism. It would be the + * default implementation for {@link SubscriptionType#Key_Shared} subscription. + */ +public class UniformRangeGenerator implements RangeGenerator { + private static final long serialVersionUID = -7292650922683609268L; + + @Override + public List range(TopicMetadata metadata, int parallelism) { + List results = new ArrayList<>(parallelism); + + int startRange = 0; + for (int i = 1; i < parallelism; i++) { + int nextStartRange = i * RANGE_SIZE / parallelism; + results.add(new TopicRange(startRange, nextStartRange - 1)); + startRange = nextStartRange; + } + results.add(new TopicRange(startRange, MAX_RANGE)); + + return results; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/PulsarSourceReaderFactory.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/PulsarSourceReaderFactory.java new file mode 100644 index 000000000..00bd6feaa --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/PulsarSourceReaderFactory.java @@ -0,0 +1,135 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; + +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; +import com.bytedance.bitsail.connector.pulsar.source.reader.source.PulsarUnorderedSourceReader; +import com.bytedance.bitsail.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; +import com.bytedance.bitsail.connector.pulsar.source.reader.split.PulsarUnorderedPartitionSplitReader; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; +import org.apache.pulsar.client.impl.PulsarClientImpl; + +import java.util.function.Supplier; + +import static org.apache.flink.connector.base.source.reader.SourceReaderOptions.ELEMENT_QUEUE_CAPACITY; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarConfigUtils.createAdmin; +import static com.bytedance.bitsail.connector.pulsar.common.config.PulsarConfigUtils.createClient; + +/** + * This factory class is used for creating different types of source reader for different + * subscription type. + * + *

    + *
  1. Failover, Exclusive: We would create {@link PulsarOrderedSourceReader}. + *
  2. Shared, Key_Shared: We would create {@link PulsarUnorderedSourceReader}. + *
+ */ +@Internal +public final class PulsarSourceReaderFactory { + + private PulsarSourceReaderFactory() { + // No public constructor. + } + + @SuppressWarnings("java:S2095") + public static SourceReader create( + SourceReaderContext readerContext, + PulsarDeserializationSchema deserializationSchema, + Configuration configuration, + SourceConfiguration sourceConfiguration) { + + PulsarClient pulsarClient = createClient(configuration); + PulsarAdmin pulsarAdmin = createAdmin(configuration); + + // Create a message queue with the predefined source option. + int queueSize = configuration.getInteger(ELEMENT_QUEUE_CAPACITY); + FutureCompletingBlockingQueue>> elementsQueue = + new FutureCompletingBlockingQueue<>(queueSize); + + // Create different pulsar source reader by subscription type. + SubscriptionType subscriptionType = sourceConfiguration.getSubscriptionType(); + if (subscriptionType == SubscriptionType.Failover + || subscriptionType == SubscriptionType.Exclusive) { + // Create a ordered split reader supplier. + Supplier> splitReaderSupplier = + () -> + new PulsarOrderedPartitionSplitReader<>( + pulsarClient, + pulsarAdmin, + configuration, + sourceConfiguration, + deserializationSchema); + + return new PulsarOrderedSourceReader<>( + elementsQueue, + splitReaderSupplier, + configuration, + readerContext, + sourceConfiguration, + pulsarClient, + pulsarAdmin); + } else if (subscriptionType == SubscriptionType.Shared + || subscriptionType == SubscriptionType.Key_Shared) { + TransactionCoordinatorClient coordinatorClient = + ((PulsarClientImpl) pulsarClient).getTcClient(); + if (coordinatorClient == null + && !sourceConfiguration.isEnableAutoAcknowledgeMessage()) { + throw new IllegalStateException("Transaction is required but didn't enabled"); + } + + Supplier> splitReaderSupplier = + () -> + new PulsarUnorderedPartitionSplitReader<>( + pulsarClient, + pulsarAdmin, + configuration, + sourceConfiguration, + deserializationSchema, + coordinatorClient); + + return new PulsarUnorderedSourceReader<>( + elementsQueue, + splitReaderSupplier, + configuration, + readerContext, + sourceConfiguration, + pulsarClient, + pulsarAdmin, + coordinatorClient); + } else { + throw new UnsupportedOperationException( + "This subscription type is not " + subscriptionType + " supported currently."); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java new file mode 100644 index 000000000..57db78405 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java @@ -0,0 +1,127 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.deserializer; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.DeserializationSchema.InitializationContext; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchema; +import org.apache.flink.util.Collector; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.schema.KeyValue; + +import java.io.Serializable; + +/** + * A schema bridge for deserializing the pulsar's {@code Message} into a flink managed + * instance. We support both the pulsar's self managed schema and flink managed schema. + * + * @param The output message type for sinking to downstream flink operator. + */ +@PublicEvolving +public interface PulsarDeserializationSchema extends Serializable, ResultTypeQueryable { + + /** + * Initialization method for the schema. It is called before the actual working methods {@link + * #deserialize} and thus suitable for one time setup work. + * + *

The provided {@link InitializationContext} can be used to access additional features such + * as e.g. registering user metrics. + * + * @param context Contextual information that can be used during initialization. + */ + default void open(InitializationContext context) throws Exception { + // Nothing to do here for the default implementation. + } + + /** + * Deserializes the pulsar message. This message could be a raw byte message or some parsed + * message which decoded by pulsar schema. + * + *

You can output multiple message by using the {@link Collector}. Note that number and size + * of the produced records should be relatively small. Depending on the source implementation + * records can be buffered in memory or collecting records might delay emitting checkpoint + * barrier. + * + * @param message The message decoded by pulsar. + * @param out The collector to put the resulting messages. + */ + void deserialize(Message message, Collector out) throws Exception; + + /** + * Create a PulsarDeserializationSchema by using the flink's {@link DeserializationSchema}. It + * would consume the pulsar message as byte array and decode the message by using flink's logic. + */ + static PulsarDeserializationSchema flinkSchema( + DeserializationSchema deserializationSchema) { + return new PulsarDeserializationSchemaWrapper<>(deserializationSchema); + } + + /** + * Create a PulsarDeserializationSchema by using the Pulsar {@link Schema} instance. The message + * bytes must be encoded by pulsar Schema. + * + *

We only support primitive + * types here. + */ + static PulsarDeserializationSchema pulsarSchema(Schema schema) { + PulsarSchema pulsarSchema = new PulsarSchema<>(schema); + return new PulsarSchemaWrapper<>(pulsarSchema); + } + + /** + * Create a PulsarDeserializationSchema by using the Pulsar {@link Schema} instance. The message + * bytes must be encoded by pulsar Schema. + * + *

We only support struct types here. + */ + static PulsarDeserializationSchema pulsarSchema(Schema schema, Class typeClass) { + PulsarSchema pulsarSchema = new PulsarSchema<>(schema, typeClass); + return new PulsarSchemaWrapper<>(pulsarSchema); + } + + /** + * Create a PulsarDeserializationSchema by using the Pulsar {@link Schema} instance. The message + * bytes must be encoded by pulsar Schema. + * + *

We only support keyvalue types here. + */ + static PulsarDeserializationSchema> pulsarSchema( + Schema> schema, Class keyClass, Class valueClass) { + PulsarSchema> pulsarSchema = + new PulsarSchema<>(schema, keyClass, valueClass); + return new PulsarSchemaWrapper<>(pulsarSchema); + } + + /** + * Create a PulsarDeserializationSchema by using the given {@link TypeInformation}. This method + * is only used for treating message that was written into pulsar by {@link TypeInformation}. + */ + static PulsarDeserializationSchema flinkTypeInfo( + TypeInformation information, ExecutionConfig config) { + return new PulsarTypeInformationWrapper<>(information, config); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaInitializationContext.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaInitializationContext.java new file mode 100644 index 000000000..85d064e98 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaInitializationContext.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 com.bytedance.bitsail.connector.pulsar.source.reader.deserializer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.metrics.MetricGroup; + +/** + * Convert the {@link SourceReaderContext} into a {@link + * DeserializationSchema.InitializationContext}, we would use a pulsar named metric group for this + * content. + */ +@Internal +public class PulsarDeserializationSchemaInitializationContext + implements DeserializationSchema.InitializationContext { + + private final SourceReaderContext readerContext; + + public PulsarDeserializationSchemaInitializationContext(SourceReaderContext readerContext) { + this.readerContext = readerContext; + } + + @Override + public MetricGroup getMetricGroup() { + return readerContext.metricGroup().addGroup("pulsarDeserializer"); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java new file mode 100644 index 000000000..408838d6f --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.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 com.bytedance.bitsail.connector.pulsar.source.reader.deserializer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.DeserializationSchema.InitializationContext; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.util.Collector; +import org.apache.pulsar.client.api.Message; + +/** + * A {@link PulsarDeserializationSchema} implementation which based on the given flink's {@link + * DeserializationSchema}. We would consume the message as byte array from pulsar and deserialize it + * by using flink serialization logic. + * + * @param The output type of the message. + */ +@Internal +class PulsarDeserializationSchemaWrapper implements PulsarDeserializationSchema { + private static final long serialVersionUID = -630646912412751300L; + + private final DeserializationSchema deserializationSchema; + + public PulsarDeserializationSchemaWrapper(DeserializationSchema deserializationSchema) { + this.deserializationSchema = deserializationSchema; + } + + @Override + public void open(InitializationContext context) throws Exception { + // Initialize it for some custom logic. + deserializationSchema.open(context); + } + + @Override + public void deserialize(Message message, Collector out) throws Exception { + byte[] bytes = message.getData(); + T instance = deserializationSchema.deserialize(bytes); + + out.collect(instance); + } + + @Override + public TypeInformation getProducedType() { + return deserializationSchema.getProducedType(); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java new file mode 100644 index 000000000..a6fb3b11b --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java @@ -0,0 +1,71 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.deserializer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema.InitializationContext; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchema; +import org.apache.flink.util.Collector; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.schema.SchemaInfo; + +import static com.bytedance.bitsail.connector.pulsar.common.schema.PulsarSchemaUtils.createTypeInformation; + +/** + * The deserialization schema wrapper for pulsar original {@link Schema}. Pulsar would deserialize + * the message and pass it to flink with a auto generate or given {@link TypeInformation}. + * + * @param The output type of the message. + */ +@Internal +class PulsarSchemaWrapper implements PulsarDeserializationSchema { + private static final long serialVersionUID = -4864701207257059158L; + + /** The serializable pulsar schema, it wrap the schema with type class. */ + private final PulsarSchema pulsarSchema; + + @SuppressWarnings("java:S2065") + private transient Schema schema; + + public PulsarSchemaWrapper(PulsarSchema pulsarSchema) { + this.pulsarSchema = pulsarSchema; + } + + @Override + public void open(InitializationContext context) throws Exception { + if (schema == null) { + this.schema = pulsarSchema.getPulsarSchema(); + } + } + + @Override + public void deserialize(Message message, Collector out) throws Exception { + byte[] bytes = message.getData(); + T instance = schema.decode(bytes); + + out.collect(instance); + } + + @Override + public TypeInformation getProducedType() { + SchemaInfo info = pulsarSchema.getSchemaInfo(); + return createTypeInformation(info); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java new file mode 100644 index 000000000..01ea25851 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.connector.pulsar.source.reader.deserializer; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.util.Collector; +import org.apache.pulsar.client.api.Message; + +/** + * Wrap the flink TypeInformation into a {@code PulsarDeserializationSchema}. We would create a + * flink {@code TypeSerializer} by using given ExecutionConfig. + */ +public class PulsarTypeInformationWrapper implements PulsarDeserializationSchema { + private static final long serialVersionUID = 6647084180084963022L; + + /** + * PulsarDeserializationSchema would be shared for multiple SplitReaders in different fetcher + * thread. Use a thread-local DataInputDeserializer would be better. + */ + @SuppressWarnings("java:S5164") + private static final ThreadLocal DESERIALIZER = + ThreadLocal.withInitial(DataInputDeserializer::new); + + private final TypeInformation information; + private final TypeSerializer serializer; + + public PulsarTypeInformationWrapper(TypeInformation information, ExecutionConfig config) { + this.information = information; + this.serializer = information.createSerializer(config); + } + + @Override + public void deserialize(Message message, Collector out) throws Exception { + DataInputDeserializer dis = DESERIALIZER.get(); + dis.setBuffer(message.getData()); + T instance = serializer.deserialize(dis); + + out.collect(instance); + } + + @Override + public TypeInformation getProducedType() { + return information; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/v1/PulsarDeserializationSchema.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/v1/PulsarDeserializationSchema.java new file mode 100644 index 000000000..2f16ce8a5 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/deserializer/v1/PulsarDeserializationSchema.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 com.bytedance.bitsail.connector.pulsar.source.reader.deserializer.v1; + +import com.bytedance.bitsail.base.format.DeserializationSchema; +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.common.row.Row; +import com.bytedance.bitsail.common.typeinfo.TypeInfo; +import com.bytedance.bitsail.component.format.json.JsonDeserializationSchema; + +public class PulsarDeserializationSchema implements DeserializationSchema { + + private BitSailConfiguration deserializationConfiguration; + + private TypeInfo[] typeInfos; + + private String[] fieldNames; + + private transient JsonDeserializationSchema deserializationSchema; + + public PulsarDeserializationSchema(BitSailConfiguration deserializationConfiguration, + TypeInfo[] typeInfos, + String[] fieldNames) { + this.deserializationConfiguration = deserializationConfiguration; + this.typeInfos = typeInfos; + this.fieldNames = fieldNames; + //todo spi. + this.deserializationSchema = new JsonDeserializationSchema(deserializationConfiguration, + typeInfos, + fieldNames); + } + + @Override + public Row deserialize(byte[] message) { + return deserializationSchema.deserialize(message); + } + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/emitter/PulsarRecordEmitter.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/emitter/PulsarRecordEmitter.java new file mode 100644 index 000000000..a26e73864 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/emitter/PulsarRecordEmitter.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 com.bytedance.bitsail.connector.pulsar.source.reader.emitter; + +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; +import com.bytedance.bitsail.connector.pulsar.source.reader.source.PulsarUnorderedSourceReader; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplitState; + +/** + * The {@link RecordEmitter} implementation for both {@link PulsarOrderedSourceReader} and {@link + * PulsarUnorderedSourceReader}. We would always update the last consumed message id in this + * emitter. + */ +public class PulsarRecordEmitter + implements RecordEmitter, T, PulsarPartitionSplitState> { + + @Override + public void emitRecord( + PulsarMessage element, SourceOutput output, PulsarPartitionSplitState splitState) + throws Exception { + // Sink the record to source output. + output.collect(element.getValue(), element.getEventTime()); + // Update the split state. + splitState.setLatestConsumedId(element.getId()); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java new file mode 100644 index 000000000..84ca3d2ce --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java @@ -0,0 +1,121 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.fetcher; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SourceReaderBase; +import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager; +import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; + +import java.lang.reflect.Field; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; + +import static java.util.Collections.singletonList; + +/** + * Common fetcher manager abstraction for both ordered & unordered message. + * + * @param The decoded message type for flink. + */ +@Internal +public abstract class PulsarFetcherManagerBase + extends SingleThreadFetcherManager, PulsarPartitionSplit> { + + private final Map splitFetcherMapping = new HashMap<>(); + private final Map fetcherStatus = new HashMap<>(); + + /** + * Creates a new SplitFetcherManager with multiple I/O threads. + * + * @param elementsQueue The queue that is used to hand over data from the I/O thread (the + * fetchers) to the reader (which emits the records and book-keeps the state. This must be + * the same queue instance that is also passed to the {@link SourceReaderBase}. + * @param splitReaderSupplier The factory for the split reader that connects to the source + */ + protected PulsarFetcherManagerBase( + FutureCompletingBlockingQueue>> elementsQueue, + Supplier, PulsarPartitionSplit>> splitReaderSupplier) { + super(elementsQueue, splitReaderSupplier); + } + + /** + * Override this method for supporting multiple thread fetching, one fetcher thread for one + * split. + */ + @Override + public void addSplits(List splitsToAdd) { + for (PulsarPartitionSplit split : splitsToAdd) { + SplitFetcher, PulsarPartitionSplit> fetcher = + getOrCreateFetcher(split.splitId()); + fetcher.addSplits(singletonList(split)); + // This method could be executed multiple times. + startFetcher(fetcher); + } + } + + @Override + protected void startFetcher(SplitFetcher, PulsarPartitionSplit> fetcher) { + int id = getFetcherId(fetcher); + + if (fetcherStatus.get(id) != Boolean.TRUE) { + fetcherStatus.put(id, true); + super.startFetcher(fetcher); + } + } + + private int getFetcherId(SplitFetcher, PulsarPartitionSplit> fetcher) { + int id = -1; + try { + Field idField; + idField = fetcher.getClass().getDeclaredField("id"); + idField.setAccessible(true); + id = (int) idField.get(fetcher); + } catch (NoSuchFieldException | IllegalAccessException ignored) { + } + return id; + } + + protected SplitFetcher, PulsarPartitionSplit> getOrCreateFetcher( + String splitId) { + SplitFetcher, PulsarPartitionSplit> fetcher; + Integer fetcherId = splitFetcherMapping.get(splitId); + + if (fetcherId == null) { + fetcher = createSplitFetcher(); + } else { + fetcher = fetchers.get(fetcherId); + // This fetcher has been stopped. + if (fetcher == null) { + fetcherStatus.remove(fetcherId); + fetcher = createSplitFetcher(); + } + } + splitFetcherMapping.put(splitId, getFetcherId(fetcher)); + + return fetcher; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/fetcher/PulsarOrderedFetcherManager.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/fetcher/PulsarOrderedFetcherManager.java new file mode 100644 index 000000000..48404c263 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/fetcher/PulsarOrderedFetcherManager.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 com.bytedance.bitsail.connector.pulsar.source.reader.fetcher; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.MessageId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.function.Supplier; + +/** + * Pulsar's FetcherManager implementation for ordered consuming. This class is needed to help + * acknowledge the message to Pulsar using the {@link Consumer} inside the {@link + * PulsarOrderedPartitionSplitReader}. + * + * @param The message type for pulsar decoded message. + */ +@Internal +public class PulsarOrderedFetcherManager extends PulsarFetcherManagerBase { + private static final Logger LOG = LoggerFactory.getLogger(PulsarOrderedFetcherManager.class); + + public PulsarOrderedFetcherManager( + FutureCompletingBlockingQueue>> elementsQueue, + Supplier, PulsarPartitionSplit>> splitReaderSupplier) { + super(elementsQueue, splitReaderSupplier); + } + + public void acknowledgeMessages(Map cursorsToCommit) { + LOG.debug("Acknowledge messages {}", cursorsToCommit); + cursorsToCommit.forEach( + (partition, messageId) -> { + SplitFetcher, PulsarPartitionSplit> fetcher = + getOrCreateFetcher(partition.toString()); + triggerAcknowledge(fetcher, partition, messageId); + }); + } + + private void triggerAcknowledge( + SplitFetcher, PulsarPartitionSplit> splitFetcher, + TopicPartition partition, + MessageId messageId) { + PulsarOrderedPartitionSplitReader splitReader = + (PulsarOrderedPartitionSplitReader) splitFetcher.getSplitReader(); + splitReader.notifyCheckpointComplete(partition, messageId); + startFetcher(splitFetcher); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/fetcher/PulsarUnorderedFetcherManager.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/fetcher/PulsarUnorderedFetcherManager.java new file mode 100644 index 000000000..30071f6cf --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/fetcher/PulsarUnorderedFetcherManager.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.bytedance.bitsail.connector.pulsar.source.reader.fetcher; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.reader.split.PulsarUnorderedPartitionSplitReader; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.pulsar.client.api.Consumer; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Supplier; + +import static java.util.stream.Collectors.toCollection; + +/** + * Pulsar's FetcherManager implementation for unordered consuming. This class is needed to help + * acknowledge the message to Pulsar using the {@link Consumer} inside the {@link + * PulsarUnorderedPartitionSplitReader}. + * + * @param The message type for pulsar decoded message. + */ +@Internal +public class PulsarUnorderedFetcherManager extends PulsarFetcherManagerBase { + + public PulsarUnorderedFetcherManager( + FutureCompletingBlockingQueue>> elementsQueue, + Supplier, PulsarPartitionSplit>> splitReaderSupplier) { + super(elementsQueue, splitReaderSupplier); + } + + public List snapshotState(long checkpointId) { + return fetchers.values().stream() + .map(SplitFetcher::getSplitReader) + .map(splitReader -> snapshotReader(checkpointId, splitReader)) + .collect(toCollection(() -> new ArrayList<>(fetchers.size()))); + } + + private PulsarPartitionSplit snapshotReader( + long checkpointId, SplitReader, PulsarPartitionSplit> splitReader) { + return ((PulsarUnorderedPartitionSplitReader) splitReader) + .snapshotState(checkpointId) + .toPulsarPartitionSplit(); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/message/PulsarMessage.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/message/PulsarMessage.java new file mode 100644 index 000000000..fe9ce8180 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/message/PulsarMessage.java @@ -0,0 +1,73 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.message; + +import org.apache.flink.annotation.Internal; +import com.bytedance.bitsail.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; + +/** + * The message instance that contains the required information which would be used for committing + * the consuming status. + */ +@Internal +public class PulsarMessage { + + /** + * The id of a given message. This id could be same for multiple {@link PulsarMessage}, although + * it is unique for every {@link Message}. + */ + private final MessageId id; + + /** The value which deserialized by {@link PulsarDeserializationSchema}. */ + private final T value; + + /** The produce time for this message, it's a event time. */ + private final long eventTime; + + public PulsarMessage(MessageId id, T value, long eventTime) { + this.id = id; + this.value = value; + this.eventTime = eventTime; + } + + public MessageId getId() { + return id; + } + + public T getValue() { + return value; + } + + public long getEventTime() { + return eventTime; + } + + @Override + public String toString() { + return "PulsarMessage{" + + "id=" + + id + + ", value=" + + value + + ", eventTime=" + + eventTime + + '}'; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/message/PulsarMessageCollector.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/message/PulsarMessageCollector.java new file mode 100644 index 000000000..873149171 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/message/PulsarMessageCollector.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 com.bytedance.bitsail.connector.pulsar.source.reader.message; + +import org.apache.flink.connector.base.source.reader.RecordsBySplits; +import com.bytedance.bitsail.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.util.Collector; +import org.apache.pulsar.client.api.Message; + +/** + * This collector supplier is providing the {@link Collector} for accepting the deserialized {@link + * PulsarMessage} from pulsar {@link PulsarDeserializationSchema}. + * + * @param The deserialized pulsar message type, aka the source message type. + */ +public class PulsarMessageCollector implements Collector { + + private final String splitId; + private final RecordsBySplits.Builder> builder; + private Message message; + + public PulsarMessageCollector( + String splitId, RecordsBySplits.Builder> builder) { + this.splitId = splitId; + this.builder = builder; + } + + public void setMessage(Message message) { + this.message = message; + } + + @Override + public void collect(T t) { + PulsarMessage result = + new PulsarMessage<>(message.getMessageId(), t, message.getEventTime()); + builder.add(splitId, result); + } + + @Override + public void close() { + // Nothing to do for this collector. + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java new file mode 100644 index 000000000..695df65ff --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java @@ -0,0 +1,204 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.source; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.reader.fetcher.PulsarOrderedFetcherManager; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplitState; +import org.apache.flink.core.io.InputStatus; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; + +/** + * The source reader for pulsar subscription Failover and Exclusive, which consumes the ordered + * messages. + */ +@Internal +public class PulsarOrderedSourceReader extends PulsarSourceReaderBase { + private static final Logger LOG = LoggerFactory.getLogger(PulsarOrderedSourceReader.class); + + private final SortedMap> cursorsToCommit; + private final ConcurrentMap cursorsOfFinishedSplits; + private final AtomicReference cursorCommitThrowable = new AtomicReference<>(); + private ScheduledExecutorService cursorScheduler; + + public PulsarOrderedSourceReader( + FutureCompletingBlockingQueue>> elementsQueue, + Supplier> splitReaderSupplier, + Configuration configuration, + SourceReaderContext context, + SourceConfiguration sourceConfiguration, + PulsarClient pulsarClient, + PulsarAdmin pulsarAdmin) { + super( + elementsQueue, + new PulsarOrderedFetcherManager<>(elementsQueue, splitReaderSupplier::get), + configuration, + context, + sourceConfiguration, + pulsarClient, + pulsarAdmin); + + this.cursorsToCommit = Collections.synchronizedSortedMap(new TreeMap<>()); + this.cursorsOfFinishedSplits = new ConcurrentHashMap<>(); + } + + @Override + public void start() { + super.start(); + if (sourceConfiguration.isEnableAutoAcknowledgeMessage()) { + this.cursorScheduler = Executors.newSingleThreadScheduledExecutor(); + + // Auto commit cursor, this could be enabled when checkpoint is also enabled. + cursorScheduler.scheduleAtFixedRate( + this::cumulativeAcknowledgmentMessage, + sourceConfiguration.getMaxFetchTime().toMillis(), + sourceConfiguration.getAutoCommitCursorInterval(), + TimeUnit.MILLISECONDS); + } + } + + @Override + public InputStatus pollNext(ReaderOutput output) throws Exception { + checkErrorAndRethrow(); + return super.pollNext(output); + } + + @Override + protected void onSplitFinished(Map finishedSplitIds) { + // We don't require new splits, all the splits are pre-assigned by source enumerator. + if (LOG.isDebugEnabled()) { + LOG.debug("onSplitFinished event: {}", finishedSplitIds); + } + + for (Map.Entry entry : finishedSplitIds.entrySet()) { + PulsarPartitionSplitState state = entry.getValue(); + MessageId latestConsumedId = state.getLatestConsumedId(); + if (latestConsumedId != null) { + cursorsOfFinishedSplits.put(state.getPartition(), latestConsumedId); + } + } + } + + @Override + public List snapshotState(long checkpointId) { + List splits = super.snapshotState(checkpointId); + + // Perform a snapshot for these splits. + Map cursors = + cursorsToCommit.computeIfAbsent(checkpointId, id -> new HashMap<>()); + // Put the cursors of the active splits. + for (PulsarPartitionSplit split : splits) { + MessageId latestConsumedId = split.getLatestConsumedId(); + if (latestConsumedId != null) { + cursors.put(split.getPartition(), latestConsumedId); + } + } + // Put cursors of all the finished splits. + cursors.putAll(cursorsOfFinishedSplits); + + return splits; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + LOG.debug("Committing cursors for checkpoint {}", checkpointId); + Map cursors = cursorsToCommit.get(checkpointId); + try { + ((PulsarOrderedFetcherManager) splitFetcherManager).acknowledgeMessages(cursors); + LOG.debug("Successfully acknowledge cursors for checkpoint {}", checkpointId); + + // Clean up the cursors. + cursorsOfFinishedSplits.keySet().removeAll(cursors.keySet()); + cursorsToCommit.headMap(checkpointId + 1).clear(); + } catch (Exception e) { + LOG.error("Failed to acknowledge cursors for checkpoint {}", checkpointId, e); + cursorCommitThrowable.compareAndSet(null, e); + } + } + + @Override + public void close() throws Exception { + if (cursorScheduler != null) { + cursorScheduler.shutdown(); + } + + super.close(); + } + + // ----------------- helper methods -------------- + + private void checkErrorAndRethrow() { + Throwable cause = cursorCommitThrowable.get(); + if (cause != null) { + throw new RuntimeException("An error occurred in acknowledge message.", cause); + } + } + + /** Acknowledge the pulsar topic partition cursor by the last consumed message id. */ + private void cumulativeAcknowledgmentMessage() { + Map cursors = new HashMap<>(cursorsOfFinishedSplits); + + // We reuse snapshotState for acquiring a consume status snapshot. + // So the checkpoint didn't really happen, so we just pass a fake checkpoint id. + List splits = super.snapshotState(1L); + for (PulsarPartitionSplit split : splits) { + MessageId latestConsumedId = split.getLatestConsumedId(); + if (latestConsumedId != null) { + cursors.put(split.getPartition(), latestConsumedId); + } + } + + try { + ((PulsarOrderedFetcherManager) splitFetcherManager).acknowledgeMessages(cursors); + // Clean up the finish splits. + cursorsOfFinishedSplits.keySet().removeAll(cursors.keySet()); + } catch (Exception e) { + LOG.error("Fail in auto cursor commit.", e); + cursorCommitThrowable.compareAndSet(null, e); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java new file mode 100644 index 000000000..054d3b8c5 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/PulsarSourceReaderBase.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 com.bytedance.bitsail.connector.pulsar.source.reader.source; + +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.reader.emitter.PulsarRecordEmitter; +import com.bytedance.bitsail.connector.pulsar.source.reader.fetcher.PulsarFetcherManagerBase; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplitState; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.PulsarClient; + +/** + * The common pulsar source reader for both ordered & unordered message consuming. + * + * @param The output message type for flink. + */ +abstract class PulsarSourceReaderBase + extends SingleThreadMultiplexSourceReaderBase< + PulsarMessage, OUT, PulsarPartitionSplit, PulsarPartitionSplitState> { + + protected final SourceConfiguration sourceConfiguration; + protected final PulsarClient pulsarClient; + protected final PulsarAdmin pulsarAdmin; + + protected PulsarSourceReaderBase( + FutureCompletingBlockingQueue>> elementsQueue, + PulsarFetcherManagerBase splitFetcherManager, + Configuration configuration, + SourceReaderContext context, + SourceConfiguration sourceConfiguration, + PulsarClient pulsarClient, + PulsarAdmin pulsarAdmin) { + super( + elementsQueue, + splitFetcherManager, + new PulsarRecordEmitter<>(), + configuration, + context); + + this.sourceConfiguration = sourceConfiguration; + this.pulsarClient = pulsarClient; + this.pulsarAdmin = pulsarAdmin; + } + + @Override + protected PulsarPartitionSplitState initializedState(PulsarPartitionSplit split) { + return new PulsarPartitionSplitState(split); + } + + @Override + protected PulsarPartitionSplit toSplitType( + String splitId, PulsarPartitionSplitState splitState) { + return splitState.toPulsarPartitionSplit(); + } + + @Override + public void close() throws Exception { + // Close the all the consumers first. + super.close(); + + // Close shared pulsar resources. + pulsarClient.shutdown(); + pulsarAdmin.close(); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java new file mode 100644 index 000000000..2663a5986 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java @@ -0,0 +1,146 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.source; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.reader.fetcher.PulsarUnorderedFetcherManager; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.reader.split.PulsarUnorderedPartitionSplitReader; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplitState; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; +import org.apache.pulsar.client.api.transaction.TxnID; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.function.Supplier; + +/** + * The source reader for pulsar subscription Shared and Key_Shared, which consumes the unordered + * messages. + */ +@Internal +public class PulsarUnorderedSourceReader extends PulsarSourceReaderBase { + private static final Logger LOG = LoggerFactory.getLogger(PulsarUnorderedSourceReader.class); + + @Nullable private final TransactionCoordinatorClient coordinatorClient; + private final SortedMap> transactionsToCommit; + private final List transactionsOfFinishedSplits; + + public PulsarUnorderedSourceReader( + FutureCompletingBlockingQueue>> elementsQueue, + Supplier> splitReaderSupplier, + Configuration configuration, + SourceReaderContext context, + SourceConfiguration sourceConfiguration, + PulsarClient pulsarClient, + PulsarAdmin pulsarAdmin, + @Nullable TransactionCoordinatorClient coordinatorClient) { + super( + elementsQueue, + new PulsarUnorderedFetcherManager<>(elementsQueue, splitReaderSupplier::get), + configuration, + context, + sourceConfiguration, + pulsarClient, + pulsarAdmin); + + this.coordinatorClient = coordinatorClient; + this.transactionsToCommit = Collections.synchronizedSortedMap(new TreeMap<>()); + this.transactionsOfFinishedSplits = Collections.synchronizedList(new ArrayList<>()); + } + + @Override + protected void onSplitFinished(Map finishedSplitIds) { + // We don't require new splits, all the splits are pre-assigned by source enumerator. + if (LOG.isDebugEnabled()) { + LOG.debug("onSplitFinished event: {}", finishedSplitIds); + } + + if (coordinatorClient != null) { + // Commit the uncommitted transaction + for (Map.Entry entry : finishedSplitIds.entrySet()) { + PulsarPartitionSplitState state = entry.getValue(); + TxnID uncommittedTransactionId = state.getUncommittedTransactionId(); + if (uncommittedTransactionId != null) { + transactionsOfFinishedSplits.add(uncommittedTransactionId); + } + } + } + } + + @Override + public List snapshotState(long checkpointId) { + LOG.debug("Trigger the new transaction for downstream readers."); + List splits = + ((PulsarUnorderedFetcherManager) splitFetcherManager) + .snapshotState(checkpointId); + + if (coordinatorClient != null) { + // Snapshot the transaction status and commit it after checkpoint finished. + List txnIDs = + transactionsToCommit.computeIfAbsent(checkpointId, id -> new ArrayList<>()); + for (PulsarPartitionSplit split : splits) { + TxnID uncommittedTransactionId = split.getUncommittedTransactionId(); + if (uncommittedTransactionId != null) { + txnIDs.add(uncommittedTransactionId); + } + } + } + + return splits; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + LOG.debug("Committing transactions for checkpoint {}", checkpointId); + + if (coordinatorClient != null) { + for (Map.Entry> entry : transactionsToCommit.entrySet()) { + Long currentCheckpointId = entry.getKey(); + if (currentCheckpointId > checkpointId) { + continue; + } + + List transactions = entry.getValue(); + for (TxnID transaction : transactions) { + coordinatorClient.commit(transaction); + transactionsOfFinishedSplits.remove(transaction); + } + + transactionsToCommit.remove(currentCheckpointId); + } + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/v1/PulsarOrderedSourceReaderV1.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/v1/PulsarOrderedSourceReaderV1.java new file mode 100644 index 000000000..251e7ff14 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/v1/PulsarOrderedSourceReaderV1.java @@ -0,0 +1,205 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.source.v1; + +import com.bytedance.bitsail.base.connector.reader.v1.SourcePipeline; +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.common.row.Row; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.reader.split.v1.PulsarOrderedPartitionSplitReader; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplit; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplitState; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; + +/** + * The source reader for pulsar subscription Failover and Exclusive, which consumes the ordered + * messages. + */ +@Internal +public class PulsarOrderedSourceReaderV1 extends PulsarSourceReaderBaseV1 { + private static final Logger LOG = LoggerFactory.getLogger(PulsarOrderedSourceReaderV1.class); + + private final SortedMap> cursorsToCommit; + private final ConcurrentMap cursorsOfFinishedSplits; + private final AtomicReference cursorCommitThrowable = new AtomicReference<>(); + private ScheduledExecutorService cursorScheduler; + + public PulsarOrderedSourceReaderV1( + FutureCompletingBlockingQueue>> elementsQueue, + Supplier splitReaderSupplier, + BitSailConfiguration readerConfiguration, + Context context, + SourceConfiguration sourceConfiguration, + PulsarClient pulsarClient, + PulsarAdmin pulsarAdmin) { + super( + elementsQueue, + splitReaderSupplier, + readerConfiguration, + context, + sourceConfiguration, + pulsarClient, + pulsarAdmin); + + this.cursorsToCommit = Collections.synchronizedSortedMap(new TreeMap<>()); + this.cursorsOfFinishedSplits = new ConcurrentHashMap<>(); + } + + @Override + public void start() { + if (sourceConfiguration.isEnableAutoAcknowledgeMessage()) { + this.cursorScheduler = Executors.newSingleThreadScheduledExecutor(); + + // Auto commit cursor, this could be enabled when checkpoint is also enabled. + cursorScheduler.scheduleAtFixedRate( + this::cumulativeAcknowledgmentMessage, + sourceConfiguration.getMaxFetchTime().toMillis(), + sourceConfiguration.getAutoCommitCursorInterval(), + TimeUnit.MILLISECONDS); + } + } + + @Override + public void pollNext(SourcePipeline pipeline) throws Exception { + super.pollNext(pipeline); + } + + + @Override + protected void onSplitFinished(Map finishedSplitIds) { + // We don't require new splits, all the splits are pre-assigned by source enumerator. + if (LOG.isDebugEnabled()) { + LOG.debug("onSplitFinished event: {}", finishedSplitIds); + } + + for (Map.Entry entry : finishedSplitIds.entrySet()) { + PulsarPartitionSplitState state = entry.getValue(); + MessageId latestConsumedId = state.getLatestConsumedId(); + if (latestConsumedId != null) { + cursorsOfFinishedSplits.put(state.getPartition(), latestConsumedId); + } + } + } + + @Override + public List snapshotState(long checkpointId) { + List splits = super.snapshotState(checkpointId); + + // Perform a snapshot for these splits. + Map cursors = + cursorsToCommit.computeIfAbsent(checkpointId, id -> new HashMap<>()); + // Put the cursors of the active splits. + for (PulsarPartitionSplit split : splits) { + MessageId latestConsumedId = split.getLatestConsumedId(); + if (latestConsumedId != null) { + cursors.put(split.getPartition(), latestConsumedId); + } + } + // Put cursors of all the finished splits. + cursors.putAll(cursorsOfFinishedSplits); + + return splits; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + LOG.debug("Committing cursors for checkpoint {}", checkpointId); + Map cursors = cursorsToCommit.get(checkpointId); + try { + super.commitCursors(cursors); + LOG.debug("Successfully acknowledge cursors for checkpoint {}", checkpointId); + + // Clean up the cursors. + cursorsOfFinishedSplits.keySet().removeAll(cursors.keySet()); + cursorsToCommit.headMap(checkpointId + 1).clear(); + } catch (Exception e) { + LOG.error("Failed to acknowledge cursors for checkpoint {}", checkpointId, e); + cursorCommitThrowable.compareAndSet(null, e); + } + } + + @Override + public void close() throws Exception { + if (cursorScheduler != null) { + cursorScheduler.shutdown(); + } + + super.close(); + } + + // ----------------- helper methods -------------- + + private void checkErrorAndRethrow() { + Throwable cause = cursorCommitThrowable.get(); + if (cause != null) { + throw new RuntimeException("An error occurred in acknowledge message.", cause); + } + } + + /** Acknowledge the pulsar topic partition cursor by the last consumed message id. */ + private void cumulativeAcknowledgmentMessage() { + Map cursors = new HashMap<>(cursorsOfFinishedSplits); + + // We reuse snapshotState for acquiring a consume status snapshot. + // So the checkpoint didn't really happen, so we just pass a fake checkpoint id. + List splits = super.snapshotState(1L); + for (PulsarPartitionSplit split : splits) { + MessageId latestConsumedId = split.getLatestConsumedId(); + if (latestConsumedId != null) { + cursors.put(split.getPartition(), latestConsumedId); + } + } + + try { + super.commitCursors(cursors); + // Clean up the finish splits. + cursorsOfFinishedSplits.keySet().removeAll(cursors.keySet()); + } catch (Exception e) { + LOG.error("Fail in auto cursor commit.", e); + cursorCommitThrowable.compareAndSet(null, e); + } + } + + +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/v1/PulsarSourceReaderBaseV1.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/v1/PulsarSourceReaderBaseV1.java new file mode 100644 index 000000000..c72a9fb6a --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/v1/PulsarSourceReaderBaseV1.java @@ -0,0 +1,302 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.source.v1; + +import com.bytedance.bitsail.base.connector.reader.v1.SourcePipeline; +import com.bytedance.bitsail.base.connector.reader.v1.SourceReader; +import com.bytedance.bitsail.base.format.DeserializationSchema; +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.common.row.Row; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.reader.deserializer.v1.PulsarDeserializationSchema; +import com.bytedance.bitsail.connector.pulsar.source.reader.emitter.PulsarRecordEmitter; +import com.bytedance.bitsail.connector.pulsar.source.reader.fetcher.PulsarFetcherManagerBase; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.reader.split.v1.PulsarPartitionSplitReader; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplit; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplitState; + +import com.google.common.collect.Sets; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SourceReaderBase; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.core.io.InputStatus; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; + +import static org.apache.flink.util.Preconditions.checkState; + +abstract class PulsarSourceReaderBaseV1 implements SourceReader { + private static final Logger LOG = LoggerFactory.getLogger(SourceReaderBase.class); + + protected final SourceConfiguration sourceConfiguration; + protected final PulsarClient pulsarClient; + protected final PulsarAdmin pulsarAdmin; + private final FutureCompletingBlockingQueue>> elementsQueue; + private final PulsarFetcherManagerBase splitFetcherManager = null; + private final PulsarRecordEmitter recordEmitter; + private final HashMap> splitStates; + private final BitSailConfiguration readerConfiguration; + private final transient DeserializationSchema deserializationSchema; + private final Context context; + private final Supplier splitReaderSupplier; + + /** The latest fetched batch of records-by-split from the split reader. */ + @Nullable private RecordsWithSplitIds> currentFetch; + + @Override + public List snapshotState(long checkpointId) { + List splits = new ArrayList<>(); + splitStates.forEach((id, context) -> splits.add(toSplitType(id, context.state))); + return splits; + } + + @Nullable private SplitContext currentSplitContext; + @Nullable private SourceOutput currentSplitOutput; + private boolean noMoreSplitsAssignment; + private final transient Set assignedPulsarSplits; + private final Map splitReaderMapping = new HashMap<>(); + private final Map fetcherStatus = new HashMap<>(); + + protected PulsarSourceReaderBaseV1( + FutureCompletingBlockingQueue>> elementsQueue, + Supplier splitReaderSupplier, + BitSailConfiguration readerConfiguration, + Context context, + SourceConfiguration sourceConfiguration, + PulsarClient pulsarClient, + PulsarAdmin pulsarAdmin) { + + this.elementsQueue = elementsQueue; + this.splitReaderSupplier = splitReaderSupplier; + this.recordEmitter = new PulsarRecordEmitter<>(); + this.splitStates = new HashMap<>(); + this.readerConfiguration = readerConfiguration; + this.context = context; + this.noMoreSplitsAssignment = false; + this.deserializationSchema = new PulsarDeserializationSchema( + readerConfiguration, + context.getTypeInfos(), + context.getFieldNames()); + this.sourceConfiguration = sourceConfiguration; + this.pulsarClient = pulsarClient; + this.pulsarAdmin = pulsarAdmin; + this.assignedPulsarSplits = Sets.newHashSet(); + } + + protected PulsarPartitionSplitState initializedState(PulsarPartitionSplit split) { + return new PulsarPartitionSplitState(split); + } + + protected PulsarPartitionSplit toSplitType( + String splitId, PulsarPartitionSplitState splitState) { + return splitState.toPulsarPartitionSplit(); + } + + @Override + public void addSplits(List splits) { + LOG.info("Adding split(s) to reader: {}", splits); + // Initialize the state for each split. + splits.forEach( + s -> { + splitStates.put(s.getPartition().toString(), new SplitContext<>(s.splitId(), initializedState(s))); + PulsarPartitionSplitReader splitReader = getOrCreateSplitReader(s.splitId()); + splitReader.handleSplitsChanges(new SplitsAddition<>(splits)); + }); + // Hand over the splits to the split fetcher to start fetch. + assignedPulsarSplits.addAll(splits); + } + + protected PulsarPartitionSplitReader getOrCreateSplitReader(String splitId) { + PulsarPartitionSplitReader splitReader = splitReaderMapping.get(splitId); + + if (splitReader == null) { + splitReader = splitReaderSupplier.get(); + splitReaderMapping.put(splitId, splitReader); + } + + return splitReader; + } + + @Override + public boolean hasMoreElements() { + return true; + } + + @Override + public void close() throws Exception { + // Close the all the consumers first. + for (PulsarPartitionSplitReader splitReader : splitReaderMapping.values()) { + splitReader.close(); + } + + // Close shared pulsar resources. + pulsarClient.shutdown(); + pulsarAdmin.close(); + } + + @Override + public void pollNext(SourcePipeline output) throws Exception { + + for (PulsarPartitionSplitReader splitReader : splitReaderMapping.values()) { + RecordsWithSplitIds> recordsWithSplitId = splitReader.fetch(); + + if (recordsWithSplitId == null) { + continue; + } + + while (recordsWithSplitId.nextSplit() != null) { + PulsarMessage record = recordsWithSplitId.nextRecordFromSplit(); + + while (record != null) { + Row deserialize = deserializationSchema.deserialize(record.getValue()); + output.output(deserialize); + LOG.trace("Emitted record: {}", record); + record = recordsWithSplitId.nextRecordFromSplit(); + } + } + } + } + + private InputStatus trace(InputStatus status) { + LOG.trace("Source reader status: {}", status); + return status; + } + + @Nullable + private RecordsWithSplitIds> getNextFetch(final SourcePipeline output) { + splitFetcherManager.checkErrors(); + + LOG.trace("Getting next source data batch from queue"); + final RecordsWithSplitIds> recordsWithSplitId = elementsQueue.poll(); + if (recordsWithSplitId == null || !moveToNextSplit(recordsWithSplitId, output)) { + // No element available, set to available later if needed. + return null; + } + + currentFetch = recordsWithSplitId; + return recordsWithSplitId; + } + + private void finishCurrentFetch( + final RecordsWithSplitIds> fetch, final SourcePipeline output) { + currentFetch = null; + currentSplitContext = null; + currentSplitOutput = null; + + final Set finishedSplits = fetch.finishedSplits(); + if (!finishedSplits.isEmpty()) { + LOG.info("Finished reading split(s) {}", finishedSplits); + Map stateOfFinishedSplits = new HashMap<>(); + for (String finishedSplitId : finishedSplits) { + stateOfFinishedSplits.put( + finishedSplitId, splitStates.remove(finishedSplitId).state); + } + onSplitFinished(stateOfFinishedSplits); + } + + fetch.recycle(); + } + + protected void onSplitFinished(Map stateOfFinishedSplits) {}; + + private boolean moveToNextSplit( + RecordsWithSplitIds> recordsWithSplitIds, SourcePipeline output) { + final String nextSplitId = recordsWithSplitIds.nextSplit(); + if (nextSplitId == null) { + LOG.trace("Current fetch is finished."); + finishCurrentFetch(recordsWithSplitIds, output); + return false; + } + + currentSplitContext = splitStates.get(nextSplitId); + checkState(currentSplitContext != null, "Have records for a split that was not registered"); + LOG.trace("Emitting records from fetch for split {}", nextSplitId); + return true; + } + + public CompletableFuture isAvailable() { + return currentFetch != null + ? FutureCompletingBlockingQueue.AVAILABLE + : elementsQueue.getAvailabilityFuture(); + } + + + private InputStatus finishedOrAvailableLater() { + final boolean allFetchersHaveShutdown = splitFetcherManager.maybeShutdownFinishedFetchers(); + if (!(noMoreSplitsAssignment && allFetchersHaveShutdown)) { + return InputStatus.NOTHING_AVAILABLE; + } + if (elementsQueue.isEmpty()) { + // We may reach here because of exceptional split fetcher, check it. + splitFetcherManager.checkErrors(); + return InputStatus.END_OF_INPUT; + } else { + // We can reach this case if we just processed all data from the queue and finished a + // split, + // and concurrently the fetcher finished another split, whose data is then in the queue. + return InputStatus.MORE_AVAILABLE; + } + } + + public void commitCursors(Map cursors) { + cursors.forEach((partition, messageId) -> { + + PulsarPartitionSplitReader splitReader = getOrCreateSplitReader(partition.toString()); + splitReader.commit(partition, messageId); + }); + } + + private static final class SplitContext { + + final String splitId; + final PulsarPartitionSplitState state; + SourceOutput sourceOutput; + + private SplitContext(String splitId, PulsarPartitionSplitState state) { + this.state = state; + this.splitId = splitId; + } + + SourceOutput getOrCreateSplitOutput(ReaderOutput mainOutput) { + if (sourceOutput == null) { + sourceOutput = mainOutput.createOutputForSplit(splitId); + } + return sourceOutput; + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/v1/PulsarUnorderedSourceReaderV1.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/v1/PulsarUnorderedSourceReaderV1.java new file mode 100644 index 000000000..20dd5d7bb --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/source/v1/PulsarUnorderedSourceReaderV1.java @@ -0,0 +1,147 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.source.v1; + +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.reader.split.v1.PulsarUnorderedPartitionSplitReader; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplit; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplitState; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; +import org.apache.pulsar.client.api.transaction.TxnID; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.function.Supplier; + +/** + * The source reader for pulsar subscription Shared and Key_Shared, which consumes the unordered + * messages. + */ +@Internal +public class PulsarUnorderedSourceReaderV1 extends PulsarSourceReaderBaseV1 { + private static final Logger LOG = LoggerFactory.getLogger(PulsarUnorderedSourceReaderV1.class); + + @Nullable private final TransactionCoordinatorClient coordinatorClient; + private final SortedMap> transactionsToCommit; + private final List transactionsOfFinishedSplits; + + public PulsarUnorderedSourceReaderV1( + FutureCompletingBlockingQueue>> elementsQueue, + Supplier splitReaderSupplier, + BitSailConfiguration readerConfiguration, + Context context, + SourceConfiguration sourceConfiguration, + PulsarClient pulsarClient, + PulsarAdmin pulsarAdmin, + @Nullable TransactionCoordinatorClient coordinatorClient) { + super( + elementsQueue, + splitReaderSupplier, + readerConfiguration, + context, + sourceConfiguration, + pulsarClient, + pulsarAdmin); + + this.coordinatorClient = coordinatorClient; + this.transactionsToCommit = Collections.synchronizedSortedMap(new TreeMap<>()); + this.transactionsOfFinishedSplits = Collections.synchronizedList(new ArrayList<>()); + } + + @Override + protected void onSplitFinished(Map finishedSplitIds) { + // We don't require new splits, all the splits are pre-assigned by source enumerator. + if (LOG.isDebugEnabled()) { + LOG.debug("onSplitFinished event: {}", finishedSplitIds); + } + + if (coordinatorClient != null) { + // Commit the uncommitted transaction + for (Map.Entry entry : finishedSplitIds.entrySet()) { + PulsarPartitionSplitState state = entry.getValue(); + TxnID uncommittedTransactionId = state.getUncommittedTransactionId(); + if (uncommittedTransactionId != null) { + transactionsOfFinishedSplits.add(uncommittedTransactionId); + } + } + } + } + + @Override + public void start() {} + + @Override + public List snapshotState(long checkpointId) { + LOG.debug("Trigger the new transaction for downstream readers."); + List splits = super.snapshotState(checkpointId); + + if (coordinatorClient != null) { + // Snapshot the transaction status and commit it after checkpoint finished. + List txnIDs = + transactionsToCommit.computeIfAbsent(checkpointId, id -> new ArrayList<>()); + for (PulsarPartitionSplit split : splits) { + TxnID uncommittedTransactionId = split.getUncommittedTransactionId(); + if (uncommittedTransactionId != null) { + txnIDs.add(uncommittedTransactionId); + } + } + } + + return splits; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + LOG.debug("Committing transactions for checkpoint {}", checkpointId); + + if (coordinatorClient != null) { + for (Map.Entry> entry : transactionsToCommit.entrySet()) { + Long currentCheckpointId = entry.getKey(); + if (currentCheckpointId > checkpointId) { + continue; + } + + List transactions = entry.getValue(); + for (TxnID transaction : transactions) { + coordinatorClient.commit(transaction); + transactionsOfFinishedSplits.remove(transaction); + } + + transactionsToCommit.remove(currentCheckpointId); + } + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java new file mode 100644 index 000000000..3fbc318d8 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java @@ -0,0 +1,115 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.split; + +import org.apache.flink.annotation.Internal; + +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StartCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import com.bytedance.bitsail.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; + +import org.apache.flink.configuration.Configuration; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.concurrent.TimeUnit; + +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; +import static com.bytedance.bitsail.connector.pulsar.source.config.CursorVerification.FAIL_ON_MISMATCH; + +/** + * The split reader a given {@link PulsarPartitionSplit}, it would be closed once the {@link + * PulsarOrderedSourceReader} is closed. + * + * @param the type of the pulsar source message that would be serialized to downstream. + */ +@Internal +public class PulsarOrderedPartitionSplitReader extends PulsarPartitionSplitReaderBase { + private static final Logger LOG = + LoggerFactory.getLogger(PulsarOrderedPartitionSplitReader.class); + + public PulsarOrderedPartitionSplitReader( + PulsarClient pulsarClient, + PulsarAdmin pulsarAdmin, + Configuration configuration, + SourceConfiguration sourceConfiguration, + PulsarDeserializationSchema deserializationSchema) { + super(pulsarClient, pulsarAdmin, configuration, sourceConfiguration, deserializationSchema); + } + + @Override + protected Message pollMessage(Duration timeout) throws PulsarClientException { + return pulsarConsumer.receive(Math.toIntExact(timeout.toMillis()), TimeUnit.MILLISECONDS); + } + + @Override + protected void finishedPollMessage(Message message) { + // Nothing to do here. + LOG.debug("Finished polling message {}", message); + + // Release message + message.release(); + } + + @Override + protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + MessageId latestConsumedId = split.getLatestConsumedId(); + + // Reset the start position for ordered pulsar consumer. + if (latestConsumedId != null) { + StartCursor startCursor = StartCursor.fromMessageId(latestConsumedId, false); + TopicPartition partition = split.getPartition(); + + try { + startCursor.seekPosition( + partition.getTopic(), partition.getPartitionId(), consumer); + } catch (PulsarClientException e) { + if (sourceConfiguration.getVerifyInitialOffsets() == FAIL_ON_MISMATCH) { + throw new IllegalArgumentException(e); + } else { + // WARN_ON_MISMATCH would just print this warning message. + // No need to print the stacktrace. + LOG.warn( + "Failed to reset cursor to {} on partition {}", + latestConsumedId, + partition, + e); + } + } + } + } + + public void notifyCheckpointComplete(TopicPartition partition, MessageId offsetsToCommit) { + if (pulsarConsumer == null) { + this.pulsarConsumer = createPulsarConsumer(partition); + } + + sneakyClient(() -> pulsarConsumer.acknowledgeCumulative(offsetsToCommit)); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java new file mode 100644 index 000000000..0c14d80b6 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java @@ -0,0 +1,234 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.split; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsBySplits; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; + +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessageCollector; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.flink.util.Preconditions; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.KeySharedPolicy; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.time.Duration; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; +import static com.bytedance.bitsail.connector.pulsar.source.config.PulsarSourceConfigUtils.createConsumerBuilder; + +/** + * The common partition split reader. + * + * @param the type of the pulsar source message that would be serialized to downstream. + */ +public abstract class PulsarPartitionSplitReaderBase + implements SplitReader, PulsarPartitionSplit> { + private static final Logger LOG = LoggerFactory.getLogger(PulsarPartitionSplitReaderBase.class); + + protected final PulsarClient pulsarClient; + protected final PulsarAdmin pulsarAdmin; + protected final Configuration configuration; + protected final SourceConfiguration sourceConfiguration; + protected final PulsarDeserializationSchema deserializationSchema; + protected final AtomicBoolean wakeup; + + protected Consumer pulsarConsumer; + protected PulsarPartitionSplit registeredSplit; + + protected PulsarPartitionSplitReaderBase( + PulsarClient pulsarClient, + PulsarAdmin pulsarAdmin, + Configuration configuration, + SourceConfiguration sourceConfiguration, + PulsarDeserializationSchema deserializationSchema) { + this.pulsarClient = pulsarClient; + this.pulsarAdmin = pulsarAdmin; + this.configuration = configuration; + this.sourceConfiguration = sourceConfiguration; + this.deserializationSchema = deserializationSchema; + this.wakeup = new AtomicBoolean(false); + } + + @Override + public RecordsWithSplitIds> fetch() throws IOException { + RecordsBySplits.Builder> builder = new RecordsBySplits.Builder<>(); + + // Return when no split registered to this reader. + if (pulsarConsumer == null || registeredSplit == null) { + return builder.build(); + } + + // Set wakeup to false for start consuming. + wakeup.compareAndSet(true, false); + + StopCursor stopCursor = registeredSplit.getStopCursor(); + String splitId = registeredSplit.splitId(); + PulsarMessageCollector collector = new PulsarMessageCollector<>(splitId, builder); + Deadline deadline = Deadline.fromNow(sourceConfiguration.getMaxFetchTime()); + + // Consume message from pulsar until it was woke up by flink reader. + for (int messageNum = 0; + messageNum < sourceConfiguration.getMaxFetchRecords() + && deadline.hasTimeLeft() + && isNotWakeup(); + messageNum++) { + try { + Duration timeout = deadline.timeLeftIfAny(); + Message message = pollMessage(timeout); + if (message == null) { + break; + } + + // Deserialize message. + collector.setMessage(message); + deserializationSchema.deserialize(message, collector); + + // Acknowledge message if need. + finishedPollMessage(message); + + if (stopCursor.shouldStop(message)) { + builder.addFinishedSplit(splitId); + break; + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } catch (TimeoutException e) { + break; + } catch (ExecutionException e) { + LOG.error("Error in polling message from pulsar consumer.", e); + break; + } catch (Exception e) { + throw new IOException(e); + } + } + + return builder.build(); + } + + @Override + public void handleSplitsChanges(SplitsChange splitsChanges) { + LOG.debug("Handle split changes {}", splitsChanges); + + // Get all the partition assignments and stopping offsets. + if (!(splitsChanges instanceof SplitsAddition)) { + throw new UnsupportedOperationException( + String.format( + "The SplitChange type of %s is not supported.", + splitsChanges.getClass())); + } + + if (registeredSplit != null) { + throw new IllegalStateException("This split reader have assigned split."); + } + + List newSplits = splitsChanges.splits(); + Preconditions.checkArgument( + newSplits.size() == 1, "This pulsar split reader only support one split."); + PulsarPartitionSplit newSplit = newSplits.get(0); + + // Create pulsar consumer. + Consumer consumer = createPulsarConsumer(newSplit); + + // Open start & stop cursor. + newSplit.open(pulsarAdmin); + + // Start Consumer. + startConsumer(newSplit, consumer); + + LOG.info("Register split {} consumer for current reader.", newSplit); + this.registeredSplit = newSplit; + this.pulsarConsumer = consumer; + } + + @Override + public void wakeUp() { + wakeup.compareAndSet(false, true); + } + + @Override + public void close() { + if (pulsarConsumer != null) { + sneakyClient(() -> pulsarConsumer.close()); + } + } + + @Nullable + protected abstract Message pollMessage(Duration timeout) + throws ExecutionException, InterruptedException, PulsarClientException; + + protected abstract void finishedPollMessage(Message message); + + protected abstract void startConsumer(PulsarPartitionSplit split, Consumer consumer); + + // --------------------------- Helper Methods ----------------------------- + + protected boolean isNotWakeup() { + return !wakeup.get(); + } + + /** Create a specified {@link Consumer} by the given split information. */ + protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { + return createPulsarConsumer(split.getPartition()); + } + + /** Create a specified {@link Consumer} by the given topic partition. */ + protected Consumer createPulsarConsumer(TopicPartition partition) { + ConsumerBuilder consumerBuilder = + createConsumerBuilder(pulsarClient, Schema.BYTES, configuration); + + consumerBuilder.topic(partition.getFullTopicName()); + + // Add KeySharedPolicy for Key_Shared subscription. + if (sourceConfiguration.getSubscriptionType() == SubscriptionType.Key_Shared) { + KeySharedPolicy policy = + KeySharedPolicy.stickyHashRange().ranges(partition.getPulsarRange()); + consumerBuilder.keySharedPolicy(policy); + } + + // Create the consumer configuration by using common utils. + return sneakyClient(consumerBuilder::subscribe); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java new file mode 100644 index 000000000..1560f39d2 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java @@ -0,0 +1,183 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.split; + +import org.apache.flink.annotation.Internal; + +import com.bytedance.bitsail.connector.pulsar.common.utils.PulsarTransactionUtils; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import com.bytedance.bitsail.connector.pulsar.source.reader.source.PulsarUnorderedSourceReader; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplitState; + +import org.apache.flink.configuration.Configuration; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; +import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException; +import org.apache.pulsar.client.api.transaction.TxnID; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; + +/** + * The split reader a given {@link PulsarPartitionSplit}, it would be closed once the {@link + * PulsarUnorderedSourceReader} is closed. + * + * @param the type of the pulsar source message that would be serialized to downstream. + */ +@Internal +public class PulsarUnorderedPartitionSplitReader extends PulsarPartitionSplitReaderBase { + private static final Logger LOG = + LoggerFactory.getLogger(PulsarUnorderedPartitionSplitReader.class); + + private static final Duration REDELIVER_TIME = Duration.ofSeconds(3); + + private final TransactionCoordinatorClient coordinatorClient; + + @Nullable private Transaction uncommittedTransaction; + + public PulsarUnorderedPartitionSplitReader( + PulsarClient pulsarClient, + PulsarAdmin pulsarAdmin, + Configuration configuration, + SourceConfiguration sourceConfiguration, + PulsarDeserializationSchema deserializationSchema, + TransactionCoordinatorClient coordinatorClient) { + super(pulsarClient, pulsarAdmin, configuration, sourceConfiguration, deserializationSchema); + + this.coordinatorClient = coordinatorClient; + } + + @Override + protected Message pollMessage(Duration timeout) + throws ExecutionException, InterruptedException, PulsarClientException { + Message message = + pulsarConsumer.receive(Math.toIntExact(timeout.toMillis()), TimeUnit.MILLISECONDS); + + // Skip the message when receive timeout + if (message == null) { + return null; + } + + if (!sourceConfiguration.isEnableAutoAcknowledgeMessage()) { + if (uncommittedTransaction == null) { + // Create a transaction. + this.uncommittedTransaction = newTransaction(); + } + + try { + // Add this message into transaction. + pulsarConsumer + .acknowledgeAsync(message.getMessageId(), uncommittedTransaction) + .get(); + } catch (InterruptedException e) { + sneakyClient( + () -> + pulsarConsumer.reconsumeLater( + message, REDELIVER_TIME.toMillis(), TimeUnit.MILLISECONDS)); + Thread.currentThread().interrupt(); + throw e; + } catch (ExecutionException e) { + sneakyClient( + () -> + pulsarConsumer.reconsumeLater( + message, REDELIVER_TIME.toMillis(), TimeUnit.MILLISECONDS)); + throw e; + } + } + + return message; + } + + @Override + protected void finishedPollMessage(Message message) { + if (sourceConfiguration.isEnableAutoAcknowledgeMessage()) { + sneakyClient(() -> pulsarConsumer.acknowledge(message)); + } + + // Release message + message.release(); + } + + @Override + protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + TxnID uncommittedTransactionId = split.getUncommittedTransactionId(); + + // Abort the uncommitted pulsar transaction. + if (uncommittedTransactionId != null) { + if (coordinatorClient != null) { + try { + coordinatorClient.abort(uncommittedTransactionId); + } catch (TransactionCoordinatorClientException e) { + LOG.error( + "Failed to abort the uncommitted transaction {} when restart the reader", + uncommittedTransactionId, + e); + } + } + + // Redeliver unacknowledged messages because of the message is out of order. + consumer.redeliverUnacknowledgedMessages(); + } + } + + public PulsarPartitionSplitState snapshotState(long checkpointId) { + PulsarPartitionSplitState state = new PulsarPartitionSplitState(registeredSplit); + + // Avoiding NP problem when Pulsar don't get the message before Flink checkpoint. + if (uncommittedTransaction != null) { + TxnID txnID = PulsarTransactionUtils.getId(uncommittedTransaction); + this.uncommittedTransaction = newTransaction(); + state.setUncommittedTransactionId(txnID); + } + + return state; + } + + private Transaction newTransaction() { + long timeoutMillis = sourceConfiguration.getTransactionTimeoutMillis(); + CompletableFuture future = + sneakyClient(pulsarClient::newTransaction) + .withTransactionTimeout(timeoutMillis, TimeUnit.MILLISECONDS) + .build(); + + try { + return future.get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } catch (ExecutionException e) { + throw new RuntimeException(e); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/v1/PulsarOrderedPartitionSplitReader.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/v1/PulsarOrderedPartitionSplitReader.java new file mode 100644 index 000000000..1c48248bd --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/v1/PulsarOrderedPartitionSplitReader.java @@ -0,0 +1,114 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.split.v1; + +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StartCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplit; + +import org.apache.flink.annotation.Internal; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.concurrent.TimeUnit; + +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; +import static com.bytedance.bitsail.connector.pulsar.source.config.CursorVerification.FAIL_ON_MISMATCH; + +@Internal +public class PulsarOrderedPartitionSplitReader extends PulsarPartitionSplitReader { + private static final Logger LOG = + LoggerFactory.getLogger(PulsarOrderedPartitionSplitReader.class); + + public PulsarOrderedPartitionSplitReader( + PulsarClient pulsarClient, + PulsarAdmin pulsarAdmin, + BitSailConfiguration configuration, + SourceConfiguration sourceConfiguration) { + super(pulsarClient, pulsarAdmin, configuration, sourceConfiguration); + } + + @Override + public void commit(TopicPartition partition, MessageId offsetsToCommit) { + if (pulsarConsumer == null) { + this.pulsarConsumer = createPulsarConsumer(partition); + } + + sneakyClient(() -> pulsarConsumer.acknowledgeCumulative(offsetsToCommit)); + } + + @Override + protected Message pollMessage(Duration timeout) throws PulsarClientException { + return pulsarConsumer.receive(Math.toIntExact(timeout.toMillis()), TimeUnit.MILLISECONDS); + } + + @Override + protected void finishedPollMessage(Message message) { + // Nothing to do here. + LOG.debug("Finished polling message {}", message); + + // Release message + message.release(); + } + + @Override + protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + MessageId latestConsumedId = split.getLatestConsumedId(); + + // Reset the start position for ordered pulsar consumer. + if (latestConsumedId != null) { + StartCursor startCursor = StartCursor.fromMessageId(latestConsumedId, false); + TopicPartition partition = split.getPartition(); + + try { + startCursor.seekPosition( + partition.getTopic(), partition.getPartitionId(), consumer); + } catch (PulsarClientException e) { + if (sourceConfiguration.getVerifyInitialOffsets() == FAIL_ON_MISMATCH) { + throw new IllegalArgumentException(e); + } else { + // WARN_ON_MISMATCH would just print this warning message. + // No need to print the stacktrace. + LOG.warn( + "Failed to reset cursor to {} on partition {}", + latestConsumedId, + partition, + e); + } + } + } + } + + public void notifyCheckpointComplete(TopicPartition partition, MessageId offsetsToCommit) { + if (pulsarConsumer == null) { + this.pulsarConsumer = createPulsarConsumer(partition); + } + + sneakyClient(() -> pulsarConsumer.acknowledgeCumulative(offsetsToCommit)); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/v1/PulsarPartitionSplitReader.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/v1/PulsarPartitionSplitReader.java new file mode 100644 index 000000000..63884511f --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/v1/PulsarPartitionSplitReader.java @@ -0,0 +1,227 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.split.v1; + +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.connector.pulsar.common.config.v1.PulsarUtils; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessage; +import com.bytedance.bitsail.connector.pulsar.source.reader.message.PulsarMessageCollector; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplit; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsBySplits; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.util.Preconditions; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.KeySharedPolicy; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.time.Duration; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; +import static com.bytedance.bitsail.connector.pulsar.source.config.PulsarSourceConfigUtils.createConsumerBuilder; + +public abstract class PulsarPartitionSplitReader { + private static final Logger LOG = LoggerFactory.getLogger(PulsarPartitionSplitReader.class); + + protected final PulsarClient pulsarClient; + protected final PulsarAdmin pulsarAdmin; + protected final BitSailConfiguration configuration; + protected final SourceConfiguration sourceConfiguration; + protected final AtomicBoolean wakeup; + + protected Consumer pulsarConsumer; + protected PulsarPartitionSplit registeredSplit; + + protected PulsarPartitionSplitReader( + PulsarClient pulsarClient, + PulsarAdmin pulsarAdmin, + BitSailConfiguration readerConfiguration, + SourceConfiguration sourceConfiguration) { + this.pulsarClient = pulsarClient; + this.pulsarAdmin = pulsarAdmin; + this.configuration = readerConfiguration; + this.sourceConfiguration = sourceConfiguration; + this.wakeup = new AtomicBoolean(false); + } + + public RecordsWithSplitIds> fetch() throws IOException { + RecordsBySplits.Builder> builder = new RecordsBySplits.Builder<>(); + + // Return when no split registered to this reader. + if (pulsarConsumer == null || registeredSplit == null) { + return builder.build(); + } + + // Set wakeup to false for start consuming. + wakeup.compareAndSet(true, false); + + StopCursor stopCursor = registeredSplit.getStopCursor(); + String splitId = registeredSplit.splitId(); + PulsarMessageCollector collector = new PulsarMessageCollector<>(splitId, builder); + Deadline deadline = Deadline.fromNow(sourceConfiguration.getMaxFetchTime()); + + // Consume message from pulsar until it was woke up by flink reader. + for (int messageNum = 0; + messageNum < sourceConfiguration.getMaxFetchRecords() + && deadline.hasTimeLeft() + && isNotWakeup(); + messageNum++) { + try { + Duration timeout = deadline.timeLeftIfAny(); + Message message = pollMessage(timeout); + if (message == null) { + break; + } + + // Add message. + collector.setMessage(message); + collector.collect(message.getValue()); + + // Acknowledge message if need. + finishedPollMessage(message); + + if (stopCursor.shouldStop(message)) { + builder.addFinishedSplit(splitId); + break; + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } catch (TimeoutException e) { + break; + } catch (ExecutionException e) { + LOG.error("Error in polling message from pulsar consumer.", e); + break; + } catch (Exception e) { + throw new IOException(e); + } + } + + return builder.build(); + } + + public void handleSplitsChanges(SplitsChange splitsChanges) { + LOG.debug("Handle split changes {}", splitsChanges); + + // Get all the partition assignments and stopping offsets. + if (!(splitsChanges instanceof SplitsAddition)) { + throw new UnsupportedOperationException( + String.format( + "The SplitChange type of %s is not supported.", + splitsChanges.getClass())); + } + + if (registeredSplit != null) { + throw new IllegalStateException("This split reader have assigned split."); + } + + List newSplits = splitsChanges.splits(); + Preconditions.checkArgument( + newSplits.size() == 1, "This pulsar split reader only support one split."); + PulsarPartitionSplit newSplit = newSplits.get(0); + + // Create pulsar consumer. + Consumer consumer = createPulsarConsumer(newSplit); + + // Open start & stop cursor. + newSplit.open(pulsarAdmin); + + // Start Consumer. + startConsumer(newSplit, consumer); + + LOG.info("Register split {} consumer for current reader.", newSplit); + this.registeredSplit = newSplit; + this.pulsarConsumer = consumer; + } + + public void wakeUp() { + wakeup.compareAndSet(false, true); + } + + public void close() { + if (pulsarConsumer != null) { + sneakyClient(() -> pulsarConsumer.close()); + } + } + + @Nullable + protected abstract Message pollMessage(Duration timeout) + throws ExecutionException, InterruptedException, PulsarClientException; + + protected abstract void finishedPollMessage(Message message); + + protected abstract void startConsumer(PulsarPartitionSplit split, Consumer consumer); + + // --------------------------- Helper Methods ----------------------------- + + protected boolean isNotWakeup() { + return !wakeup.get(); + } + + /** Create a specified {@link Consumer} by the given split information. */ + protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { + return createPulsarConsumer(split.getPartition()); + } + + /** Create a specified {@link Consumer} by the given topic partition. */ + protected Consumer createPulsarConsumer(TopicPartition partition) { + ConsumerBuilder consumerBuilder = + PulsarUtils.createConsumerBuilder(pulsarClient, Schema.BYTES, configuration); + + consumerBuilder.topic(partition.getFullTopicName()); + + // Add KeySharedPolicy for Key_Shared subscription. + if (sourceConfiguration.getSubscriptionType() == SubscriptionType.Key_Shared) { + KeySharedPolicy policy = + KeySharedPolicy.stickyHashRange().ranges(partition.getPulsarRange()); + consumerBuilder.keySharedPolicy(policy); + } + + // Create the consumer configuration by using common utils. + return sneakyClient(consumerBuilder::subscribe); + } + + + public void commit(TopicPartition partition, MessageId offsetsToCommit) { + + }; +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/v1/PulsarUnorderedPartitionSplitReader.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/v1/PulsarUnorderedPartitionSplitReader.java new file mode 100644 index 000000000..dd7a33388 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/reader/split/v1/PulsarUnorderedPartitionSplitReader.java @@ -0,0 +1,173 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.reader.split.v1; + +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.connector.pulsar.common.utils.PulsarTransactionUtils; +import com.bytedance.bitsail.connector.pulsar.source.config.SourceConfiguration; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplit; +import com.bytedance.bitsail.connector.pulsar.source.split.v1.PulsarPartitionSplitState; + +import org.apache.flink.annotation.Internal; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; +import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException; +import org.apache.pulsar.client.api.transaction.TxnID; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; + +@Internal +public class PulsarUnorderedPartitionSplitReader extends PulsarPartitionSplitReader{ + private static final Logger LOG = + LoggerFactory.getLogger(PulsarUnorderedPartitionSplitReader.class); + + private static final Duration REDELIVER_TIME = Duration.ofSeconds(3); + + private final TransactionCoordinatorClient coordinatorClient; + + @Nullable private Transaction uncommittedTransaction; + + public PulsarUnorderedPartitionSplitReader( + PulsarClient pulsarClient, + PulsarAdmin pulsarAdmin, + BitSailConfiguration configuration, + SourceConfiguration sourceConfiguration, + TransactionCoordinatorClient coordinatorClient) { + super(pulsarClient, pulsarAdmin, configuration, sourceConfiguration); + + this.coordinatorClient = coordinatorClient; + } + + @Override + protected Message pollMessage(Duration timeout) + throws ExecutionException, InterruptedException, PulsarClientException { + Message message = + pulsarConsumer.receive(Math.toIntExact(timeout.toMillis()), TimeUnit.MILLISECONDS); + + // Skip the message when receive timeout + if (message == null) { + return null; + } + + if (!sourceConfiguration.isEnableAutoAcknowledgeMessage()) { + if (uncommittedTransaction == null) { + // Create a transaction. + this.uncommittedTransaction = newTransaction(); + } + + try { + // Add this message into transaction. + pulsarConsumer + .acknowledgeAsync(message.getMessageId(), uncommittedTransaction) + .get(); + } catch (InterruptedException e) { + sneakyClient( + () -> + pulsarConsumer.reconsumeLater( + message, REDELIVER_TIME.toMillis(), TimeUnit.MILLISECONDS)); + Thread.currentThread().interrupt(); + throw e; + } catch (ExecutionException e) { + sneakyClient( + () -> + pulsarConsumer.reconsumeLater( + message, REDELIVER_TIME.toMillis(), TimeUnit.MILLISECONDS)); + throw e; + } + } + + return message; + } + + @Override + protected void finishedPollMessage(Message message) { + if (sourceConfiguration.isEnableAutoAcknowledgeMessage()) { + sneakyClient(() -> pulsarConsumer.acknowledge(message)); + } + + // Release message + message.release(); + } + + @Override + protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + TxnID uncommittedTransactionId = split.getUncommittedTransactionId(); + + // Abort the uncommitted pulsar transaction. + if (uncommittedTransactionId != null) { + if (coordinatorClient != null) { + try { + coordinatorClient.abort(uncommittedTransactionId); + } catch (TransactionCoordinatorClientException e) { + LOG.error( + "Failed to abort the uncommitted transaction {} when restart the reader", + uncommittedTransactionId, + e); + } + } + + // Redeliver unacknowledged messages because of the message is out of order. + consumer.redeliverUnacknowledgedMessages(); + } + } + + public PulsarPartitionSplitState snapshotState(long checkpointId) { + PulsarPartitionSplitState state = new PulsarPartitionSplitState(registeredSplit); + + // Avoiding NP problem when Pulsar don't get the message before Flink checkpoint. + if (uncommittedTransaction != null) { + TxnID txnID = PulsarTransactionUtils.getId(uncommittedTransaction); + this.uncommittedTransaction = newTransaction(); + state.setUncommittedTransactionId(txnID); + } + + return state; + } + + private Transaction newTransaction() { + long timeoutMillis = sourceConfiguration.getTransactionTimeoutMillis(); + CompletableFuture future = + sneakyClient(pulsarClient::newTransaction) + .withTransactionTimeout(timeoutMillis, TimeUnit.MILLISECONDS) + .build(); + + try { + return future.get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } catch (ExecutionException e) { + throw new RuntimeException(e); + } + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/PulsarPartitionSplit.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/PulsarPartitionSplit.java new file mode 100644 index 000000000..b7ebfce3e --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/PulsarPartitionSplit.java @@ -0,0 +1,119 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.split; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceSplit; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.transaction.TxnID; + +import javax.annotation.Nullable; + +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** A {@link SourceSplit} implementation for a Pulsar's partition. */ +@Internal +public class PulsarPartitionSplit implements SourceSplit { + + private final TopicPartition partition; + + private final StopCursor stopCursor; + + /** + * Since this field in only used in {@link PulsarOrderedSourceReader#snapshotState(long)}, it's + * no need to serialize this field into flink checkpoint state. + */ + @Nullable private final MessageId latestConsumedId; + + @Nullable private final TxnID uncommittedTransactionId; + + public PulsarPartitionSplit(TopicPartition partition, StopCursor stopCursor) { + this.partition = checkNotNull(partition); + this.stopCursor = checkNotNull(stopCursor); + this.latestConsumedId = null; + this.uncommittedTransactionId = null; + } + + public PulsarPartitionSplit( + TopicPartition partition, + StopCursor stopCursor, + MessageId latestConsumedId, + TxnID uncommittedTransactionId) { + this.partition = checkNotNull(partition); + this.stopCursor = checkNotNull(stopCursor); + this.latestConsumedId = latestConsumedId; + this.uncommittedTransactionId = uncommittedTransactionId; + } + + @Override + public String splitId() { + return partition.toString(); + } + + public TopicPartition getPartition() { + return partition; + } + + public StopCursor getStopCursor() { + return stopCursor; + } + + @Nullable + public MessageId getLatestConsumedId() { + return latestConsumedId; + } + + @Nullable + public TxnID getUncommittedTransactionId() { + return uncommittedTransactionId; + } + + /** Open stop cursor. */ + public void open(PulsarAdmin admin) { + stopCursor.open(admin, partition); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit that = (com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit) o; + return partition.equals(that.partition); + } + + @Override + public int hashCode() { + return Objects.hash(partition); + } + + @Override + public String toString() { + return "PulsarPartitionSplit{partition=" + partition + '}'; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/PulsarPartitionSplitSerializer.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/PulsarPartitionSplitSerializer.java new file mode 100644 index 000000000..227d7bcd2 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/PulsarPartitionSplitSerializer.java @@ -0,0 +1,158 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.split; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.transaction.TxnID; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarSerdeUtils.deserializeBytes; +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarSerdeUtils.deserializeObject; +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarSerdeUtils.serializeBytes; +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarSerdeUtils.serializeObject; + +/** The {@link SimpleVersionedSerializer serializer} for {@link PulsarPartitionSplit}. */ +public class PulsarPartitionSplitSerializer + implements SimpleVersionedSerializer { + + public static final com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplitSerializer INSTANCE = + new com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplitSerializer(); + + // This version should be bumped after modifying the PulsarPartitionSplit. + public static final int CURRENT_VERSION = 0; + + private PulsarPartitionSplitSerializer() { + // Singleton instance. + } + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(PulsarPartitionSplit obj) throws IOException { + // VERSION 0 serialization + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + serializePulsarPartitionSplit(out, obj); + out.flush(); + return baos.toByteArray(); + } + } + + @Override + public PulsarPartitionSplit deserialize(int version, byte[] serialized) throws IOException { + // VERSION 0 deserialization + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + return deserializePulsarPartitionSplit(version, in); + } + } + + // ----------------- helper methods -------------- + + public void serializePulsarPartitionSplit(DataOutputStream out, PulsarPartitionSplit split) + throws IOException { + // partition + serializeTopicPartition(out, split.getPartition()); + + // stopCursor + serializeObject(out, split.getStopCursor()); + + // latestConsumedId + MessageId latestConsumedId = split.getLatestConsumedId(); + if (latestConsumedId == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + serializeBytes(out, latestConsumedId.toByteArray()); + } + + // uncommittedTransactionId + TxnID uncommittedTransactionId = split.getUncommittedTransactionId(); + if (uncommittedTransactionId == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeLong(uncommittedTransactionId.getMostSigBits()); + out.writeLong(uncommittedTransactionId.getLeastSigBits()); + } + } + + public PulsarPartitionSplit deserializePulsarPartitionSplit(int version, DataInputStream in) + throws IOException { + // partition + TopicPartition partition = deserializeTopicPartition(version, in); + + // stopCursor + StopCursor stopCursor = deserializeObject(in); + + // latestConsumedId + MessageId latestConsumedId = null; + if (in.readBoolean()) { + byte[] messageIdBytes = deserializeBytes(in); + latestConsumedId = MessageId.fromByteArray(messageIdBytes); + } + + // uncommittedTransactionId + TxnID uncommittedTransactionId = null; + if (in.readBoolean()) { + long mostSigBits = in.readLong(); + long leastSigBits = in.readLong(); + uncommittedTransactionId = new TxnID(mostSigBits, leastSigBits); + } + + // Creation + return new PulsarPartitionSplit( + partition, stopCursor, latestConsumedId, uncommittedTransactionId); + } + + public void serializeTopicPartition(DataOutputStream out, TopicPartition partition) + throws IOException { + // VERSION 0 serialization + TopicRange range = partition.getRange(); + out.writeUTF(partition.getTopic()); + out.writeInt(partition.getPartitionId()); + out.writeInt(range.getStart()); + out.writeInt(range.getEnd()); + } + + public TopicPartition deserializeTopicPartition(int version, DataInputStream in) + throws IOException { + // VERSION 0 deserialization + String topic = in.readUTF(); + int partitionId = in.readInt(); + int start = in.readInt(); + int end = in.readInt(); + + TopicRange range = new TopicRange(start, end); + return new TopicPartition(topic, partitionId, range); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/PulsarPartitionSplitState.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/PulsarPartitionSplitState.java new file mode 100644 index 000000000..fd7a56555 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/PulsarPartitionSplitState.java @@ -0,0 +1,73 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.split; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.transaction.TxnID; + +import javax.annotation.Nullable; + +/** Pulsar partition split state. */ +public class PulsarPartitionSplitState { + + private final PulsarPartitionSplit split; + + @Nullable private TxnID uncommittedTransactionId; + + @Nullable private MessageId latestConsumedId; + + public PulsarPartitionSplitState(PulsarPartitionSplit split) { + this.split = split; + } + + /** + * Create a partition split which contains the latest consumed message id as the start position. + */ + public PulsarPartitionSplit toPulsarPartitionSplit() { + return new PulsarPartitionSplit( + split.getPartition(), + split.getStopCursor(), + latestConsumedId, + uncommittedTransactionId); + } + + public TopicPartition getPartition() { + return split.getPartition(); + } + + @Nullable + public TxnID getUncommittedTransactionId() { + return uncommittedTransactionId; + } + + public void setUncommittedTransactionId(@Nullable TxnID uncommittedTransactionId) { + this.uncommittedTransactionId = uncommittedTransactionId; + } + + @Nullable + public MessageId getLatestConsumedId() { + return latestConsumedId; + } + + public void setLatestConsumedId(@Nullable MessageId latestConsumedId) { + this.latestConsumedId = latestConsumedId; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/v1/PulsarPartitionSplit.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/v1/PulsarPartitionSplit.java new file mode 100644 index 000000000..904558970 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/v1/PulsarPartitionSplit.java @@ -0,0 +1,124 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.split.v1; + +import com.bytedance.bitsail.base.connector.reader.v1.SourceSplit; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; + +import org.apache.flink.annotation.Internal; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.transaction.TxnID; + +import javax.annotation.Nullable; + +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** A {@link SourceSplit} implementation for a Pulsar's partition. */ +@Internal +public class PulsarPartitionSplit implements SourceSplit { + + private final TopicPartition partition; + + private final StopCursor stopCursor; + + /** + * Since this field in only used in {@link PulsarOrderedSourceReader#snapshotState(long)}, it's + * no need to serialize this field into flink checkpoint state. + */ + @Nullable private final MessageId latestConsumedId; + + @Nullable private final TxnID uncommittedTransactionId; + + public PulsarPartitionSplit(TopicPartition partition, StopCursor stopCursor) { + this.partition = checkNotNull(partition); + this.stopCursor = checkNotNull(stopCursor); + this.latestConsumedId = null; + this.uncommittedTransactionId = null; + } + + public PulsarPartitionSplit( + TopicPartition partition, + StopCursor stopCursor, + MessageId latestConsumedId, + TxnID uncommittedTransactionId) { + this.partition = checkNotNull(partition); + this.stopCursor = checkNotNull(stopCursor); + this.latestConsumedId = latestConsumedId; + this.uncommittedTransactionId = uncommittedTransactionId; + } + + public String splitId() { + return partition.toString(); + } + + public TopicPartition getPartition() { + return partition; + } + + public StopCursor getStopCursor() { + return stopCursor; + } + + @Nullable + public MessageId getLatestConsumedId() { + return latestConsumedId; + } + + @Nullable + public TxnID getUncommittedTransactionId() { + return uncommittedTransactionId; + } + + /** Open stop cursor. */ + public void open(PulsarAdmin admin) { + stopCursor.open(admin, partition); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarPartitionSplit that = (PulsarPartitionSplit) o; + return partition.equals(that.partition); + } + + @Override + public int hashCode() { + return Objects.hash(partition); + } + + @Override + public String toString() { + return "PulsarPartitionSplit{partition=" + partition + '}'; + } + + @Override + public String uniqSplitId() { + return partition.toString(); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/v1/PulsarPartitionSplitSerializer.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/v1/PulsarPartitionSplitSerializer.java new file mode 100644 index 000000000..796ea5ccc --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/v1/PulsarPartitionSplitSerializer.java @@ -0,0 +1,158 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.split.v1; + +import com.bytedance.bitsail.base.serializer.BinarySerializer; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StopCursor; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicRange; + +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.transaction.TxnID; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarSerdeUtils.deserializeBytes; +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarSerdeUtils.deserializeObject; +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarSerdeUtils.serializeBytes; +import static com.bytedance.bitsail.connector.pulsar.common.utils.PulsarSerdeUtils.serializeObject; + +/** The {@link SimpleVersionedSerializer serializer} for {@link PulsarPartitionSplit}. */ +public class PulsarPartitionSplitSerializer + implements BinarySerializer { + + public static final PulsarPartitionSplitSerializer INSTANCE = + new PulsarPartitionSplitSerializer(); + + // This version should be bumped after modifying the PulsarPartitionSplit. + public static final int CURRENT_VERSION = 0; + + private PulsarPartitionSplitSerializer() { + // Singleton instance. + } + + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(PulsarPartitionSplit obj) throws IOException { + // VERSION 0 serialization + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + serializePulsarPartitionSplit(out, obj); + out.flush(); + return baos.toByteArray(); + } + } + + @Override + public PulsarPartitionSplit deserialize(byte[] serialized) throws IOException { + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + return deserializePulsarPartitionSplit(0, in); + } + } + + + // ----------------- helper methods -------------- + + public void serializePulsarPartitionSplit(DataOutputStream out, PulsarPartitionSplit split) + throws IOException { + // partition + serializeTopicPartition(out, split.getPartition()); + + // stopCursor + serializeObject(out, split.getStopCursor()); + + // latestConsumedId + MessageId latestConsumedId = split.getLatestConsumedId(); + if (latestConsumedId == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + serializeBytes(out, latestConsumedId.toByteArray()); + } + + // uncommittedTransactionId + TxnID uncommittedTransactionId = split.getUncommittedTransactionId(); + if (uncommittedTransactionId == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeLong(uncommittedTransactionId.getMostSigBits()); + out.writeLong(uncommittedTransactionId.getLeastSigBits()); + } + } + + public PulsarPartitionSplit deserializePulsarPartitionSplit(int version, DataInputStream in) + throws IOException { + // partition + TopicPartition partition = deserializeTopicPartition(version, in); + + // stopCursor + StopCursor stopCursor = deserializeObject(in); + + // latestConsumedId + MessageId latestConsumedId = null; + if (in.readBoolean()) { + byte[] messageIdBytes = deserializeBytes(in); + latestConsumedId = MessageId.fromByteArray(messageIdBytes); + } + + // uncommittedTransactionId + TxnID uncommittedTransactionId = null; + if (in.readBoolean()) { + long mostSigBits = in.readLong(); + long leastSigBits = in.readLong(); + uncommittedTransactionId = new TxnID(mostSigBits, leastSigBits); + } + + // Creation + return new PulsarPartitionSplit( + partition, stopCursor, latestConsumedId, uncommittedTransactionId); + } + + public void serializeTopicPartition(DataOutputStream out, TopicPartition partition) + throws IOException { + // VERSION 0 serialization + TopicRange range = partition.getRange(); + out.writeUTF(partition.getTopic()); + out.writeInt(partition.getPartitionId()); + out.writeInt(range.getStart()); + out.writeInt(range.getEnd()); + } + + public TopicPartition deserializeTopicPartition(int version, DataInputStream in) + throws IOException { + // VERSION 0 deserialization + String topic = in.readUTF(); + int partitionId = in.readInt(); + int start = in.readInt(); + int end = in.readInt(); + + TopicRange range = new TopicRange(start, end); + return new TopicPartition(topic, partitionId, range); + } +} diff --git a/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/v1/PulsarPartitionSplitState.java b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/v1/PulsarPartitionSplitState.java new file mode 100644 index 000000000..73af720b5 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/main/java/com/bytedance/bitsail/connector/pulsar/source/split/v1/PulsarPartitionSplitState.java @@ -0,0 +1,73 @@ +/* + * 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 com.bytedance.bitsail.connector.pulsar.source.split.v1; + +import com.bytedance.bitsail.connector.pulsar.source.enumerator.topic.TopicPartition; + +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.transaction.TxnID; + +import javax.annotation.Nullable; + +/** Pulsar partition split state. */ +public class PulsarPartitionSplitState { + + private final PulsarPartitionSplit split; + + @Nullable private TxnID uncommittedTransactionId; + + @Nullable private MessageId latestConsumedId; + + public PulsarPartitionSplitState(PulsarPartitionSplit split) { + this.split = split; + } + + /** + * Create a partition split which contains the latest consumed message id as the start position. + */ + public PulsarPartitionSplit toPulsarPartitionSplit() { + return new PulsarPartitionSplit( + split.getPartition(), + split.getStopCursor(), + latestConsumedId, + uncommittedTransactionId); + } + + public TopicPartition getPartition() { + return split.getPartition(); + } + + @Nullable + public TxnID getUncommittedTransactionId() { + return uncommittedTransactionId; + } + + public void setUncommittedTransactionId(@Nullable TxnID uncommittedTransactionId) { + this.uncommittedTransactionId = uncommittedTransactionId; + } + + @Nullable + public MessageId getLatestConsumedId() { + return latestConsumedId; + } + + public void setLatestConsumedId(@Nullable MessageId latestConsumedId) { + this.latestConsumedId = latestConsumedId; + } +} diff --git a/bitsail-connectors/connector-pulsar/src/test/java/com/bytedance/bitsail/connector/pulsar/source/PulsarConnectorTest.java b/bitsail-connectors/connector-pulsar/src/test/java/com/bytedance/bitsail/connector/pulsar/source/PulsarConnectorTest.java new file mode 100644 index 000000000..8ec2f8260 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/test/java/com/bytedance/bitsail/connector/pulsar/source/PulsarConnectorTest.java @@ -0,0 +1,308 @@ +package com.bytedance.bitsail.connector.pulsar.source; + +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; +import com.bytedance.bitsail.common.option.ReaderOptions; +import com.bytedance.bitsail.common.typeinfo.TypeInfo; +import com.bytedance.bitsail.common.typeinfo.TypeInfoUtils; +import com.bytedance.bitsail.common.util.JsonSerializer; +import com.bytedance.bitsail.connector.pulsar.source.enumerator.cursor.StartCursor; +import com.bytedance.bitsail.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import com.bytedance.bitsail.connector.pulsar.testutils.IntegerSource; +import com.bytedance.bitsail.test.connector.test.EmbeddedFlinkCluster; +import com.bytedance.bitsail.test.connector.test.utils.JobConfUtils; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.connectors.pulsar.FlinkPulsarSink; +import org.apache.flink.streaming.connectors.pulsar.internal.PulsarSerializationSchemaWrapper; +import org.apache.flink.streaming.util.TestStreamEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.pulsar.client.admin.PulsarAdmin; +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.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.BindMode; +import org.testcontainers.containers.Network; +import org.testcontainers.containers.PulsarContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; +import org.testcontainers.utility.DockerImageName; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import static com.bytedance.bitsail.connector.pulsar.common.config.v1.PulsarOptionsV1.PULSAR_ADMIN_URL; +import static com.bytedance.bitsail.connector.pulsar.common.config.v1.PulsarOptionsV1.PULSAR_SERVICE_URL; +import static java.time.temporal.ChronoUnit.SECONDS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import static org.testcontainers.containers.PulsarContainer.BROKER_HTTP_PORT; + +public class PulsarConnectorTest extends EmbeddedFlinkCluster { + private static final Logger log = LoggerFactory.getLogger(PulsarConnectorTest.class); + + private static final String PULSAR_SOURCE_VERSION = "4.9.4"; + private static final DockerImageName PULSAR_DOCKER_IMAGE = DockerImageName + .parse("apache/rocketmq:4.9.4") + .withTag(PULSAR_SOURCE_VERSION); + private static final Network NETWORK = Network.newNetwork(); + + private static final String DEFAULT_PRODUCE_GROUP = "bitsail"; + private static final String DEFAULT_TOPIC = "TBW102"; + private static final int DEFAULT_QUEUE_NUMBER = 5; + private PulsarContainer pulsarService; + private static String serviceUrl; + private static String adminUrl; + + private PulsarAdmin admin; + + @Before + public void before() throws Throwable { + log.info("Starting PulsarTestBase "); + final String pulsarImage = + System.getProperty("pulsar.systemtest.image", "apachepulsar/pulsar:2.8.0"); + DockerImageName pulsar = + DockerImageName.parse(pulsarImage).asCompatibleSubstituteFor("apachepulsar/pulsar"); + pulsarService = new PulsarContainer(pulsar); + pulsarService.withClasspathResourceMapping( + "pulsar/txnStandalone.conf", "/pulsar/conf/standalone.conf", BindMode.READ_ONLY); + pulsarService.waitingFor( + new HttpWaitStrategy() + .forPort(BROKER_HTTP_PORT) + .forStatusCode(200) + .forPath("/admin/v2/namespaces/public/default") + .withStartupTimeout(Duration.of(40, SECONDS))); + pulsarService.start(); + pulsarService.followOutput(new Slf4jLogConsumer(log)); + serviceUrl = pulsarService.getPulsarBrokerUrl(); + adminUrl = pulsarService.getHttpServiceUrl(); + + log.info( + "Successfully started pulsar service at cluster " + + pulsarService.getContainerName()); + } + + + @Test + public void testPulsarSourceV1() throws Exception { + BitSailConfiguration jobConf = JobConfUtils.fromClasspath("bitsail_pulsar_print.json"); + jobConf.set(PULSAR_SERVICE_URL, serviceUrl); + jobConf.set(PULSAR_ADMIN_URL, adminUrl); + PulsarSourceV1 pulsarSource = new PulsarSourceV1(); + TypeInfo[] typeInfos = TypeInfoUtils.getTypeInfos(pulsarSource.createTypeInfoConverter(), + jobConf.get(ReaderOptions.BaseReaderOptions.COLUMNS)); + + startProduceMessages(typeInfos); + EmbeddedFlinkCluster.submitJob(jobConf); + } + + + + private void startProduceMessages(TypeInfo[] typeInfos) throws PulsarClientException { + PulsarClient client = + PulsarClient.builder().enableTransaction(true).serviceUrl(serviceUrl).build(); + Producer producer = client.newProducer(Schema.BYTES).topic(DEFAULT_TOPIC).create(); + ScheduledExecutorService scheduledExecutor = Executors.newSingleThreadScheduledExecutor(); + scheduledExecutor.scheduleAtFixedRate( + new Thread(() -> { + try { + for (int i = 0; i < 100; i++) { + producer.send(fakeJsonObject(i, typeInfos)); + } + } catch (Exception e) { + log.error("Produce failed.", e); + } + }), 0, 15, TimeUnit.SECONDS); + } + + private static byte[] fakeJsonObject(int index, TypeInfo[] typeInfos) { + Map demo = Maps.newHashMap(); + demo.put("id", index); + return JsonSerializer.serialize(demo).getBytes(); + } + + + + @Test + public void testPulsarSource() throws Exception { + final String topic = "ExactlyOnceTopicSource" + UUID.randomUUID(); + + // produce messages + PulsarClient client = + PulsarClient.builder().enableTransaction(true).serviceUrl(serviceUrl).build(); + Producer producer = client.newProducer(Schema.STRING).topic(topic).create(); + for (int i = 0; i < 100; i++) { + producer.send(String.valueOf(i)); + } + + PulsarSource source = PulsarSource.builder() + .setServiceUrl(serviceUrl) + .setAdminUrl(adminUrl) + .setStartCursor(StartCursor.earliest()) + .setTopics(topic) + .setDeserializationSchema(PulsarDeserializationSchema.flinkSchema(new SimpleStringSchema())) + .setSubscriptionName("my-subscription") + .setSubscriptionType(SubscriptionType.Exclusive) + .build(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + DataStreamSource ds = env.fromSource(source, WatermarkStrategy.noWatermarks(), "Pulsar Source"); + ds.print(); + env.execute("Exactly once test"); + + } + + @Test + public void testPulsarSink() throws Exception { + testSink(1); + } + + protected void testSink(int sinksCount) throws Exception { + final String topic = "ExactlyOnceTopicSink" + UUID.randomUUID(); + final int numElements = 1000; + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(500); + env.setParallelism(1); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); + + // process exactly failAfterElements number of elements and then shutdown Pulsar broker and + // fail application + List expectedElements = getIntegersSequence(numElements); + + DataStream inputStream = + env.addSource(new IntegerSource(numElements)); + + for (int i = 0; i < sinksCount; i++) { + ClientConfigurationData clientConfigurationData = new ClientConfigurationData(); + clientConfigurationData.setServiceUrl(serviceUrl); + clientConfigurationData.setEnableTransaction(true); + SinkFunction sink = + new FlinkPulsarSink<>( + adminUrl, + Optional.of(topic), + clientConfigurationData, + new Properties(), + new PulsarSerializationSchemaWrapper.Builder<>( + (SerializationSchema) + element -> Schema.INT32.encode(element)) + .useAtomicMode(DataTypes.INT()) + .build()); + inputStream.addSink(sink); + } + + env.execute("Exactly once test"); + for (int i = 0; i < sinksCount; i++) { + // assert that before failure we successfully snapshot/flushed all expected elements + assertExactlyOnceForTopic(topic, expectedElements, 60000L); + } + } + + private List getIntegersSequence(int size) { + List result = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + result.add(i); + } + return result; + } + + /** + * We manually handle the timeout instead of using JUnit's timeout to return failure instead of + * timeout error. After timeout we assume that there are missing records and there is a bug, not + * that the test has run out of time. + */ + public void assertExactlyOnceForTopic( + String topic, List expectedElements, long timeoutMillis) throws Exception { + + long startMillis = System.currentTimeMillis(); + List actualElements = new ArrayList<>(); + + // until we timeout... + PulsarClient client = + PulsarClient.builder().enableTransaction(true).serviceUrl(serviceUrl).build(); + Consumer test = + client.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("test-exactly" + UUID.randomUUID()) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + while (System.currentTimeMillis() < startMillis + timeoutMillis) { + // query pulsar for new records ... + Message message = test.receive(); + log.info( + "consume the message {} with the value {}", + message.getMessageId(), + message.getValue()); + actualElements.add(message.getValue()); + // succeed if we got all expectedElements + if (actualElements.size() == expectedElements.size()) { + assertEquals(expectedElements, actualElements); + return; + } + if (actualElements.equals(expectedElements)) { + return; + } + // fail early if we already have too many elements + if (actualElements.size() > expectedElements.size()) { + break; + } + } + + fail( + String.format( + "Expected %s, but was: %s", + formatElements(expectedElements), formatElements(actualElements))); + } + + private String formatElements(List elements) { + if (elements.size() > 50) { + return String.format("number of elements: <%s>", elements.size()); + } else { + return String.format("elements: <%s>", elements); + } + } + + + @After + public void after() { + log.info("-------------------------------------------------------------------------"); + log.info(" Shut down PulsarTestBase "); + log.info("-------------------------------------------------------------------------"); + + TestStreamEnvironment.unsetAsContext(); + + if (pulsarService != null) { + pulsarService.stop(); + } + + log.info("-------------------------------------------------------------------------"); + log.info(" PulsarTestBase finished"); + log.info("-------------------------------------------------------------------------"); + } +} \ No newline at end of file diff --git a/bitsail-connectors/connector-pulsar/src/test/java/com/bytedance/bitsail/connector/pulsar/testutils/IntegerSource.java b/bitsail-connectors/connector-pulsar/src/test/java/com/bytedance/bitsail/connector/pulsar/testutils/IntegerSource.java new file mode 100644 index 000000000..1d1d6b6d3 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/test/java/com/bytedance/bitsail/connector/pulsar/testutils/IntegerSource.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 com.bytedance.bitsail.connector.pulsar.testutils; + +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.runtime.state.CheckpointListener; +import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.util.SerializableObject; + +import java.util.Collections; +import java.util.List; + +/** A source generate integer test data. */ +@Slf4j +public class IntegerSource extends RichParallelSourceFunction + implements ListCheckpointed, CheckpointListener { + /** + * Blocker when the generator needs to wait for the checkpoint to happen. Eager initialization + * means it must be serializable (pick any serializable type). + */ + private final Object blocker = new SerializableObject(); + + /** The total number of events to generate. */ + private final int numEventsTotal; + + /** The current position in the sequence of numbers. */ + private int currentPosition = -1; + + private long lastCheckpointTriggered; + + private long lastCheckpointConfirmed; + + private boolean restored; + + private volatile boolean running = true; + + public IntegerSource(int numEventsTotal) { + this.numEventsTotal = numEventsTotal; + } + + @Override + public void run(SourceContext ctx) throws Exception { + + // each source subtask emits only the numbers where (num % parallelism == subtask_index) + final int stepSize = getRuntimeContext().getNumberOfParallelSubtasks(); + int current = + this.currentPosition >= 0 + ? this.currentPosition + : getRuntimeContext().getIndexOfThisSubtask(); + + while (this.running && current < this.numEventsTotal) { + // emit the next element + synchronized (ctx.getCheckpointLock()) { + ctx.collect(current); + current += stepSize; + this.currentPosition = current; + } + // give some time to trigger checkpoint while we are not holding the lock (to prevent + // starvation) + if (!restored && current % 10 == 0) { + Thread.sleep(1); + } + } + + // after we are done, we need to wait for two more checkpoint to complete + // before finishing the program - that is to be on the safe side that + // the sink also got the "commit" notification for all relevant checkpoints + // and committed the data + final long lastCheckpoint; + synchronized (ctx.getCheckpointLock()) { + lastCheckpoint = this.lastCheckpointTriggered; + } + + synchronized (this.blocker) { + while (this.lastCheckpointConfirmed <= lastCheckpoint + 1) { + this.blocker.wait(); + } + } + } + + @Override + public void cancel() { + this.running = false; + } + + @Override + public List snapshotState(long checkpointId, long checkpointTimestamp) + throws Exception { + this.lastCheckpointTriggered = checkpointId; + log.info( + "checkpoint {} is snapshotState, the currentPosition is {}", + checkpointId, + this.currentPosition); + return Collections.singletonList(this.currentPosition); + } + + @Override + public void restoreState(List state) throws Exception { + this.currentPosition = state.get(0); + log.info("source restoreState and set the position to {}", currentPosition); + // at least one checkpoint must have happened so far + this.lastCheckpointTriggered = 1L; + this.lastCheckpointConfirmed = 1L; + this.restored = true; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + synchronized (blocker) { + this.lastCheckpointConfirmed = checkpointId; + blocker.notifyAll(); + } + } + + @Override + public void notifyCheckpointAborted(long checkpointId) { + // Nothing to do. + } +} diff --git a/bitsail-connectors/connector-pulsar/src/test/resources/bitsail_pulsar_print.json b/bitsail-connectors/connector-pulsar/src/test/resources/bitsail_pulsar_print.json new file mode 100644 index 000000000..dd75322dd --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/test/resources/bitsail_pulsar_print.json @@ -0,0 +1,48 @@ +{ + "job": { + "common": { + "job_type": "STREAMING", + "job_plugin_lib_dir": "plugin", + "job_plugin_conf_dir": "plugin_conf", + "enable_dynamic_loader": true, + "instance_id": "1", + "internal_instance_id": "1", + "extra_properties": { + "update-mode": "append" + } + }, + "reader": { + "pulsar": { + "source": { + "enableAutoAcknowledgeMessage": true + }, + "client": { + "serviceUrl": "pulsar://lcoalhost:8080" + }, + "admin": { + "adminUrl": "http://localhost:6650", + "topics": "TBW102", + "topicMode": "list" + }, + "consumer": { + "startCursorMode": "latest", + "subscriptionType": "Exclusive", + "consumerName": "my_consumer", + "subscriptionName": "my_subscription" + } + }, + + "columns": [ + { + "name": "id", + "type": "string" + } + ], + "format_type": "json", + "class": "com.bytedance.bitsail.connector.pulsar.source.PulsarSourceV1" + }, + "writer": { + "class": "com.bytedance.bitsail.connector.legacy.print.sink.PrintSink" + } + } +} \ No newline at end of file diff --git a/bitsail-connectors/connector-pulsar/src/test/resources/pulsar/auth-client.conf b/bitsail-connectors/connector-pulsar/src/test/resources/pulsar/auth-client.conf new file mode 100644 index 000000000..3ff6605f6 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/test/resources/pulsar/auth-client.conf @@ -0,0 +1,64 @@ +# +# Licensed 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 for pulsar-client and pulsar-admin CLI tools + +# URL for Pulsar REST API (for admin operations) +# For TLS: +# webServiceUrl=https://localhost:8443/ +webServiceUrl=http://localhost:8080/ + +# URL for Pulsar Binary Protocol (for produce and consume operations) +# For TLS: +# brokerServiceUrl=pulsar+ssl://localhost:6651/ +brokerServiceUrl=pulsar://localhost:6650 + +# Authentication plugin to authenticate with servers +# e.g. for TLS +# authPlugin=org.apache.pulsar.client.impl.auth.AuthenticationTls +authPlugin=org.apache.pulsar.client.impl.auth.AuthenticationToken + +# Parameters passed to authentication plugin.# +# A comma separated list of key:value pairs. +# Keys depend on the configured authPlugin. +# e.g. for TLS +# authParams=tlsCertFile:/path/to/client-cert.pem,tlsKeyFile:/path/to/client-key.pem +authParams=token:eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJ1c2VyMSJ9.2AgtxHe8-2QBV529B5DrRtpuqP6RJjrk21Mhnomfivo + +# Allow TLS connections to servers whose certificate cannot be +# be verified to have been signed by a trusted certificate +# authority. +tlsAllowInsecureConnection=false + +# Whether server hostname must match the common name of the certificate +# the server is using. +tlsEnableHostnameVerification=false + +# Path for the trusted TLS certificate file. +# This cert is used to verify that any cert presented by a server +# is signed by a certificate authority. If this verification +# 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/bitsail-connectors/connector-pulsar/src/test/resources/pulsar/auth-standalone.conf b/bitsail-connectors/connector-pulsar/src/test/resources/pulsar/auth-standalone.conf new file mode 100644 index 000000000..dda46f919 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/test/resources/pulsar/auth-standalone.conf @@ -0,0 +1,832 @@ +# +# Licensed 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. +# + +### --- General broker settings --- ### + +# Zookeeper quorum connection string +zookeeperServers= + +# Configuration Store connection string +configurationStoreServers= + +brokerServicePort=6650 + +# Port to use to server HTTP request +webServicePort=8080 + +# Hostname or IP address the service binds on, default is 0.0.0.0. +bindAddress=0.0.0.0 + +# Hostname or IP address the service advertises to the outside world. If not set, the value of InetAddress.getLocalHost().getHostName() is used. +advertisedAddress= + +# Number of threads to use for Netty IO. Default is set to 2 * Runtime.getRuntime().availableProcessors() +numIOThreads= + +# Number of threads to use for HTTP requests processing. Default is set to 2 * Runtime.getRuntime().availableProcessors() +numHttpServerThreads= + +# Name of the cluster to which this broker belongs to +clusterName=standalone + +# Enable cluster's failure-domain which can distribute brokers into logical region +failureDomainsEnabled=false + +# Zookeeper session timeout in milliseconds +zooKeeperSessionTimeoutMillis=30000 + +# ZooKeeper operation timeout in seconds +zooKeeperOperationTimeoutSeconds=30 + +# ZooKeeper cache expiry time in seconds +zooKeeperCacheExpirySeconds=300 + +# Time to wait for broker graceful shutdown. After this time elapses, the process will be killed +brokerShutdownTimeoutMs=60000 + +# Flag to skip broker shutdown when broker handles Out of memory error +skipBrokerShutdownOnOOM=false + +# Enable backlog quota check. Enforces action on topic when the quota is reached +backlogQuotaCheckEnabled=true + +# How often to check for topics that have reached the quota +backlogQuotaCheckIntervalInSeconds=60 + +# Default per-topic backlog quota limit +backlogQuotaDefaultLimitGB=10 + +# Default ttl for namespaces if ttl is not already configured at namespace policies. (disable default-ttl with value 0) +ttlDurationDefaultInSeconds=0 + +# Enable the deletion of inactive topics +brokerDeleteInactiveTopicsEnabled=true + +# How often to check for inactive topics +brokerDeleteInactiveTopicsFrequencySeconds=60 + +# Max pending publish requests per connection to avoid keeping large number of pending +# requests in memory. Default: 1000 +maxPendingPublishdRequestsPerConnection=1000 + +# How frequently to proactively check and purge expired messages +messageExpiryCheckIntervalInMinutes=5 + +# How long to delay rewinding cursor and dispatching messages when active consumer is changed +activeConsumerFailoverDelayTimeMillis=1000 + +# How long to delete inactive subscriptions from last consuming +# When it is 0, inactive subscriptions are not deleted automatically +subscriptionExpirationTimeMinutes=0 + +# Enable subscription message redelivery tracker to send redelivery count to consumer (default is enabled) +subscriptionRedeliveryTrackerEnabled=true + +# On KeyShared subscriptions, with default AUTO_SPLIT mode, use splitting ranges or +# consistent hashing to reassign keys to new consumers +subscriptionKeySharedUseConsistentHashing=false + +# On KeyShared subscriptions, number of points in the consistent-hashing ring. +# The higher the number, the more equal the assignment of keys to consumers +subscriptionKeySharedConsistentHashingReplicaPoints=100 + +# How frequently to proactively check and purge expired subscription +subscriptionExpiryCheckIntervalInMinutes=5 + +# Set the default behavior for message deduplication in the broker +# This can be overridden per-namespace. If enabled, broker will reject +# messages that were already stored in the topic +brokerDeduplicationEnabled=false + +# Maximum number of producer information that it's going to be +# persisted for deduplication purposes +brokerDeduplicationMaxNumberOfProducers=10000 + +# Number of entries after which a dedup info snapshot is taken. +# A bigger interval will lead to less snapshots being taken though it would +# increase the topic recovery time, when the entries published after the +# snapshot need to be replayed +brokerDeduplicationEntriesInterval=1000 + +# Time of inactivity after which the broker will discard the deduplication information +# relative to a disconnected producer. Default is 6 hours. +brokerDeduplicationProducerInactivityTimeoutMinutes=360 + +# When a namespace is created without specifying the number of bundle, this +# value will be used as the default +defaultNumberOfNamespaceBundles=4 + +# Enable check for minimum allowed client library version +clientLibraryVersionCheckEnabled=false + +# Path for the file used to determine the rotation status for the broker when responding +# to service discovery health checks +statusFilePath=/usr/local/apache/htdocs + +# Max number of unacknowledged messages allowed to receive messages by a consumer on a shared subscription. Broker will stop sending +# messages to consumer once, this limit reaches until consumer starts acknowledging messages back +# Using a value of 0, is disabling unackeMessage limit check and consumer can receive messages without any restriction +maxUnackedMessagesPerConsumer=50000 + +# Max number of unacknowledged messages allowed per shared subscription. Broker will stop dispatching messages to +# all consumers of the subscription once this limit reaches until consumer starts acknowledging messages back and +# unack count reaches to limit/2. Using a value of 0, is disabling unackedMessage-limit +# check and dispatcher can dispatch messages without any restriction +maxUnackedMessagesPerSubscription=200000 + +# Max number of unacknowledged messages allowed per broker. Once this limit reaches, broker will stop dispatching +# messages to all shared subscription which has higher number of unack messages until subscriptions start +# acknowledging messages back and unack count reaches to limit/2. Using a value of 0, is disabling +# unackedMessage-limit check and broker doesn't block dispatchers +maxUnackedMessagesPerBroker=0 + +# Once broker reaches maxUnackedMessagesPerBroker limit, it blocks subscriptions which has higher unacked messages +# than this percentage limit and subscription will not receive any new messages until that subscription acks back +# limit/2 messages +maxUnackedMessagesPerSubscriptionOnBrokerBlocked=0.16 + +# Tick time to schedule task that checks topic publish rate limiting across all topics +# Reducing to lower value can give more accuracy while throttling publish but +# it uses more CPU to perform frequent check. (Disable publish throttling with value 0) +topicPublisherThrottlingTickTimeMillis=2 + +# Tick time to schedule task that checks broker publish rate limiting across all topics +# Reducing to lower value can give more accuracy while throttling publish but +# it uses more CPU to perform frequent check. (Disable publish throttling with value 0) +brokerPublisherThrottlingTickTimeMillis=50 + +# Max Rate(in 1 seconds) of Message allowed to publish for a broker if broker publish rate limiting enabled +# (Disable message rate limit with value 0) +brokerPublisherThrottlingMaxMessageRate=0 + +# Max Rate(in 1 seconds) of Byte allowed to publish for a broker if broker publish rate limiting enabled +# (Disable byte rate limit with value 0) +brokerPublisherThrottlingMaxByteRate=0 + +# Default messages per second dispatch throttling-limit for every topic. Using a value of 0, is disabling default +# message dispatch-throttling +dispatchThrottlingRatePerTopicInMsg=0 + +# Default bytes per second dispatch throttling-limit for every topic. Using a value of 0, is disabling +# 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: +# throttle-dispatch-rate = (publish-rate + configured dispatch-rate). +dispatchThrottlingRateRelativeToPublishRate=false + +# By default we enable dispatch-throttling for both caught up consumers as well as consumers who have +# backlog. +dispatchThrottlingOnNonBacklogConsumerEnabled=true + +# Precise dispathcer flow control according to history message number of each entry +preciseDispatcherFlowControl=false + +# Max number of concurrent lookup request broker allows to throttle heavy incoming lookup traffic +maxConcurrentLookupRequest=50000 + +# Max number of concurrent topic loading request broker allows to control number of zk-operations +maxConcurrentTopicLoadRequest=5000 + +# Max concurrent non-persistent message can be processed per connection +maxConcurrentNonPersistentMessagePerConnection=1000 + +# Number of worker threads to serve non-persistent topic +numWorkerThreadsForNonPersistentTopic=8 + +# Enable broker to load persistent topics +enablePersistentTopics=true + +# Enable broker to load non-persistent topics +enableNonPersistentTopics=true + +# Max number of producers allowed to connect to topic. Once this limit reaches, Broker will reject new producers +# until the number of connected producers decrease. +# Using a value of 0, is disabling maxProducersPerTopic-limit check. +maxProducersPerTopic=0 + +# Max number of consumers allowed to connect to topic. Once this limit reaches, Broker will reject new consumers +# until the number of connected consumers decrease. +# Using a value of 0, is disabling maxConsumersPerTopic-limit check. +maxConsumersPerTopic=0 + +# Max number of consumers allowed to connect to subscription. Once this limit reaches, Broker will reject new consumers +# until the number of connected consumers decrease. +# Using a value of 0, is disabling maxConsumersPerSubscription-limit check. +maxConsumersPerSubscription=0 + +# Max number of partitions per partitioned topic +# Use 0 or negative number to disable the check +maxNumPartitionsPerPartitionedTopic=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= + +# Enable or disable system topic +systemTopicEnabled=false + +# Enable or disable topic level policies, topic level policies depends on the system topic +# Please enable the system topic first. +topicLevelPoliciesEnabled=false + +### --- 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. +proxyRoles= + +# If this flag is set then the broker authenticates the original Auth data +# else it just accepts the originalPrincipal and authorizes it (if required). +authenticateOriginalAuthData=false + +# Enable authentication +authenticationEnabled=true + +# Autentication provider name list, which is comma separated list of class names +authenticationProviders=org.apache.pulsar.broker.authentication.AuthenticationProviderToken + +tokenSecretKey=data:base64,duaQk8phAAWZH5ohiZV92EE2/qqB3u//XHqE2T3BP10= + +# Enforce authorization +authorizationEnabled=true + +# Authorization provider fully qualified class-name +authorizationProvider=org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider + +# Allow wildcard matching in authorization +# (wildcard matching only applicable if wildcard-char: +# * presents at first or last position eg: *.pulsar.service, pulsar.service.*) +authorizationAllowWildcardsMatching=false + +# Role names that are treated as "super-user", meaning they will be able to do all admin +# operations and publish/consume from all topics +superUserRoles=user1 +#PULSAR_PREFIX_tokenPublicKey=file:///Users/zhaojianyun/auth/token/public.key +# Authentication settings of the broker itself. Used when the broker connects to other brokers, +# either in same or other clusters +brokerClientAuthenticationPlugin=org.apache.pulsar.client.impl.auth.AuthenticationToken +brokerClientAuthenticationParameters=token:eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJ1c2VyMSJ9.2AgtxHe8-2QBV529B5DrRtpuqP6RJjrk21Mhnomfivo + +# Supported Athenz provider domain names(comma separated) for authentication +athenzDomainNames= + +# When this parameter is not empty, unauthenticated users perform as anonymousUserRole +anonymousUserRole= + +# The token "claim" that will be interpreted as the authentication "role" or "principal" by AuthenticationProviderToken (defaults to "sub" if blank) +tokenAuthClaim= +# The token audience "claim" name, e.g. "aud", that will be used to get the audience from token. +# If not set, audience will not be verified. +tokenAudienceClaim= + +# The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this. +tokenAudience= + +### --- BookKeeper Client --- ### + +# Authentication plugin to use when connecting to bookies +bookkeeperClientAuthenticationPlugin= + +# BookKeeper auth plugin implementatation specifics parameters name and values +bookkeeperClientAuthenticationParametersName= +bookkeeperClientAuthenticationParameters= + +# Timeout for BK add / read operations +bookkeeperClientTimeoutInSeconds=30 + +# Speculative reads are initiated if a read request doesn't complete within a certain time +# Using a value of 0, is disabling the speculative reads +bookkeeperClientSpeculativeReadTimeoutInMillis=0 + +# Enable bookies health check. Bookies that have more than the configured number of failure within +# the interval will be quarantined for some time. During this period, new ledgers won't be created +# on these bookies +bookkeeperClientHealthCheckEnabled=true +bookkeeperClientHealthCheckIntervalSeconds=60 +bookkeeperClientHealthCheckErrorThresholdPerInterval=5 +bookkeeperClientHealthCheckQuarantineTimeInSeconds=1800 + +# Enable rack-aware bookie selection policy. BK will chose bookies from different racks when +# forming a new bookie ensemble +bookkeeperClientRackawarePolicyEnabled=true + +# Enable region-aware bookie selection policy. BK will chose bookies from +# different regions and racks when forming a new bookie ensemble. +# If enabled, the value of bookkeeperClientRackawarePolicyEnabled is ignored +bookkeeperClientRegionawarePolicyEnabled=false + +# Enable/disable reordering read sequence on reading entries. +bookkeeperClientReorderReadSequenceEnabled=false + +# Enable bookie isolation by specifying a list of bookie groups to choose from. Any bookie +# outside the specified groups will not be used by the broker +bookkeeperClientIsolationGroups= + +# Enable bookie secondary-isolation group if bookkeeperClientIsolationGroups doesn't +# have enough bookie available. +bookkeeperClientSecondaryIsolationGroups= + +# Minimum bookies that should be available as part of bookkeeperClientIsolationGroups +# else broker will include bookkeeperClientSecondaryIsolationGroups bookies in isolated list. +bookkeeperClientMinAvailableBookiesInIsolationGroups= + +# Set the client security provider factory class name. +# Default: org.apache.bookkeeper.tls.TLSContextFactory +bookkeeperTLSProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory + +# Enable tls authentication with bookie +bookkeeperTLSClientAuthentication=false + +# Supported type: PEM, JKS, PKCS12. Default value: PEM +bookkeeperTLSKeyFileType=PEM + +#Supported type: PEM, JKS, PKCS12. Default value: PEM +bookkeeperTLSTrustCertTypes=PEM + +# Path to file containing keystore password, if the client keystore is password protected. +bookkeeperTLSKeyStorePasswordPath= + +# Path to file containing truststore password, if the client truststore is password protected. +bookkeeperTLSTrustStorePasswordPath= + +# Path for the TLS private key file +bookkeeperTLSKeyFilePath= + +# Path for the TLS certificate file +bookkeeperTLSCertificateFilePath= + +# Path for the trusted TLS certificate file +bookkeeperTLSTrustCertsFilePath= + +# Enable/disable disk weight based placement. Default is false +bookkeeperDiskWeightBasedPlacementEnabled=false + +# Set the interval to check the need for sending an explicit LAC +# A value of '0' disables sending any explicit LACs. Default is 0. +bookkeeperExplicitLacIntervalInMills=0 + +# Expose bookkeeper client managed ledger stats to prometheus. default is false +# bookkeeperClientExposeStatsToPrometheus=false + +### --- Managed Ledger --- ### + +# Number of bookies to use when creating a ledger +managedLedgerDefaultEnsembleSize=1 + +# Number of copies to store for each message +managedLedgerDefaultWriteQuorum=1 + +# Number of guaranteed copies (acks to wait before write is complete) +managedLedgerDefaultAckQuorum=1 + +# Default type of checksum to use when writing to BookKeeper. Default is "CRC32C" +# Other possible options are "CRC32", "MAC" or "DUMMY" (no checksum). +managedLedgerDigestType=CRC32C + +# Number of threads to be used for managed ledger tasks dispatching +managedLedgerNumWorkerThreads=4 + +# Number of threads to be used for managed ledger scheduled tasks +managedLedgerNumSchedulerThreads=4 + +# Amount of memory to use for caching data payload in managed ledger. This memory +# is allocated from JVM direct memory and it's shared across all the topics +# running in the same broker. By default, uses 1/5th of available direct memory +managedLedgerCacheSizeMB= + +# Whether we should make a copy of the entry payloads when inserting in cache +managedLedgerCacheCopyEntries=false + +# Threshold to which bring down the cache level when eviction is triggered +managedLedgerCacheEvictionWatermark=0.9 + +# Configure the cache eviction frequency for the managed ledger cache (evictions/sec) +managedLedgerCacheEvictionFrequency=100.0 + +# All entries that have stayed in cache for more than the configured time, will be evicted +managedLedgerCacheEvictionTimeThresholdMillis=1000 + +# Configure the threshold (in number of entries) from where a cursor should be considered 'backlogged' +# and thus should be set as inactive. +managedLedgerCursorBackloggedThreshold=1000 + +# Rate limit the amount of writes generated by consumer acking the messages +managedLedgerDefaultMarkDeleteRateLimit=0.1 + +# Max number of entries to append to a ledger before triggering a rollover +# A ledger rollover is triggered on these conditions +# * Either the max rollover time has been reached +# * or max entries have been written to the ledged and at least min-time +# has passed +managedLedgerMaxEntriesPerLedger=50000 + +# Minimum time between ledger rollover for a topic +managedLedgerMinLedgerRolloverTimeMinutes=10 + +# Maximum time before forcing a ledger rollover for a topic +managedLedgerMaxLedgerRolloverTimeMinutes=240 + +# Max number of entries to append to a cursor ledger +managedLedgerCursorMaxEntriesPerLedger=50000 + +# Max time before triggering a rollover on a cursor ledger +managedLedgerCursorRolloverTimeInSeconds=14400 + +# Maximum ledger size before triggering a rollover for a topic (MB) +managedLedgerMaxSizePerLedgerMbytes=2048 + +# Max number of "acknowledgment holes" that are going to be persistently stored. +# When acknowledging out of order, a consumer will leave holes that are supposed +# to be quickly filled by acking all the messages. The information of which +# messages are acknowledged is persisted by compressing in "ranges" of messages +# that were acknowledged. After the max number of ranges is reached, the information +# will only be tracked in memory and messages will be redelivered in case of +# crashes. +managedLedgerMaxUnackedRangesToPersist=10000 + +# Max number of "acknowledgment holes" that can be stored in Zookeeper. If number of unack message range is higher +# than this limit then broker will persist unacked ranges into bookkeeper to avoid additional data overhead into +# zookeeper. +managedLedgerMaxUnackedRangesToPersistInZooKeeper=1000 + +# Skip reading non-recoverable/unreadable data-ledger under managed-ledger's list. It helps when data-ledgers gets +# corrupted at bookkeeper and managed-cursor is stuck at that ledger. +autoSkipNonRecoverableData=false + +# operation timeout while updating managed-ledger metadata. +managedLedgerMetadataOperationsTimeoutSeconds=60 + +# Read entries timeout when broker tries to read messages from bookkeeper. +managedLedgerReadEntryTimeoutSeconds=0 + +# Add entry timeout when broker tries to publish message to bookkeeper (0 to disable it). +managedLedgerAddEntryTimeoutSeconds=0 + +# New entries check delay for the cursor under the managed ledger. +# If no new messages in the topic, the cursor will try to check again after the delay time. +# For consumption latency sensitive scenario, can set to a smaller value or set to 0. +# Of course, use a smaller value may degrade consumption throughput. Default is 10ms. +managedLedgerNewEntriesCheckDelayInMillis=10 + +# Use Open Range-Set to cache unacked messages +managedLedgerUnackedRangesOpenCacheSetEnabled=true + +# Managed ledger prometheus stats latency rollover seconds (default: 60s) +managedLedgerPrometheusStatsLatencyRolloverSeconds=60 + +# Whether trace managed ledger task execution time +managedLedgerTraceTaskExecution=true + +### --- Load balancer --- ### + +loadManagerClassName=org.apache.pulsar.broker.loadbalance.NoopLoadManager + +# Enable load balancer +loadBalancerEnabled=false + +# Percentage of change to trigger load report update +loadBalancerReportUpdateThresholdPercentage=10 + +# maximum interval to update load report +loadBalancerReportUpdateMaxIntervalMinutes=15 + +# Frequency of report to collect +loadBalancerHostUsageCheckIntervalMinutes=1 + +# Load shedding interval. Broker periodically checks whether some traffic should be offload from +# some over-loaded broker to other under-loaded brokers +loadBalancerSheddingIntervalMinutes=1 + +# Prevent the same topics to be shed and moved to other broker more that once within this timeframe +loadBalancerSheddingGracePeriodMinutes=30 + +# Usage threshold to allocate max number of topics to broker +loadBalancerBrokerMaxTopics=50000 + +# Interval to flush dynamic resource quota to ZooKeeper +loadBalancerResourceQuotaUpdateIntervalMinutes=15 + +# enable/disable namespace bundle auto split +loadBalancerAutoBundleSplitEnabled=true + +# enable/disable automatic unloading of split bundles +loadBalancerAutoUnloadSplitBundlesEnabled=true + +# maximum topics in a bundle, otherwise bundle split will be triggered +loadBalancerNamespaceBundleMaxTopics=1000 + +# maximum sessions (producers + consumers) in a bundle, otherwise bundle split will be triggered +loadBalancerNamespaceBundleMaxSessions=1000 + +# maximum msgRate (in + out) in a bundle, otherwise bundle split will be triggered +loadBalancerNamespaceBundleMaxMsgRate=30000 + +# maximum bandwidth (in + out) in a bundle, otherwise bundle split will be triggered +loadBalancerNamespaceBundleMaxBandwidthMbytes=100 + +# maximum number of bundles in a namespace +loadBalancerNamespaceMaximumBundles=128 + +# The broker resource usage threshold. +# When the broker resource usage is gratter than the pulsar cluster average resource usge, +# the threshold shedder will be triggered to offload bundles from the broker. +# It only take effect in ThresholdSheddler strategy. +loadBalancerBrokerThresholdShedderPercentage=10 + +# When calculating new resource usage, the history usage accounts for. +# It only take effect in ThresholdSheddler strategy. +loadBalancerHistoryResourcePercentage=0.9 + +# The BandWithIn usage weight when calculating new resourde usage. +# It only take effect in ThresholdShedder strategy. +loadBalancerBandwithInResourceWeight=1.0 + +# The BandWithOut usage weight when calculating new resourde usage. +# It only take effect in ThresholdShedder strategy. +loadBalancerBandwithOutResourceWeight=1.0 + +# The CPU usage weight when calculating new resourde usage. +# It only take effect in ThresholdShedder strategy. +loadBalancerCPUResourceWeight=1.0 + +# The heap memory usage weight when calculating new resourde usage. +# It only take effect in ThresholdShedder strategy. +loadBalancerMemoryResourceWeight=1.0 + +# The direct memory usage weight when calculating new resourde usage. +# It only take effect in ThresholdShedder strategy. +loadBalancerDirectMemoryResourceWeight=1.0 + +# Bundle unload minimum throughput threshold (MB), avoding bundle unload frequently. +# It only take effect in ThresholdShedder strategy. +loadBalancerBundleUnloadMinThroughputThreshold=10 + +### --- Replication --- ### + +# Enable replication metrics +replicationMetricsEnabled=true + +# Max number of connections to open for each broker in a remote cluster +# More connections host-to-host lead to better throughput over high-latency +# links. +replicationConnectionsPerBroker=16 + +# Replicator producer queue size +replicationProducerQueueSize=1000 + +# Duration to check replication policy to avoid replicator inconsistency +# due to missing ZooKeeper watch (disable with value 0) +replicationPolicyCheckDurationSeconds=600 + +# Default message retention time +defaultRetentionTimeInMinutes=0 + +# Default retention size +defaultRetentionSizeInMB=0 + +# How often to check whether the connections are still alive +keepAliveIntervalSeconds=30 + +### --- WebSocket --- ### + +# Enable the WebSocket API service in broker +webSocketServiceEnabled=true + +# Number of IO threads in Pulsar Client used in WebSocket proxy +webSocketNumIoThreads=8 + +# Number of connections per Broker in Pulsar Client used in WebSocket proxy +webSocketConnectionsPerBroker=8 + +# Time in milliseconds that idle WebSocket session times out +webSocketSessionIdleTimeoutMillis=300000 + +# The maximum size of a text message during parsing in WebSocket proxy +webSocketMaxTextFrameSize=1048576 + +### --- Metrics --- ### + +# Enable topic level metrics +exposeTopicLevelMetricsInPrometheus=true + +# Classname of Pluggable JVM GC metrics logger that can log GC specific metrics +# jvmGCMetricsLoggerClassName= + +### --- Broker Web Stats --- ### + +# Enable topic level metrics +exposePublisherStats=true + +# Enable expose the precise backlog stats. +# Set false to use published counter and consumed counter to calculate, this would be more efficient but may be inaccurate. +# Default is false. +exposePreciseBacklogInPrometheus=false + +### --- Deprecated config variables --- ### + +# Deprecated. Use configurationStoreServers +globalZookeeperServers= + +# Deprecated. Use brokerDeleteInactiveTopicsFrequencySeconds +brokerServicePurgeInactiveFrequencyInSeconds=60 + +### --- BookKeeper Configuration --- ##### + +ledgerStorageClass=org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage + +# Size of Write Cache. Memory is allocated from JVM direct memory. +# Write cache is used to buffer entries before flushing into the entry log +# For good performance, it should be big enough to hold a substantial amount +# of entries in the flush interval +# By default it will be allocated to 1/4th of the available direct memory +dbStorage_writeCacheMaxSizeMb= + +# Size of Read cache. Memory is allocated from JVM direct memory. +# This read cache is pre-filled doing read-ahead whenever a cache miss happens +# By default it will be allocated to 1/4th of the available direct memory +dbStorage_readAheadCacheMaxSizeMb= + +# How many entries to pre-fill in cache after a read cache miss +dbStorage_readAheadCacheBatchSize=1000 + +flushInterval=60000 + +## RocksDB specific configurations +## DbLedgerStorage uses RocksDB to store the indexes from +## (ledgerId, entryId) -> (entryLog, offset) + +# Size of RocksDB block-cache. For best performance, this cache +# should be big enough to hold a significant portion of the index +# database which can reach ~2GB in some cases +# Default is to use 10% of the direct memory size +dbStorage_rocksDB_blockCacheSize= + +# Other RocksDB specific tunables +dbStorage_rocksDB_writeBufferSizeMB=4 +dbStorage_rocksDB_sstSizeInMB=4 +dbStorage_rocksDB_blockSize=4096 +dbStorage_rocksDB_bloomFilterBitsPerKey=10 +dbStorage_rocksDB_numLevels=-1 +dbStorage_rocksDB_numFilesInLevel0=4 +dbStorage_rocksDB_maxSizeInLevel1MB=256 + +# Maximum latency to impose on a journal write to achieve grouping +journalMaxGroupWaitMSec=1 + +# Should the data be fsynced on journal before acknowledgment. +journalSyncData=false + + +# For each ledger dir, maximum disk space which can be used. +# Default is 0.95f. i.e. 95% of disk can be used at most after which nothing will +# be written to that partition. If all ledger dir partions are full, then bookie +# will turn to readonly mode if 'readOnlyModeEnabled=true' is set, else it will +# shutdown. +# Valid values should be in between 0 and 1 (exclusive). +diskUsageThreshold=0.99 + +# The disk free space low water mark threshold. +# Disk is considered full when usage threshold is exceeded. +# Disk returns back to non-full state when usage is below low water mark threshold. +# This prevents it from going back and forth between these states frequently +# when concurrent writes and compaction are happening. This also prevent bookie from +# switching frequently between read-only and read-writes states in the same cases. +diskUsageWarnThreshold=0.99 + +# Whether the bookie allowed to use a loopback interface as its primary +# interface(i.e. the interface it uses to establish its identity)? +# By default, loopback interfaces are not allowed as the primary +# interface. +# Using a loopback interface as the primary interface usually indicates +# a configuration error. For example, its fairly common in some VPS setups +# to not configure a hostname, or to have the hostname resolve to +# 127.0.0.1. If this is the case, then all bookies in the cluster will +# establish their identities as 127.0.0.1:3181, and only one will be able +# to join the cluster. For VPSs configured like this, you should explicitly +# set the listening interface. +allowLoopback=true + +# How long the interval to trigger next garbage collection, in milliseconds +# Since garbage collection is running in background, too frequent gc +# will heart performance. It is better to give a higher number of gc +# interval if there is enough disk capacity. +gcWaitTime=300000 + +# Enable topic auto creation if new producer or consumer connected (disable auto creation with value false) +allowAutoTopicCreation=true + +# The type of topic that is allowed to be automatically created.(partitioned/non-partitioned) +allowAutoTopicCreationType=non-partitioned + +# Enable subscription auto creation if new consumer connected (disable auto creation with value false) +allowAutoSubscriptionCreation=true + +# The number of partitioned topics that is allowed to be automatically created if allowAutoTopicCreationType is partitioned. +defaultNumPartitions=1 + +### --- Transaction config variables --- ### +transactionMetadataStoreProviderClassName=org.apache.pulsar.transaction.coordinator.impl.InMemTransactionMetadataStoreProvider diff --git a/bitsail-connectors/connector-pulsar/src/test/resources/pulsar/txnStandalone.conf b/bitsail-connectors/connector-pulsar/src/test/resources/pulsar/txnStandalone.conf new file mode 100644 index 000000000..2c541ad25 --- /dev/null +++ b/bitsail-connectors/connector-pulsar/src/test/resources/pulsar/txnStandalone.conf @@ -0,0 +1,980 @@ +# +# Licensed 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. +# + +### --- General broker settings --- ### + +# Zookeeper quorum connection string +zookeeperServers= + +# Configuration Store connection string +configurationStoreServers= + +brokerServicePort=6650 + +# Port to use to server HTTP request +webServicePort=8080 + +# Hostname or IP address the service binds on, default is 0.0.0.0. +bindAddress=0.0.0.0 + +# Hostname or IP address the service advertises to the outside world. If not set, the value of InetAddress.getLocalHost().getHostName() is used. +advertisedAddress= + +# Enable or disable the HAProxy protocol. +haProxyProtocolEnabled=false + +# Number of threads to use for Netty IO. Default is set to 2 * Runtime.getRuntime().availableProcessors() +numIOThreads= + +# Number of threads to use for ordered executor. The ordered executor is used to operate with zookeeper, +# such as init zookeeper client, get namespace policies from zookeeper etc. It also used to split bundle. Default is 8 +numOrderedExecutorThreads=8 + +# Number of threads to use for HTTP requests processing. Default is set to 2 * Runtime.getRuntime().availableProcessors() +numHttpServerThreads= + +# Number of thread pool size to use for pulsar broker service. +# The executor in thread pool will do basic broker operation like load/unload bundle, update managedLedgerConfig, +# update topic/subscription/replicator message dispatch rate, do leader election etc. +# Default is Runtime.getRuntime().availableProcessors() +numExecutorThreadPoolSize= + +# Number of thread pool size to use for pulsar zookeeper callback service +# The cache executor thread pool is used for restarting global zookeeper session. +# Default is 10 +numCacheExecutorThreadPoolSize=10 + +# Max concurrent web requests +maxConcurrentHttpRequests=1024 + +# Name of the cluster to which this broker belongs to +clusterName=standalone + +# Enable cluster's failure-domain which can distribute brokers into logical region +failureDomainsEnabled=false + +# Zookeeper session timeout in milliseconds +zooKeeperSessionTimeoutMillis=30000 + +# ZooKeeper operation timeout in seconds +zooKeeperOperationTimeoutSeconds=30 + +# ZooKeeper cache expiry time in seconds +zooKeeperCacheExpirySeconds=300 + +# Time to wait for broker graceful shutdown. After this time elapses, the process will be killed +brokerShutdownTimeoutMs=60000 + +# Flag to skip broker shutdown when broker handles Out of memory error +skipBrokerShutdownOnOOM=false + +# Enable backlog quota check. Enforces action on topic when the quota is reached +backlogQuotaCheckEnabled=true + +# How often to check for topics that have reached the quota +backlogQuotaCheckIntervalInSeconds=60 + +# Default per-topic backlog quota limit +backlogQuotaDefaultLimitGB=10 + +# Default per-topic backlog quota time limit in second, less than 0 means no limitation. default is -1. +backlogQuotaDefaultLimitSecond=-1 + +# Default ttl for namespaces if ttl is not already configured at namespace policies. (disable default-ttl with value 0) +ttlDurationDefaultInSeconds=0 + +# Enable the deletion of inactive topics +brokerDeleteInactiveTopicsEnabled=true + +# How often to check for inactive topics +brokerDeleteInactiveTopicsFrequencySeconds=60 + +# Max pending publish requests per connection to avoid keeping large number of pending +# requests in memory. Default: 1000 +maxPendingPublishRequestsPerConnection=1000 + +# How frequently to proactively check and purge expired messages +messageExpiryCheckIntervalInMinutes=5 + +# How long to delay rewinding cursor and dispatching messages when active consumer is changed +activeConsumerFailoverDelayTimeMillis=1000 + +# How long to delete inactive subscriptions from last consuming +# When it is 0, inactive subscriptions are not deleted automatically +subscriptionExpirationTimeMinutes=0 + +# Enable subscription message redelivery tracker to send redelivery count to consumer (default is enabled) +subscriptionRedeliveryTrackerEnabled=true + +# On KeyShared subscriptions, with default AUTO_SPLIT mode, use splitting ranges or +# consistent hashing to reassign keys to new consumers +subscriptionKeySharedUseConsistentHashing=false + +# On KeyShared subscriptions, number of points in the consistent-hashing ring. +# The higher the number, the more equal the assignment of keys to consumers +subscriptionKeySharedConsistentHashingReplicaPoints=100 + +# How frequently to proactively check and purge expired subscription +subscriptionExpiryCheckIntervalInMinutes=5 + +# Set the default behavior for message deduplication in the broker +# This can be overridden per-namespace. If enabled, broker will reject +# messages that were already stored in the topic +brokerDeduplicationEnabled=false + +# Maximum number of producer information that it's going to be +# persisted for deduplication purposes +brokerDeduplicationMaxNumberOfProducers=10000 + +# Number of entries after which a dedup info snapshot is taken. +# A bigger interval will lead to less snapshots being taken though it would +# increase the topic recovery time, when the entries published after the +# snapshot need to be replayed +brokerDeduplicationEntriesInterval=1000 + +# Time of inactivity after which the broker will discard the deduplication information +# relative to a disconnected producer. Default is 6 hours. +brokerDeduplicationProducerInactivityTimeoutMinutes=360 + +# When a namespace is created without specifying the number of bundle, this +# value will be used as the default +defaultNumberOfNamespaceBundles=4 + +# Max number of topics allowed to be created in the namespace. When the topics reach the max topics of the namespace, +# the broker should reject the new topic request(include topic auto-created by the producer or consumer) +# until the number of connected consumers decrease. +# Using a value of 0, is disabling maxTopicsPerNamespace-limit check. +maxTopicsPerNamespace=0 + +# Enable check for minimum allowed client library version +clientLibraryVersionCheckEnabled=false + +# Path for the file used to determine the rotation status for the broker when responding +# to service discovery health checks +statusFilePath=/usr/local/apache/htdocs + +# Max number of unacknowledged messages allowed to receive messages by a consumer on a shared subscription. Broker will stop sending +# messages to consumer once, this limit reaches until consumer starts acknowledging messages back +# Using a value of 0, is disabling unackeMessage limit check and consumer can receive messages without any restriction +maxUnackedMessagesPerConsumer=50000 + +# Max number of unacknowledged messages allowed per shared subscription. Broker will stop dispatching messages to +# all consumers of the subscription once this limit reaches until consumer starts acknowledging messages back and +# unack count reaches to limit/2. Using a value of 0, is disabling unackedMessage-limit +# check and dispatcher can dispatch messages without any restriction +maxUnackedMessagesPerSubscription=200000 + +# Max number of unacknowledged messages allowed per broker. Once this limit reaches, broker will stop dispatching +# messages to all shared subscription which has higher number of unack messages until subscriptions start +# acknowledging messages back and unack count reaches to limit/2. Using a value of 0, is disabling +# unackedMessage-limit check and broker doesn't block dispatchers +maxUnackedMessagesPerBroker=0 + +# Once broker reaches maxUnackedMessagesPerBroker limit, it blocks subscriptions which has higher unacked messages +# than this percentage limit and subscription will not receive any new messages until that subscription acks back +# limit/2 messages +maxUnackedMessagesPerSubscriptionOnBrokerBlocked=0.16 + +# Tick time to schedule task that checks topic publish rate limiting across all topics +# Reducing to lower value can give more accuracy while throttling publish but +# it uses more CPU to perform frequent check. (Disable publish throttling with value 0) +topicPublisherThrottlingTickTimeMillis=2 + +# Enable precise rate limit for topic publish +preciseTopicPublishRateLimiterEnable=false + +# Tick time to schedule task that checks broker publish rate limiting across all topics +# Reducing to lower value can give more accuracy while throttling publish but +# it uses more CPU to perform frequent check. (Disable publish throttling with value 0) +brokerPublisherThrottlingTickTimeMillis=50 + +# Max Rate(in 1 seconds) of Message allowed to publish for a broker if broker publish rate limiting enabled +# (Disable message rate limit with value 0) +brokerPublisherThrottlingMaxMessageRate=0 + +# Max Rate(in 1 seconds) of Byte allowed to publish for a broker if broker publish rate limiting enabled +# (Disable byte rate limit with value 0) +brokerPublisherThrottlingMaxByteRate=0 + +# Default messages per second dispatch throttling-limit for every topic. Using a value of 0, is disabling default +# message dispatch-throttling +dispatchThrottlingRatePerTopicInMsg=0 + +# Default bytes per second dispatch throttling-limit for every topic. Using a value of 0, is disabling +# 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: +# throttle-dispatch-rate = (publish-rate + configured dispatch-rate). +dispatchThrottlingRateRelativeToPublishRate=false + +# By default we enable dispatch-throttling for both caught up consumers as well as consumers who have +# backlog. +dispatchThrottlingOnNonBacklogConsumerEnabled=true + +# Precise dispathcer flow control according to history message number of each entry +preciseDispatcherFlowControl=false + +# Max number of concurrent lookup request broker allows to throttle heavy incoming lookup traffic +maxConcurrentLookupRequest=50000 + +# Max number of concurrent topic loading request broker allows to control number of zk-operations +maxConcurrentTopicLoadRequest=5000 + +# Max concurrent non-persistent message can be processed per connection +maxConcurrentNonPersistentMessagePerConnection=1000 + +# Number of worker threads to serve non-persistent topic +numWorkerThreadsForNonPersistentTopic=8 + +# Enable broker to load persistent topics +enablePersistentTopics=true + +# Enable broker to load non-persistent topics +enableNonPersistentTopics=true + +# Max number of producers allowed to connect to topic. Once this limit reaches, Broker will reject new producers +# until the number of connected producers decrease. +# Using a value of 0, is disabling maxProducersPerTopic-limit check. +maxProducersPerTopic=0 + +# Max number of producers with the same IP address allowed to connect to topic. +# Once this limit reaches, Broker will reject new producers until the number of +# connected producers with the same IP address decrease. +# Using a value of 0, is disabling maxSameAddressProducersPerTopic-limit check. +maxSameAddressProducersPerTopic=0 + +# Enforce producer to publish encrypted messages.(default disable). +encryptionRequireOnProducer=false + +# Max number of consumers allowed to connect to topic. Once this limit reaches, Broker will reject new consumers +# until the number of connected consumers decrease. +# Using a value of 0, is disabling maxConsumersPerTopic-limit check. +maxConsumersPerTopic=0 + +# Max number of consumers with the same IP address allowed to connect to topic. +# Once this limit reaches, Broker will reject new consumers until the number of +# connected consumers with the same IP address decrease. +# Using a value of 0, is disabling maxSameAddressConsumersPerTopic-limit check. +maxSameAddressConsumersPerTopic=0 + +# Max number of subscriptions allowed to subscribe to topic. Once this limit reaches, broker will reject +# new subscription until the number of subscribed subscriptions decrease. +# Using a value of 0, is disabling maxSubscriptionsPerTopic limit check. +maxSubscriptionsPerTopic=0 + +# Max number of consumers allowed to connect to subscription. Once this limit reaches, Broker will reject new consumers +# until the number of connected consumers decrease. +# Using a value of 0, is disabling maxConsumersPerSubscription-limit check. +maxConsumersPerSubscription=0 + +# Max number of partitions per partitioned topic +# Use 0 or negative number to disable the check +maxNumPartitionsPerPartitionedTopic=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.3, TLSv1.2] +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.3, TLSv1.2] +# used by the internal client to authenticate with Pulsar brokers +brokerClientTlsProtocols= + +# Enable or disable system topic +systemTopicEnabled=true + +# Enable or disable topic level policies, topic level policies depends on the system topic +# Please enable the system topic first. +topicLevelPoliciesEnabled=false + +# If a topic remains fenced for this number of seconds, it will be closed forcefully. +# If it is set to 0 or a negative number, the fenced topic will not be closed. +topicFencingTimeoutSeconds=0 + +### --- 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. +proxyRoles= + +# If this flag is set then the broker authenticates the original Auth data +# else it just accepts the originalPrincipal and authorizes it (if required). +authenticateOriginalAuthData=false + +# Enable authentication +authenticationEnabled=false + +# Authentication provider name list, which is comma separated list of class names +authenticationProviders= + +# Enforce authorization +authorizationEnabled=false + +# Authorization provider fully qualified class-name +authorizationProvider=org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider + +# Allow wildcard matching in authorization +# (wildcard matching only applicable if wildcard-char: +# * presents at first or last position eg: *.pulsar.service, pulsar.service.*) +authorizationAllowWildcardsMatching=false + +# Role names that are treated as "super-user", meaning they will be able to do all admin +# operations and publish/consume from all topics +superUserRoles= + +# Authentication settings of the broker itself. Used when the broker connects to other brokers, +# either in same or other clusters +brokerClientAuthenticationPlugin= +brokerClientAuthenticationParameters= + +# Supported Athenz provider domain names(comma separated) for authentication +athenzDomainNames= + +# When this parameter is not empty, unauthenticated users perform as anonymousUserRole +anonymousUserRole= + + +### --- Token Authentication Provider --- ### + +## Symmetric key +# Configure the secret key to be used to validate auth tokens +# The key can be specified like: +# tokenSecretKey=data:;base64,xxxxxxxxx +# tokenSecretKey=file:///my/secret.key ( Note: key file must be DER-encoded ) +tokenSecretKey= + +## Asymmetric public/private key pair +# Configure the public key to be used to validate auth tokens +# The key can be specified like: +# tokenPublicKey=data:;base64,xxxxxxxxx +# tokenPublicKey=file:///my/public.key ( Note: key file must be DER-encoded ) +tokenPublicKey= + + +# The token "claim" that will be interpreted as the authentication "role" or "principal" by AuthenticationProviderToken (defaults to "sub" if blank) +tokenAuthClaim= + +# The token audience "claim" name, e.g. "aud", that will be used to get the audience from token. +# If not set, audience will not be verified. +tokenAudienceClaim= + +# The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this. +tokenAudience= + +### --- BookKeeper Client --- ### + +# Authentication plugin to use when connecting to bookies +bookkeeperClientAuthenticationPlugin= + +# BookKeeper auth plugin implementatation specifics parameters name and values +bookkeeperClientAuthenticationParametersName= +bookkeeperClientAuthenticationParameters= + +# Timeout for BK add / read operations +bookkeeperClientTimeoutInSeconds=30 + +# Number of BookKeeper client worker threads +# Default is Runtime.getRuntime().availableProcessors() +bookkeeperClientNumWorkerThreads= + +# Speculative reads are initiated if a read request doesn't complete within a certain time +# Using a value of 0, is disabling the speculative reads +bookkeeperClientSpeculativeReadTimeoutInMillis=0 + +# Number of channels per bookie +bookkeeperNumberOfChannelsPerBookie=16 + +# Enable bookies health check. Bookies that have more than the configured number of failure within +# the interval will be quarantined for some time. During this period, new ledgers won't be created +# on these bookies +bookkeeperClientHealthCheckEnabled=true +bookkeeperClientHealthCheckIntervalSeconds=60 +bookkeeperClientHealthCheckErrorThresholdPerInterval=5 +bookkeeperClientHealthCheckQuarantineTimeInSeconds=1800 + +#bookie quarantine ratio to avoid all clients quarantine the high pressure bookie servers at the same time +bookkeeperClientQuarantineRatio=1.0 + +# Enable rack-aware bookie selection policy. BK will chose bookies from different racks when +# forming a new bookie ensemble +# This parameter related to ensemblePlacementPolicy in conf/bookkeeper.conf, if enabled, ensemblePlacementPolicy +# should be set to org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy +bookkeeperClientRackawarePolicyEnabled=true + +# Enable region-aware bookie selection policy. BK will chose bookies from +# different regions and racks when forming a new bookie ensemble. +# If enabled, the value of bookkeeperClientRackawarePolicyEnabled is ignored +# This parameter related to ensemblePlacementPolicy in conf/bookkeeper.conf, if enabled, ensemblePlacementPolicy +# should be set to org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy +bookkeeperClientRegionawarePolicyEnabled=false + +# Minimum number of racks per write quorum. BK rack-aware bookie selection policy will try to +# get bookies from at least 'bookkeeperClientMinNumRacksPerWriteQuorum' racks for a write quorum. +bookkeeperClientMinNumRacksPerWriteQuorum=1 + +# Enforces rack-aware bookie selection policy to pick bookies from 'bookkeeperClientMinNumRacksPerWriteQuorum' +# racks for a writeQuorum. +# If BK can't find bookie then it would throw BKNotEnoughBookiesException instead of picking random one. +bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false + +# Enable/disable reordering read sequence on reading entries. +bookkeeperClientReorderReadSequenceEnabled=false + +# Enable bookie isolation by specifying a list of bookie groups to choose from. Any bookie +# outside the specified groups will not be used by the broker +bookkeeperClientIsolationGroups= + +# Enable bookie secondary-isolation group if bookkeeperClientIsolationGroups doesn't +# have enough bookie available. +bookkeeperClientSecondaryIsolationGroups= + +# Minimum bookies that should be available as part of bookkeeperClientIsolationGroups +# else broker will include bookkeeperClientSecondaryIsolationGroups bookies in isolated list. +bookkeeperClientMinAvailableBookiesInIsolationGroups= + +# Set the client security provider factory class name. +# Default: org.apache.bookkeeper.tls.TLSContextFactory +bookkeeperTLSProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory + +# Enable tls authentication with bookie +bookkeeperTLSClientAuthentication=false + +# Supported type: PEM, JKS, PKCS12. Default value: PEM +bookkeeperTLSKeyFileType=PEM + +#Supported type: PEM, JKS, PKCS12. Default value: PEM +bookkeeperTLSTrustCertTypes=PEM + +# Path to file containing keystore password, if the client keystore is password protected. +bookkeeperTLSKeyStorePasswordPath= + +# Path to file containing truststore password, if the client truststore is password protected. +bookkeeperTLSTrustStorePasswordPath= + +# Path for the TLS private key file +bookkeeperTLSKeyFilePath= + +# Path for the TLS certificate file +bookkeeperTLSCertificateFilePath= + +# Path for the trusted TLS certificate file +bookkeeperTLSTrustCertsFilePath= + +# Enable/disable disk weight based placement. Default is false +bookkeeperDiskWeightBasedPlacementEnabled=false + +# Set the interval to check the need for sending an explicit LAC +# A value of '0' disables sending any explicit LACs. Default is 0. +bookkeeperExplicitLacIntervalInMills=0 + +# Use older Bookkeeper wire protocol with bookie +bookkeeperUseV2WireProtocol=true + +# Expose bookkeeper client managed ledger stats to prometheus. default is false +# bookkeeperClientExposeStatsToPrometheus=false + +### --- Managed Ledger --- ### + +# Number of bookies to use when creating a ledger +managedLedgerDefaultEnsembleSize=1 + +# Number of copies to store for each message +managedLedgerDefaultWriteQuorum=1 + +# Number of guaranteed copies (acks to wait before write is complete) +managedLedgerDefaultAckQuorum=1 + +# How frequently to flush the cursor positions that were accumulated due to rate limiting. (seconds). +# Default is 60 seconds +managedLedgerCursorPositionFlushSeconds = 60 + +# Default type of checksum to use when writing to BookKeeper. Default is "CRC32C" +# Other possible options are "CRC32", "MAC" or "DUMMY" (no checksum). +managedLedgerDigestType=CRC32C + +# Number of threads to be used for managed ledger tasks dispatching +managedLedgerNumWorkerThreads=4 + +# Number of threads to be used for managed ledger scheduled tasks +managedLedgerNumSchedulerThreads=4 + +# Amount of memory to use for caching data payload in managed ledger. This memory +# is allocated from JVM direct memory and it's shared across all the topics +# running in the same broker. By default, uses 1/5th of available direct memory +managedLedgerCacheSizeMB= + +# Whether we should make a copy of the entry payloads when inserting in cache +managedLedgerCacheCopyEntries=false + +# Threshold to which bring down the cache level when eviction is triggered +managedLedgerCacheEvictionWatermark=0.9 + +# Configure the cache eviction frequency for the managed ledger cache (evictions/sec) +managedLedgerCacheEvictionFrequency=100.0 + +# All entries that have stayed in cache for more than the configured time, will be evicted +managedLedgerCacheEvictionTimeThresholdMillis=1000 + +# Configure the threshold (in number of entries) from where a cursor should be considered 'backlogged' +# and thus should be set as inactive. +managedLedgerCursorBackloggedThreshold=1000 + +# Rate limit the amount of writes generated by consumer acking the messages +managedLedgerDefaultMarkDeleteRateLimit=0.1 + +# Max number of entries to append to a ledger before triggering a rollover +# A ledger rollover is triggered on these conditions +# * Either the max rollover time has been reached +# * or max entries have been written to the ledger and at least min-time +# has passed +managedLedgerMaxEntriesPerLedger=50000 + +# Minimum time between ledger rollover for a topic +managedLedgerMinLedgerRolloverTimeMinutes=10 + +# Maximum time before forcing a ledger rollover for a topic +managedLedgerMaxLedgerRolloverTimeMinutes=240 + +# Max number of entries to append to a cursor ledger +managedLedgerCursorMaxEntriesPerLedger=50000 + +# Max time before triggering a rollover on a cursor ledger +managedLedgerCursorRolloverTimeInSeconds=14400 + +# Maximum ledger size before triggering a rollover for a topic (MB) +managedLedgerMaxSizePerLedgerMbytes=2048 + +# Max number of "acknowledgment holes" that are going to be persistently stored. +# When acknowledging out of order, a consumer will leave holes that are supposed +# to be quickly filled by acking all the messages. The information of which +# messages are acknowledged is persisted by compressing in "ranges" of messages +# that were acknowledged. After the max number of ranges is reached, the information +# will only be tracked in memory and messages will be redelivered in case of +# crashes. +managedLedgerMaxUnackedRangesToPersist=10000 + +# Max number of "acknowledgment holes" that can be stored in Zookeeper. If number of unack message range is higher +# than this limit then broker will persist unacked ranges into bookkeeper to avoid additional data overhead into +# zookeeper. +managedLedgerMaxUnackedRangesToPersistInZooKeeper=1000 + +# Skip reading non-recoverable/unreadable data-ledger under managed-ledger's list. It helps when data-ledgers gets +# corrupted at bookkeeper and managed-cursor is stuck at that ledger. +autoSkipNonRecoverableData=false + +# operation timeout while updating managed-ledger metadata. +managedLedgerMetadataOperationsTimeoutSeconds=60 + +# Read entries timeout when broker tries to read messages from bookkeeper. +managedLedgerReadEntryTimeoutSeconds=0 + +# Add entry timeout when broker tries to publish message to bookkeeper (0 to disable it). +managedLedgerAddEntryTimeoutSeconds=0 + +# New entries check delay for the cursor under the managed ledger. +# If no new messages in the topic, the cursor will try to check again after the delay time. +# For consumption latency sensitive scenario, can set to a smaller value or set to 0. +# Of course, use a smaller value may degrade consumption throughput. Default is 10ms. +managedLedgerNewEntriesCheckDelayInMillis=10 + +# Use Open Range-Set to cache unacked messages +managedLedgerUnackedRangesOpenCacheSetEnabled=true + +# Managed ledger prometheus stats latency rollover seconds (default: 60s) +managedLedgerPrometheusStatsLatencyRolloverSeconds=60 + +# Whether trace managed ledger task execution time +managedLedgerTraceTaskExecution=true + +# If you want to custom bookie ID or use a dynamic network address for the bookie, +# you can set this option. +# Bookie advertises itself using bookieId rather than +# BookieSocketAddress (hostname:port or IP:port). +# bookieId is a non empty string that can contain ASCII digits and letters ([a-zA-Z9-0]), +# colons, dashes, and dots. +# For more information about bookieId, see http://bookkeeper.apache.org/bps/BP-41-bookieid/. +# bookieId= + +### --- Load balancer --- ### + +loadManagerClassName=org.apache.pulsar.broker.loadbalance.NoopLoadManager + +# Enable load balancer +loadBalancerEnabled=false + +# Percentage of change to trigger load report update +loadBalancerReportUpdateThresholdPercentage=10 + +# maximum interval to update load report +loadBalancerReportUpdateMaxIntervalMinutes=15 + +# Frequency of report to collect +loadBalancerHostUsageCheckIntervalMinutes=1 + +# Load shedding interval. Broker periodically checks whether some traffic should be offload from +# some over-loaded broker to other under-loaded brokers +loadBalancerSheddingIntervalMinutes=1 + +# Prevent the same topics to be shed and moved to other broker more that once within this timeframe +loadBalancerSheddingGracePeriodMinutes=30 + +# Usage threshold to allocate max number of topics to broker +loadBalancerBrokerMaxTopics=50000 + +# Interval to flush dynamic resource quota to ZooKeeper +loadBalancerResourceQuotaUpdateIntervalMinutes=15 + +# enable/disable namespace bundle auto split +loadBalancerAutoBundleSplitEnabled=true + +# enable/disable automatic unloading of split bundles +loadBalancerAutoUnloadSplitBundlesEnabled=true + +# maximum topics in a bundle, otherwise bundle split will be triggered +loadBalancerNamespaceBundleMaxTopics=1000 + +# maximum sessions (producers + consumers) in a bundle, otherwise bundle split will be triggered +loadBalancerNamespaceBundleMaxSessions=1000 + +# maximum msgRate (in + out) in a bundle, otherwise bundle split will be triggered +loadBalancerNamespaceBundleMaxMsgRate=30000 + +# maximum bandwidth (in + out) in a bundle, otherwise bundle split will be triggered +loadBalancerNamespaceBundleMaxBandwidthMbytes=100 + +# maximum number of bundles in a namespace +loadBalancerNamespaceMaximumBundles=128 + +# The broker resource usage threshold. +# When the broker resource usage is greater than the pulsar cluster average resource usage, +# the threshold shedder will be triggered to offload bundles from the broker. +# It only takes effect in the ThresholdShedder strategy. +loadBalancerBrokerThresholdShedderPercentage=10 + +# When calculating new resource usage, the history usage accounts for. +# It only takes effect in the ThresholdShedder strategy. +loadBalancerHistoryResourcePercentage=0.9 + +# The BandWithIn usage weight when calculating new resource usage. +# It only takes effect in the ThresholdShedder strategy. +loadBalancerBandwithInResourceWeight=1.0 + +# The BandWithOut usage weight when calculating new resource usage. +# It only takes effect in the ThresholdShedder strategy. +loadBalancerBandwithOutResourceWeight=1.0 + +# The CPU usage weight when calculating new resource usage. +# It only takes effect in the ThresholdShedder strategy. +loadBalancerCPUResourceWeight=1.0 + +# The heap memory usage weight when calculating new resource usage. +# It only takes effect in the ThresholdShedder strategy. +loadBalancerMemoryResourceWeight=1.0 + +# The direct memory usage weight when calculating new resource usage. +# It only takes effect in the ThresholdShedder strategy. +loadBalancerDirectMemoryResourceWeight=1.0 + +# Bundle unload minimum throughput threshold (MB), avoiding bundle unload frequently. +# It only takes effect in the ThresholdShedder strategy. +loadBalancerBundleUnloadMinThroughputThreshold=10 + +### --- Replication --- ### + +# Enable replication metrics +replicationMetricsEnabled=true + +# Max number of connections to open for each broker in a remote cluster +# More connections host-to-host lead to better throughput over high-latency +# links. +replicationConnectionsPerBroker=16 + +# Replicator producer queue size +replicationProducerQueueSize=1000 + +# Duration to check replication policy to avoid replicator inconsistency +# due to missing ZooKeeper watch (disable with value 0) +replicationPolicyCheckDurationSeconds=600 + +# Default message retention time +defaultRetentionTimeInMinutes=0 + +# Default retention size +defaultRetentionSizeInMB=0 + +# How often to check whether the connections are still alive +keepAliveIntervalSeconds=30 + +### --- WebSocket --- ### + +# Enable the WebSocket API service in broker +webSocketServiceEnabled=true + +# Number of IO threads in Pulsar Client used in WebSocket proxy +webSocketNumIoThreads=8 + +# Number of connections per Broker in Pulsar Client used in WebSocket proxy +webSocketConnectionsPerBroker=8 + +# Time in milliseconds that idle WebSocket session times out +webSocketSessionIdleTimeoutMillis=300000 + +# The maximum size of a text message during parsing in WebSocket proxy +webSocketMaxTextFrameSize=1048576 + +### --- Metrics --- ### + +# Enable topic level metrics +exposeTopicLevelMetricsInPrometheus=true + +# Time in milliseconds that metrics endpoint would time out. Default is 30s. +# Increase it if there are a lot of topics to expose topic-level metrics. +# Set it to 0 to disable timeout. +metricsServletTimeoutMs=30000 + +# Classname of Pluggable JVM GC metrics logger that can log GC specific metrics +# jvmGCMetricsLoggerClassName= + +### --- Broker Web Stats --- ### + +# Enable topic level metrics +exposePublisherStats=true + +# Enable expose the precise backlog stats. +# Set false to use published counter and consumed counter to calculate, this would be more efficient but may be inaccurate. +# Default is false. +exposePreciseBacklogInPrometheus=false + +### --- Deprecated config variables --- ### + +# Deprecated. Use configurationStoreServers +globalZookeeperServers= + +# Deprecated. Use brokerDeleteInactiveTopicsFrequencySeconds +brokerServicePurgeInactiveFrequencyInSeconds=60 + +### --- BookKeeper Configuration --- ##### + +ledgerStorageClass=org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage + +# The maximum netty frame size in bytes. Any message received larger than this will be rejected. The default value is 5MB. +nettyMaxFrameSizeBytes=5253120 + +# Size of Write Cache. Memory is allocated from JVM direct memory. +# Write cache is used to buffer entries before flushing into the entry log +# For good performance, it should be big enough to hold a substantial amount +# of entries in the flush interval +# By default it will be allocated to 1/4th of the available direct memory +dbStorage_writeCacheMaxSizeMb= + +# Size of Read cache. Memory is allocated from JVM direct memory. +# This read cache is pre-filled doing read-ahead whenever a cache miss happens +# By default it will be allocated to 1/4th of the available direct memory +dbStorage_readAheadCacheMaxSizeMb= + +# How many entries to pre-fill in cache after a read cache miss +dbStorage_readAheadCacheBatchSize=1000 + +flushInterval=60000 + +## RocksDB specific configurations +## DbLedgerStorage uses RocksDB to store the indexes from +## (ledgerId, entryId) -> (entryLog, offset) + +# Size of RocksDB block-cache. For best performance, this cache +# should be big enough to hold a significant portion of the index +# database which can reach ~2GB in some cases +# Default is to use 10% of the direct memory size +dbStorage_rocksDB_blockCacheSize= + +# Other RocksDB specific tunables +dbStorage_rocksDB_writeBufferSizeMB=4 +dbStorage_rocksDB_sstSizeInMB=4 +dbStorage_rocksDB_blockSize=4096 +dbStorage_rocksDB_bloomFilterBitsPerKey=10 +dbStorage_rocksDB_numLevels=-1 +dbStorage_rocksDB_numFilesInLevel0=4 +dbStorage_rocksDB_maxSizeInLevel1MB=256 + +# Maximum latency to impose on a journal write to achieve grouping +journalMaxGroupWaitMSec=1 + +# Should the data be fsynced on journal before acknowledgment. +journalSyncData=false + + +# For each ledger dir, maximum disk space which can be used. +# Default is 0.95f. i.e. 95% of disk can be used at most after which nothing will +# be written to that partition. If all ledger dir partions are full, then bookie +# will turn to readonly mode if 'readOnlyModeEnabled=true' is set, else it will +# shutdown. +# Valid values should be in between 0 and 1 (exclusive). +diskUsageThreshold=0.99 + +# The disk free space low water mark threshold. +# Disk is considered full when usage threshold is exceeded. +# Disk returns back to non-full state when usage is below low water mark threshold. +# This prevents it from going back and forth between these states frequently +# when concurrent writes and compaction are happening. This also prevent bookie from +# switching frequently between read-only and read-writes states in the same cases. +diskUsageWarnThreshold=0.99 + +# Whether the bookie allowed to use a loopback interface as its primary +# interface(i.e. the interface it uses to establish its identity)? +# By default, loopback interfaces are not allowed as the primary +# interface. +# Using a loopback interface as the primary interface usually indicates +# a configuration error. For example, its fairly common in some VPS setups +# to not configure a hostname, or to have the hostname resolve to +# 127.0.0.1. If this is the case, then all bookies in the cluster will +# establish their identities as 127.0.0.1:3181, and only one will be able +# to join the cluster. For VPSs configured like this, you should explicitly +# set the listening interface. +allowLoopback=true + +# How long the interval to trigger next garbage collection, in milliseconds +# Since garbage collection is running in background, too frequent gc +# will heart performance. It is better to give a higher number of gc +# interval if there is enough disk capacity. +gcWaitTime=300000 + +# Enable topic auto creation if new producer or consumer connected (disable auto creation with value false) +allowAutoTopicCreation=true + +# The type of topic that is allowed to be automatically created.(partitioned/non-partitioned) +allowAutoTopicCreationType=non-partitioned + +# Enable subscription auto creation if new consumer connected (disable auto creation with value false) +allowAutoSubscriptionCreation=true + +# The number of partitioned topics that is allowed to be automatically created if allowAutoTopicCreationType is partitioned. +defaultNumPartitions=1 + +### --- Transaction config variables --- ### +# Enable transaction coordinator in broker +transactionCoordinatorEnabled=true +; transactionMetadataStoreProviderClassName=org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStoreProvider +transactionMetadataStoreProviderClassName=org.apache.pulsar.transaction.coordinator.impl.InMemTransactionMetadataStoreProvider + +# Transaction buffer take snapshot transaction count +transactionBufferSnapshotMaxTransactionCount=1000 + +# Transaction buffer take snapshot interval time +# Unit : millisecond +transactionBufferSnapshotMinTimeInMillis=5000 + +### --- Packages management service configuration variables (begin) --- ### + +# Enable the packages management service or not +enablePackagesManagement=false + +# The packages management service storage service provide +packagesManagementStorageProvider=org.apache.pulsar.packages.management.storage.bookkeeper.BookKeeperPackagesStorageProvider + +# When the packages storage provider is bookkeeper, you can use this configuration to +# control the number of replicas for storing the package +packagesReplicas=1 + +# The bookkeeper ledger root path +packagesManagementLedgerRootPath=/ledgers + +### --- Packages management service configuration variables (end) --- ### diff --git a/bitsail-connectors/pom.xml b/bitsail-connectors/pom.xml index 95b19312c..72492ed94 100644 --- a/bitsail-connectors/pom.xml +++ b/bitsail-connectors/pom.xml @@ -41,6 +41,7 @@ connector-rocketmq connector-redis connector-clickhouse + connector-pulsar diff --git a/bitsail-test/bitsail-connector-test/pom.xml b/bitsail-test/bitsail-connector-test/pom.xml index 8307767a2..0c81b247a 100644 --- a/bitsail-test/bitsail-connector-test/pom.xml +++ b/bitsail-test/bitsail-connector-test/pom.xml @@ -79,6 +79,12 @@ 1.17.5 + + org.testcontainers + pulsar + 1.17.5 + + org.elasticsearch.client elasticsearch-rest-client diff --git a/pom.xml b/pom.xml index 9af2449f1..18c539c7d 100644 --- a/pom.xml +++ b/pom.xml @@ -102,7 +102,7 @@ 4.9.2 1.1.2.6 1.8.3 - + 1.11.1.5 @@ -622,6 +622,23 @@ provided + + org.apache.flink + flink-connector-base + ${flink.version} + + + org.slf4j + slf4j-log4j12 + + + org.xerial.snappy + snappy-java + + + provided + + org.apache.flink flink-hadoop-compatibility_${scala.binary.version}