├── .asf.yaml ├── .dlc.json ├── .github ├── PULL_REQUEST_TEMPLATE.md ├── boring-cyborg.yml └── workflows │ ├── ci.yml │ └── daily.yml ├── .gitignore ├── .gitmodules ├── .idea └── vcs.xml ├── LICENSE ├── NOTICE ├── README.md ├── docs ├── content.zh │ └── docs │ │ └── connectors │ │ ├── datastream │ │ └── pulsar.md │ │ └── table │ │ └── pulsar.md ├── content │ └── docs │ │ └── connectors │ │ ├── datastream │ │ └── pulsar.md │ │ └── table │ │ └── pulsar.md ├── data │ └── pulsar.yml └── layouts │ └── shortcodes │ └── generated │ ├── pulsar_client_configuration.html │ ├── pulsar_consumer_configuration.html │ ├── pulsar_producer_configuration.html │ ├── pulsar_sink_configuration.html │ ├── pulsar_source_configuration.html │ └── pulsar_table_configuration.html ├── flink-connector-pulsar-e2e-tests ├── pom.xml └── src │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ └── tests │ │ └── util │ │ └── pulsar │ │ ├── PulsarSinkE2ECase.java │ │ ├── PulsarSourceE2ECase.java │ │ └── common │ │ ├── FlinkContainerUtils.java │ │ └── PulsarContainerTestEnvironment.java │ └── resources │ └── log4j2-test.properties ├── flink-connector-pulsar ├── archunit-violations │ ├── 185733ff-3e1c-48d3-a0c1-7652b72151f2 │ ├── 284bb951-3f59-4332-b61b-6f65b674f2ac │ ├── 5afc9aa4-9738-4cde-9966-1ca0c364183f │ ├── 660539fb-3e21-4ab9-a9c0-2a16a5d6472f │ ├── 89e608b0-1438-434c-a08e-80e6a4de4646 │ ├── b4233890-ecb8-4921-b0de-45b2f0092a1d │ ├── c10e9875-cfe1-4167-b920-9d84d8d91f18 │ ├── eec485f7-b61c-4644-bfa8-460b161d0a4a │ └── stored.rules ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── flink │ │ │ └── connector │ │ │ └── pulsar │ │ │ ├── common │ │ │ ├── config │ │ │ │ ├── PulsarClientFactory.java │ │ │ │ ├── PulsarConfigBuilder.java │ │ │ │ ├── PulsarConfigValidator.java │ │ │ │ ├── PulsarConfiguration.java │ │ │ │ └── PulsarOptions.java │ │ │ ├── crypto │ │ │ │ ├── DefaultPulsarCrypto.java │ │ │ │ ├── PulsarCrypto.java │ │ │ │ └── PulsarCryptoDisabled.java │ │ │ ├── metrics │ │ │ │ ├── MetricNames.java │ │ │ │ └── ProducerMetricsInterceptor.java │ │ │ ├── schema │ │ │ │ ├── BytesSchema.java │ │ │ │ ├── PulsarSchema.java │ │ │ │ ├── PulsarSchemaFactory.java │ │ │ │ ├── PulsarSchemaTypeInformation.java │ │ │ │ ├── PulsarSchemaTypeSerializer.java │ │ │ │ ├── PulsarSchemaUtils.java │ │ │ │ └── factories │ │ │ │ │ ├── AvroSchemaFactory.java │ │ │ │ │ ├── JSONSchemaFactory.java │ │ │ │ │ ├── KeyValueSchemaFactory.java │ │ │ │ │ ├── PrimitiveSchemaFactory.java │ │ │ │ │ ├── ProtobufNativeSchemaFactory.java │ │ │ │ │ ├── ProtobufSchemaFactory.java │ │ │ │ │ └── StringSchemaFactory.java │ │ │ └── utils │ │ │ │ ├── PulsarSerdeUtils.java │ │ │ │ └── PulsarTransactionUtils.java │ │ │ ├── sink │ │ │ ├── PulsarSink.java │ │ │ ├── PulsarSinkBuilder.java │ │ │ ├── PulsarSinkOptions.java │ │ │ ├── committer │ │ │ │ ├── PulsarCommittable.java │ │ │ │ ├── PulsarCommittableSerializer.java │ │ │ │ └── PulsarCommitter.java │ │ │ ├── config │ │ │ │ ├── PulsarSinkConfigUtils.java │ │ │ │ └── SinkConfiguration.java │ │ │ └── writer │ │ │ │ ├── PulsarWriter.java │ │ │ │ ├── context │ │ │ │ ├── PulsarSinkContext.java │ │ │ │ └── PulsarSinkContextImpl.java │ │ │ │ ├── delayer │ │ │ │ ├── FixedMessageDelayer.java │ │ │ │ └── MessageDelayer.java │ │ │ │ ├── message │ │ │ │ ├── PulsarMessage.java │ │ │ │ └── PulsarMessageBuilder.java │ │ │ │ ├── router │ │ │ │ ├── KeyHashTopicRouter.java │ │ │ │ ├── MessageKeyHash.java │ │ │ │ ├── RoundRobinTopicRouter.java │ │ │ │ ├── TopicRouter.java │ │ │ │ └── TopicRoutingMode.java │ │ │ │ ├── serializer │ │ │ │ ├── PulsarSchemaWrapper.java │ │ │ │ ├── PulsarSerializationSchema.java │ │ │ │ └── PulsarSerializationSchemaWrapper.java │ │ │ │ └── topic │ │ │ │ ├── MetadataListener.java │ │ │ │ └── ProducerRegister.java │ │ │ ├── source │ │ │ ├── PulsarSource.java │ │ │ ├── PulsarSourceBuilder.java │ │ │ ├── PulsarSourceOptions.java │ │ │ ├── config │ │ │ │ ├── CursorVerification.java │ │ │ │ ├── PulsarConsumerBuilder.java │ │ │ │ ├── PulsarSourceConfigUtils.java │ │ │ │ └── SourceConfiguration.java │ │ │ ├── enumerator │ │ │ │ ├── PulsarSourceEnumState.java │ │ │ │ ├── PulsarSourceEnumStateSerializer.java │ │ │ │ ├── PulsarSourceEnumerator.java │ │ │ │ ├── assigner │ │ │ │ │ ├── SplitAssigner.java │ │ │ │ │ └── SplitAssignerImpl.java │ │ │ │ ├── cursor │ │ │ │ │ ├── CursorPosition.java │ │ │ │ │ ├── StartCursor.java │ │ │ │ │ ├── StopCursor.java │ │ │ │ │ ├── start │ │ │ │ │ │ ├── MessageIdStartCursor.java │ │ │ │ │ │ └── TimestampStartCursor.java │ │ │ │ │ └── stop │ │ │ │ │ │ ├── EventTimestampStopCursor.java │ │ │ │ │ │ ├── LatestMessageStopCursor.java │ │ │ │ │ │ ├── MessageIdStopCursor.java │ │ │ │ │ │ ├── NeverStopCursor.java │ │ │ │ │ │ └── PublishTimestampStopCursor.java │ │ │ │ ├── subscriber │ │ │ │ │ ├── PulsarSubscriber.java │ │ │ │ │ └── impl │ │ │ │ │ │ ├── BasePulsarSubscriber.java │ │ │ │ │ │ ├── TopicListSubscriber.java │ │ │ │ │ │ └── TopicPatternSubscriber.java │ │ │ │ └── topic │ │ │ │ │ ├── TopicMetadata.java │ │ │ │ │ ├── TopicNameUtils.java │ │ │ │ │ ├── TopicPartition.java │ │ │ │ │ ├── TopicRange.java │ │ │ │ │ └── range │ │ │ │ │ ├── FixedKeysRangeGenerator.java │ │ │ │ │ ├── FullRangeGenerator.java │ │ │ │ │ ├── RangeGenerator.java │ │ │ │ │ └── TopicRangeUtils.java │ │ │ ├── reader │ │ │ │ ├── PulsarPartitionSplitReader.java │ │ │ │ ├── PulsarRecordEmitter.java │ │ │ │ ├── PulsarSourceFetcherManager.java │ │ │ │ ├── PulsarSourceReader.java │ │ │ │ └── deserializer │ │ │ │ │ ├── GenericRecordDeserializationSchema.java │ │ │ │ │ ├── GenericRecordDeserializer.java │ │ │ │ │ ├── PulsarDeserializationSchema.java │ │ │ │ │ ├── PulsarDeserializationSchemaInitializationContext.java │ │ │ │ │ ├── PulsarDeserializationSchemaWrapper.java │ │ │ │ │ ├── PulsarSchemaWrapper.java │ │ │ │ │ └── PulsarTypeInformationWrapper.java │ │ │ └── split │ │ │ │ ├── PulsarPartitionSplit.java │ │ │ │ ├── PulsarPartitionSplitSerializer.java │ │ │ │ └── PulsarPartitionSplitState.java │ │ │ └── table │ │ │ ├── PulsarTableFactory.java │ │ │ ├── PulsarTableOptionUtils.java │ │ │ ├── PulsarTableOptions.java │ │ │ ├── PulsarTableValidationUtils.java │ │ │ ├── UpsertPulsarTableFactory.java │ │ │ ├── sink │ │ │ ├── PulsarTableSerializationSchema.java │ │ │ ├── PulsarTableSerializationSchemaFactory.java │ │ │ ├── PulsarTableSink.java │ │ │ └── PulsarWritableMetadata.java │ │ │ └── source │ │ │ ├── PulsarReadableMetadata.java │ │ │ ├── PulsarRowDataConverter.java │ │ │ ├── PulsarTableDeserializationSchema.java │ │ │ ├── PulsarTableDeserializationSchemaFactory.java │ │ │ └── PulsarTableSource.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.flink.table.factories.Factory │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ ├── architecture │ │ ├── ProductionCodeArchitectureTest.java │ │ └── TestCodeArchitectureTest.java │ │ └── connector │ │ └── pulsar │ │ ├── common │ │ ├── MiniClusterTestEnvironment.java │ │ ├── config │ │ │ ├── PulsarConfigBuilderTest.java │ │ │ ├── PulsarConfigValidatorTest.java │ │ │ └── PulsarConfigurationTest.java │ │ └── schema │ │ │ ├── PulsarSchemaTest.java │ │ │ ├── PulsarSchemaTypeInformationTest.java │ │ │ ├── PulsarSchemaTypeSerializerTest.java │ │ │ ├── PulsarSchemaUtilsTest.java │ │ │ └── factories │ │ │ ├── AvroSchemaFactoryTest.java │ │ │ ├── JSONSchemaFactoryTest.java │ │ │ ├── KeyValueSchemaFactoryTest.java │ │ │ ├── ProtobufNativeSchemaFactoryTest.java │ │ │ └── ProtobufSchemaFactoryTest.java │ │ ├── sink │ │ ├── PulsarSinkBuilderTest.java │ │ ├── PulsarSinkITCase.java │ │ ├── committer │ │ │ └── PulsarCommittableSerializerTest.java │ │ └── writer │ │ │ ├── PulsarWriterTest.java │ │ │ ├── router │ │ │ ├── KeyHashTopicRouterTest.java │ │ │ └── RoundRobinTopicRouterTest.java │ │ │ └── topic │ │ │ ├── MetadataListenerTest.java │ │ │ └── ProducerRegisterTest.java │ │ ├── source │ │ ├── PulsarSourceBuilderTest.java │ │ ├── PulsarSourceITCase.java │ │ ├── enumerator │ │ │ ├── PulsarSourceEnumStateSerializerTest.java │ │ │ ├── PulsarSourceEnumeratorTest.java │ │ │ ├── assigner │ │ │ │ └── SplitAssignerImplTest.java │ │ │ ├── cursor │ │ │ │ └── StopCursorTest.java │ │ │ ├── subscriber │ │ │ │ └── PulsarSubscriberTest.java │ │ │ └── topic │ │ │ │ ├── TopicNameUtilsTest.java │ │ │ │ ├── TopicPartitionTest.java │ │ │ │ ├── TopicRangeTest.java │ │ │ │ └── range │ │ │ │ └── TopicRangeUtilsTest.java │ │ ├── reader │ │ │ ├── PulsarPartitionSplitReaderTest.java │ │ │ ├── PulsarSourceReaderTest.java │ │ │ └── deserializer │ │ │ │ ├── GenericRecordDeserializationSchemaTest.java │ │ │ │ └── PulsarDeserializationSchemaTest.java │ │ └── split │ │ │ └── PulsarPartitionSplitSerializerTest.java │ │ ├── table │ │ ├── PulsarChangelogTableITCase.java │ │ ├── PulsarTableFactoryTest.java │ │ ├── PulsarTableITCase.java │ │ ├── PulsarTableOptionUtilsTest.java │ │ ├── PulsarTableOptionsTest.java │ │ ├── PulsarTableTestBase.java │ │ ├── PulsarTableValidationUtilsTest.java │ │ ├── UpsertPulsarTableITCase.java │ │ └── testutils │ │ │ ├── MockPulsarAuthentication.java │ │ │ ├── MockTopicRouter.java │ │ │ ├── PulsarTableTestUtils.java │ │ │ └── TestingUser.java │ │ └── testutils │ │ ├── PulsarTestCommonUtils.java │ │ ├── PulsarTestContext.java │ │ ├── PulsarTestContextFactory.java │ │ ├── PulsarTestEnvironment.java │ │ ├── PulsarTestKeyReader.java │ │ ├── PulsarTestSuiteBase.java │ │ ├── SampleData.java │ │ ├── function │ │ └── ControlSource.java │ │ ├── runtime │ │ ├── PulsarRuntime.java │ │ ├── PulsarRuntimeOperator.java │ │ ├── container │ │ │ └── PulsarContainerRuntime.java │ │ ├── remote │ │ │ └── PulsarRemoteRuntime.java │ │ └── singleton │ │ │ └── PulsarSingletonRuntime.java │ │ ├── sink │ │ ├── PulsarSinkTestContext.java │ │ ├── cases │ │ │ ├── AutoCreateTopicProducingContext.java │ │ │ ├── EncryptedMessageProducingContext.java │ │ │ ├── MultipleTopicsProducingContext.java │ │ │ └── SingleTopicProducingContext.java │ │ └── reader │ │ │ ├── PulsarEncryptDataReader.java │ │ │ └── PulsarPartitionDataReader.java │ │ └── source │ │ ├── PulsarSourceTestContext.java │ │ ├── cases │ │ ├── EncryptedMessagesConsumingContext.java │ │ ├── MultipleTopicsConsumingContext.java │ │ ├── PartialKeysConsumingContext.java │ │ └── SingleTopicConsumingContext.java │ │ └── writer │ │ ├── KeyedPulsarPartitionDataWriter.java │ │ ├── PulsarEncryptDataWriter.java │ │ └── PulsarPartitionDataWriter.java │ └── resources │ ├── archunit.properties │ ├── canal-data.txt │ ├── debezium-data-schema-exclude.txt │ ├── log4j2-test.properties │ ├── maxwell-data.txt │ └── protobuf │ └── sample_message.proto ├── flink-sql-connector-pulsar ├── pom.xml └── src │ └── main │ └── resources │ └── META-INF │ ├── NOTICE │ └── licences │ └── LICENSE.bouncycastle ├── pom.xml └── tools ├── ci └── log4j.properties └── maven ├── checkstyle.xml └── suppressions.xml /.asf.yaml: -------------------------------------------------------------------------------- 1 | github: 2 | enabled_merge_buttons: 3 | squash: true 4 | merge: false 5 | rebase: true 6 | labels: 7 | - flink 8 | - pulsar 9 | - connector 10 | - datastream 11 | - table 12 | - sql 13 | autolink_jira: FLINK 14 | collaborators: 15 | - flinkbot 16 | notifications: 17 | commits: commits@flink.apache.org 18 | issues: issues@flink.apache.org 19 | pullrequests: issues@flink.apache.org 20 | jobs: builds@flink.apache.org 21 | jira_options: link label 22 | -------------------------------------------------------------------------------- /.dlc.json: -------------------------------------------------------------------------------- 1 | { 2 | "ignorePatterns": [ 3 | { 4 | "pattern": "^http://localhost" 5 | }, 6 | { 7 | "pattern": "^https://mvnrepository.com" 8 | }, 9 | { 10 | "pattern": "^https://img.shields.io" 11 | }, 12 | { 13 | "pattern": "^https://tokei.rs" 14 | }, 15 | { 16 | "pattern": "^https://json.org/" 17 | }, 18 | { 19 | "pattern": "^https://opencollective.com" 20 | } 21 | ], 22 | "timeout": "30s", 23 | "retryOn429": true, 24 | "retryCount": 10, 25 | "fallbackRetryDelay": "1000s", 26 | "aliveStatusCodes": [ 27 | 0, 28 | 200, 29 | 401, 30 | 403 31 | ] 32 | } 33 | -------------------------------------------------------------------------------- /.github/PULL_REQUEST_TEMPLATE.md: -------------------------------------------------------------------------------- 1 | 13 | 14 | ## Purpose of the change 15 | 16 | *For example: Add dynamic sink topic support for Pulsar connector.* 17 | 18 | ## Brief change log 19 | 20 | - *Change the internal design of `ProducerRegister`.* 21 | - *Expose topic metadata query in `PulsarSinkContext`.* 22 | - *Change the internal metadata cache in `MetadataListener`.* 23 | 24 | ## Verifying this change 25 | 26 | Please make sure both new and modified tests in this PR follows the conventions defined in our code quality 27 | guide: https://flink.apache.org/contributing/code-style-and-quality-common.html#testing 28 | 29 | *(Please pick either of the following options)* 30 | 31 | This change is a trivial rework / code cleanup without any test coverage. 32 | 33 | *(or)* 34 | 35 | This change is already covered by existing tests, such as *(please describe tests)*. 36 | 37 | *(or)* 38 | 39 | This change added tests and can be verified as follows: 40 | 41 | *(example:)* 42 | 43 | - *Added unit tests* 44 | - *Added integration tests for end-to-end deployment* 45 | - *Manually verified by running the Pulsar connector on a local Flink cluster.* 46 | 47 | ## Significant changes 48 | 49 | *(Please check any boxes [x] if the answer is "yes". You can first publish the PR and check them afterwards, for 50 | convenience.)* 51 | 52 | - [ ] Dependencies have been added or upgraded 53 | - [ ] Public API has been changed (Public API is any class annotated with `@Public(Evolving)`) 54 | - [ ] Serializers have been changed 55 | - [ ] New feature has been introduced 56 | - If yes, how is this documented? (not applicable / docs / JavaDocs / not documented) 57 | -------------------------------------------------------------------------------- /.github/workflows/ci.yml: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # We need to specify repo related information here since Apache INFRA doesn't differentiate 20 | # between several workflows with the same names while preparing a report for GHA usage 21 | # https://infra-reports.apache.org/#ghactions 22 | name: Flink Connector Pulsar CI 23 | on: 24 | push: 25 | branches: 26 | - 'main' 27 | - 'v*' 28 | pull_request: 29 | branches: 30 | - 'main' 31 | - 'v*' 32 | 33 | # Concurrency strategy: 34 | # github.workflow: distinguish this workflow from others 35 | # github.event_name: distinguish `push` event from `pull_request` event 36 | # github.event.number: set to the number of the pull request if `pull_request` event 37 | # github.run_id: otherwise, it's a `push` or `schedule` event, only cancel if we rerun the workflow 38 | # 39 | # Reference: 40 | # https://docs.github.com/en/actions/using-jobs/using-concurrency 41 | # https://docs.github.com/en/actions/learn-github-actions/contexts#github-context 42 | concurrency: 43 | group: ${{ github.workflow }}-${{ github.event_name }}-${{ github.event.number || github.run_id }} 44 | cancel-in-progress: true 45 | 46 | jobs: 47 | compile_and_test: 48 | strategy: 49 | matrix: 50 | flink: [ 1.17.1, 1.18.0 ] 51 | jdk: [ 8, 11, 17 ] 52 | exclude: 53 | - jdk: 17 54 | flink: 1.17.1 55 | uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils 56 | with: 57 | flink_version: ${{ matrix.flink }} 58 | jdk_version: ${{ matrix.jdk }} 59 | timeout_global: 120 60 | timeout_test: 80 61 | check_dead_links: 62 | runs-on: ubuntu-latest 63 | timeout-minutes: 30 64 | steps: 65 | - uses: actions/checkout@v3 66 | - run: sudo npm install -g markdown-link-check@3.10.0 67 | - run: | 68 | for file in $(find . -name "*.md"); do 69 | markdown-link-check -c .dlc.json -q "$file" 70 | done 71 | -------------------------------------------------------------------------------- /.github/workflows/daily.yml: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # We need to specify repo related information here since Apache INFRA doesn't differentiate 20 | # between several workflows with the same names while preparing a report for GHA usage 21 | # https://infra-reports.apache.org/#ghactions 22 | name: Nightly Flink Connector Pulsar 23 | on: 24 | schedule: 25 | - cron: "0 0 * * *" 26 | workflow_dispatch: 27 | 28 | jobs: 29 | compile_and_test: 30 | if: github.repository_owner == 'apache' 31 | strategy: 32 | fail-fast: false 33 | matrix: 34 | jdk: [ 8, 11 ] 35 | flink_branches: [{ 36 | flink: 1.17-SNAPSHOT, 37 | branch: main 38 | }, { 39 | flink: 1.18-SNAPSHOT, 40 | branch: main 41 | }, { 42 | flink: 1.16.2, 43 | branch: v3.0 44 | }, { 45 | flink: 1.17.1, 46 | branch: v4.0 47 | }] 48 | include: 49 | - jdk: 17 50 | flink_branches: { 51 | flink: 1.18-SNAPSHOT, 52 | branch: main 53 | } 54 | uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils 55 | with: 56 | jdk_version: ${{ matrix.jdk }} 57 | flink_version: ${{ matrix.flink_branches.flink }} 58 | connector_branch: ${{ matrix.flink_branches.branch }} 59 | run_dependency_convergence: false 60 | timeout_global: 120 61 | timeout_test: 80 62 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .eslintcache 2 | .cache 3 | scalastyle-output.xml 4 | .classpath 5 | .idea/* 6 | !.idea/vcs.xml 7 | .metadata 8 | .settings 9 | .project 10 | .version.properties 11 | filter.properties 12 | logs.zip 13 | .mvn/wrapper/*.jar 14 | target 15 | tmp 16 | *.class 17 | *.iml 18 | *.swp 19 | *.jar 20 | *.zip 21 | *.log 22 | *.pyc 23 | .DS_Store 24 | build-target 25 | atlassian-ide-plugin.xml 26 | out/ 27 | /docs/api 28 | /docs/.bundle 29 | /docs/.rubydeps 30 | /docs/ruby2/.bundle 31 | /docs/ruby2/.rubydeps 32 | /docs/.jekyll-metadata 33 | *.ipr 34 | *.iws 35 | tools/flink 36 | tools/flink-* 37 | tools/releasing/release 38 | tools/japicmp-output -------------------------------------------------------------------------------- /.gitmodules: -------------------------------------------------------------------------------- 1 | [submodule "tools/releasing/shared"] 2 | path = tools/releasing/shared 3 | url = https://github.com/apache/flink-connector-shared-utils 4 | branch = release_utils -------------------------------------------------------------------------------- /.idea/vcs.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | Apache Flink Pulsar Connector 2 | Copyright 2014-2024 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | 7 | Permission to use, copy, modify, and/or distribute this software for any purpose with or without fee is hereby 8 | granted, provided that this permission notice appear in all copies. 9 | 10 | THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH REGARD TO THIS SOFTWARE INCLUDING 11 | ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, 12 | DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, 13 | WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE 14 | USE OR PERFORMANCE OF THIS SOFTWARE. 15 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Apache Flink Pulsar Connector 2 | 3 | This repository contains the official Apache Flink Pulsar connector. 4 | 5 | ## Apache Flink 6 | 7 | Apache Flink is an open source stream processing framework with powerful stream- and batch-processing capabilities. 8 | 9 | Learn more about Flink at [https://flink.apache.org/](https://flink.apache.org/) 10 | 11 | ## Building the Apache Flink Pulsar Connector from Source 12 | 13 | Prerequisites: 14 | 15 | * Unix-like environment (we use Linux, Mac OS X) 16 | * Git 17 | * Maven (we recommend version 3.8.6) 18 | * Java 11 19 | 20 | ``` 21 | git clone https://github.com/apache/flink-connector-pulsar.git 22 | cd flink-connector-pulsar 23 | mvn clean package -DskipTests 24 | ``` 25 | 26 | The resulting jars can be found in the `target` directory of the respective module. 27 | 28 | ## Developing Flink 29 | 30 | The Flink committers use IntelliJ IDEA to develop the Flink codebase. 31 | We recommend IntelliJ IDEA for developing projects that involve Scala code. 32 | 33 | Minimal requirements for an IDE are: 34 | * Support for Java and Scala (also mixed projects) 35 | * Support for Maven with Java and Scala 36 | 37 | ### IntelliJ IDEA 38 | 39 | The IntelliJ IDE supports Maven out of the box and offers a plugin for Scala development. 40 | 41 | * IntelliJ download: [https://www.jetbrains.com/idea/](https://www.jetbrains.com/idea/) 42 | * IntelliJ Scala Plugin: [https://plugins.jetbrains.com/plugin/?id=1347](https://plugins.jetbrains.com/plugin/?id=1347) 43 | 44 | Check out our [Setting up IntelliJ](https://nightlies.apache.org/flink/flink-docs-master/flinkDev/ide_setup.html#intellij-idea) guide for details. 45 | 46 | ## Support 47 | 48 | Don’t hesitate to ask! 49 | 50 | Contact the developers and community on the [mailing lists](https://flink.apache.org/community.html#mailing-lists) if you need any help. 51 | 52 | [Open an issue](https://issues.apache.org/jira/browse/FLINK) if you found a bug in Flink. 53 | 54 | ## Documentation 55 | 56 | The documentation of Apache Flink is located on the website: [https://flink.apache.org](https://flink.apache.org) 57 | or in the `docs/` directory of the source code. 58 | 59 | ## Fork and Contribute 60 | 61 | This is an active open-source project. We are always open to people who want to use the system or contribute to it. 62 | Contact us if you are looking for implementation tasks that fit your skills. 63 | This article describes [how to contribute to Apache Flink](https://flink.apache.org/contributing/how-to-contribute.html). 64 | 65 | ## About 66 | 67 | Apache Flink is an open source project of The Apache Software Foundation (ASF). 68 | The Apache Flink project originated from the [Stratosphere](http://stratosphere.eu) research project. 69 | -------------------------------------------------------------------------------- /docs/content.zh/docs/connectors/table/pulsar.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: Pulsar 3 | weight: 9 4 | type: docs 5 | --- 6 | 7 | 25 | 26 | # Apache Pulsar SQL Connector 27 | 28 | {{< label "Scan Source: Unbounded" >}} 29 | {{< label "Scan Source: Bounded" >}} 30 | {{< label "Sink: Streaming Append Mode" >}} 31 | 32 | The Pulsar connector allows for reading data from and writing data into Pulsar topics. 33 | 34 | Dependencies 35 | ------------ 36 | 37 | The Pulsar connector is not part of the binary distribution. 38 | See how to link with it for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). 39 | 40 | How to create a Pulsar table 41 | ---------------- 42 | 43 | The example below shows how to create a Pulsar table: 44 | 45 | ```sql 46 | CREATE TABLE PulsarTable ( 47 | `user_id` BIGINT, 48 | `item_id` BIGINT, 49 | `behavior` STRING, 50 | `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' 51 | ) WITH ( 52 | 'connector' = 'pulsar', 53 | 'topics' = 'user_behavior', 54 | 'service-url' = 'pulsar://my-broker.com:6650', 55 | ) 56 | ``` 57 | 58 | Connector Options 59 | ---------------- 60 | 61 | {{< generated/pulsar_table_configuration >}} 62 | -------------------------------------------------------------------------------- /docs/content/docs/connectors/table/pulsar.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: Pulsar 3 | weight: 9 4 | type: docs 5 | --- 6 | 7 | 25 | 26 | # Apache Pulsar SQL Connector 27 | 28 | {{< label "Scan Source: Unbounded" >}} 29 | {{< label "Scan Source: Bounded" >}} 30 | {{< label "Sink: Streaming Append Mode" >}} 31 | 32 | The Pulsar connector allows for reading data from and writing data into Pulsar topics. 33 | 34 | Dependencies 35 | ------------ 36 | 37 | The Pulsar connector is not part of the binary distribution. 38 | See how to link with it for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). 39 | 40 | How to create a Pulsar table 41 | ---------------- 42 | 43 | The example below shows how to create a Pulsar table: 44 | 45 | ```sql 46 | CREATE TABLE PulsarTable ( 47 | `user_id` BIGINT, 48 | `item_id` BIGINT, 49 | `behavior` STRING, 50 | `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' 51 | ) WITH ( 52 | 'connector' = 'pulsar', 53 | 'topics' = 'user_behavior', 54 | 'service-url' = 'pulsar://my-broker.com:6650', 55 | ) 56 | ``` 57 | 58 | Connector Options 59 | ---------------- 60 | 61 | {{< generated/pulsar_table_configuration >}} 62 | -------------------------------------------------------------------------------- /docs/data/pulsar.yml: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | version: 4.1.0 20 | flink_compatibility: [1.17, 1.18] 21 | variants: 22 | - maven: flink-connector-pulsar 23 | sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-pulsar/$full_version/flink-sql-connector-pulsar-$full_version.jar 24 | -------------------------------------------------------------------------------- /flink-connector-pulsar-e2e-tests/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSinkE2ECase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.tests.util.pulsar; 20 | 21 | import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; 22 | import org.apache.flink.connector.pulsar.testutils.sink.cases.SingleTopicProducingContext; 23 | import org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment; 24 | import org.apache.flink.connector.testframe.external.ExternalContextFactory; 25 | import org.apache.flink.connector.testframe.junit.annotations.TestContext; 26 | import org.apache.flink.connector.testframe.junit.annotations.TestEnv; 27 | import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; 28 | import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; 29 | import org.apache.flink.connector.testframe.testsuites.SinkTestSuiteBase; 30 | import org.apache.flink.streaming.api.CheckpointingMode; 31 | import org.apache.flink.tests.util.pulsar.common.FlinkContainerUtils; 32 | import org.apache.flink.tests.util.pulsar.common.PulsarContainerTestEnvironment; 33 | 34 | import static org.apache.flink.streaming.api.CheckpointingMode.AT_LEAST_ONCE; 35 | import static org.apache.flink.streaming.api.CheckpointingMode.EXACTLY_ONCE; 36 | 37 | /** Pulsar sink E2E test based on the connector testing framework. */ 38 | @SuppressWarnings("unused") 39 | public class PulsarSinkE2ECase extends SinkTestSuiteBase { 40 | 41 | // Defines the Semantic. 42 | @TestSemantics 43 | CheckpointingMode[] semantics = new CheckpointingMode[] {AT_LEAST_ONCE, EXACTLY_ONCE}; 44 | 45 | // Defines TestEnvironment 46 | @TestEnv 47 | FlinkContainerTestEnvironment flink = 48 | new FlinkContainerTestEnvironment(FlinkContainerUtils.flinkConfiguration(), 1, 6); 49 | 50 | // Defines ConnectorExternalSystem. 51 | @TestExternalSystem PulsarTestEnvironment pulsar = new PulsarContainerTestEnvironment(flink); 52 | 53 | // Defines a set of external context Factories for different test cases. 54 | @TestContext 55 | ExternalContextFactory sinkContext = 56 | ignore -> { 57 | final SingleTopicProducingContext context = new SingleTopicProducingContext(pulsar); 58 | context.addConnectorJarPaths(FlinkContainerUtils.connectorJarPaths()); 59 | return context; 60 | }; 61 | } 62 | -------------------------------------------------------------------------------- /flink-connector-pulsar-e2e-tests/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.tests.util.pulsar.common; 20 | 21 | import org.apache.flink.configuration.Configuration; 22 | import org.apache.flink.configuration.JobManagerOptions; 23 | import org.apache.flink.configuration.MemorySize; 24 | import org.apache.flink.configuration.TaskManagerOptions; 25 | 26 | import java.net.URL; 27 | import java.util.ArrayList; 28 | import java.util.List; 29 | 30 | import static org.apache.flink.connector.pulsar.testutils.PulsarTestCommonUtils.resourcePath; 31 | 32 | /** Shared utilities for building Flink containers. */ 33 | public class FlinkContainerUtils { 34 | 35 | public static Configuration flinkConfiguration() { 36 | Configuration configuration = new Configuration(); 37 | 38 | // Increase the jvm metaspace memory to avoid java.lang.OutOfMemoryError in 39 | // - JobManager Metaspace 40 | // - TaskManager Direct Memory 41 | configuration.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(2560)); 42 | configuration.set(TaskManagerOptions.TASK_OFF_HEAP_MEMORY, MemorySize.ofMebiBytes(512)); 43 | configuration.set(TaskManagerOptions.JVM_METASPACE, MemorySize.ofMebiBytes(512)); 44 | configuration.set(JobManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(2560)); 45 | configuration.set(JobManagerOptions.JVM_METASPACE, MemorySize.ofMebiBytes(1024)); 46 | 47 | return configuration; 48 | } 49 | 50 | public static List connectorJarPaths() { 51 | List urls = new ArrayList<>(); 52 | urls.add(resourcePath("pulsar-connector.jar")); 53 | urls.add(resourcePath("flink-connector-testing.jar")); 54 | return urls; 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /flink-connector-pulsar-e2e-tests/src/test/java/org/apache/flink/tests/util/pulsar/common/PulsarContainerTestEnvironment.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.tests.util.pulsar.common; 20 | 21 | import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; 22 | import org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment; 23 | 24 | import static org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime.container; 25 | 26 | /** This test environment is used for create a Pulsar standalone instance for e2e tests. */ 27 | public class PulsarContainerTestEnvironment extends PulsarTestEnvironment { 28 | 29 | public PulsarContainerTestEnvironment(FlinkContainerTestEnvironment flinkEnvironment) { 30 | super(container(flinkEnvironment.getFlinkContainers().getJobManager())); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /flink-connector-pulsar-e2e-tests/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.test.ref = TestLogger 23 | 24 | appender.testlogger.name = TestLogger 25 | appender.testlogger.type = CONSOLE 26 | appender.testlogger.target = SYSTEM_ERR 27 | appender.testlogger.layout.type = PatternLayout 28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n 29 | -------------------------------------------------------------------------------- /flink-connector-pulsar/archunit-violations/185733ff-3e1c-48d3-a0c1-7652b72151f2: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-connector-pulsar/34f217b69701b8a8d4431bb46e32b584d7682701/flink-connector-pulsar/archunit-violations/185733ff-3e1c-48d3-a0c1-7652b72151f2 -------------------------------------------------------------------------------- /flink-connector-pulsar/archunit-violations/5afc9aa4-9738-4cde-9966-1ca0c364183f: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-connector-pulsar/34f217b69701b8a8d4431bb46e32b584d7682701/flink-connector-pulsar/archunit-violations/5afc9aa4-9738-4cde-9966-1ca0c364183f -------------------------------------------------------------------------------- /flink-connector-pulsar/archunit-violations/660539fb-3e21-4ab9-a9c0-2a16a5d6472f: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-connector-pulsar/34f217b69701b8a8d4431bb46e32b584d7682701/flink-connector-pulsar/archunit-violations/660539fb-3e21-4ab9-a9c0-2a16a5d6472f -------------------------------------------------------------------------------- /flink-connector-pulsar/archunit-violations/89e608b0-1438-434c-a08e-80e6a4de4646: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-connector-pulsar/34f217b69701b8a8d4431bb46e32b584d7682701/flink-connector-pulsar/archunit-violations/89e608b0-1438-434c-a08e-80e6a4de4646 -------------------------------------------------------------------------------- /flink-connector-pulsar/archunit-violations/b4233890-ecb8-4921-b0de-45b2f0092a1d: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-connector-pulsar/34f217b69701b8a8d4431bb46e32b584d7682701/flink-connector-pulsar/archunit-violations/b4233890-ecb8-4921-b0de-45b2f0092a1d -------------------------------------------------------------------------------- /flink-connector-pulsar/archunit-violations/eec485f7-b61c-4644-bfa8-460b161d0a4a: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-connector-pulsar/34f217b69701b8a8d4431bb46e32b584d7682701/flink-connector-pulsar/archunit-violations/eec485f7-b61c-4644-bfa8-460b161d0a4a -------------------------------------------------------------------------------- /flink-connector-pulsar/archunit-violations/stored.rules: -------------------------------------------------------------------------------- 1 | # 2 | #Tue Dec 12 23:53:57 CST 2023 3 | Production\ code\ must\ not\ call\ methods\ annotated\ with\ @VisibleForTesting=660539fb-3e21-4ab9-a9c0-2a16a5d6472f 4 | Options\ for\ connectors\ and\ formats\ should\ reside\ in\ a\ consistent\ package\ and\ be\ public\ API.=89e608b0-1438-434c-a08e-80e6a4de4646 5 | Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=b4233890-ecb8-4921-b0de-45b2f0092a1d 6 | Connector\ production\ code\ must\ depend\ only\ on\ public\ API\ when\ outside\ of\ connector\ packages=284bb951-3f59-4332-b61b-6f65b674f2ac 7 | Classes\ in\ API\ packages\ should\ have\ at\ least\ one\ API\ visibility\ annotation.=5afc9aa4-9738-4cde-9966-1ca0c364183f 8 | Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @PublicEvolving\ must\ be\ annotated\ with\ @Public(Evolving).=eec485f7-b61c-4644-bfa8-460b161d0a4a 9 | ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=c10e9875-cfe1-4167-b920-9d84d8d91f18 10 | Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @Public\ must\ be\ annotated\ with\ @Public.=185733ff-3e1c-48d3-a0c1-7652b72151f2 11 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/crypto/PulsarCryptoDisabled.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.crypto; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | 23 | import org.apache.pulsar.client.api.CryptoKeyReader; 24 | 25 | import java.util.Collections; 26 | import java.util.Set; 27 | 28 | /** A default implementation, we will use it if the user didn't enable end-to-end encryption. */ 29 | @Internal 30 | public class PulsarCryptoDisabled implements PulsarCrypto { 31 | private static final long serialVersionUID = 7735076959011427862L; 32 | 33 | @Override 34 | public CryptoKeyReader cryptoKeyReader() { 35 | return null; 36 | } 37 | 38 | @Override 39 | public Set encryptKeys() { 40 | return Collections.emptySet(); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/metrics/ProducerMetricsInterceptor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.metrics; 20 | 21 | import org.apache.flink.metrics.Counter; 22 | import org.apache.flink.metrics.groups.SinkWriterMetricGroup; 23 | 24 | import org.apache.pulsar.client.api.Message; 25 | import org.apache.pulsar.client.api.MessageId; 26 | import org.apache.pulsar.client.api.Producer; 27 | import org.apache.pulsar.client.api.interceptor.ProducerInterceptor; 28 | 29 | /** The metric statistic for Pulsar's {@link Producer}. */ 30 | public class ProducerMetricsInterceptor implements ProducerInterceptor { 31 | 32 | private final Counter numRecordsOutErrors; 33 | private final Counter numRecordsOutCounter; 34 | private final Counter numBytesOutCounter; 35 | 36 | public ProducerMetricsInterceptor(SinkWriterMetricGroup metricGroup) { 37 | this.numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); 38 | this.numRecordsOutCounter = metricGroup.getIOMetricGroup().getNumRecordsOutCounter(); 39 | this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); 40 | } 41 | 42 | @Override 43 | public void close() { 44 | // Nothing to do by default. 45 | } 46 | 47 | @Override 48 | public boolean eligible(Message message) { 49 | return true; 50 | } 51 | 52 | @Override 53 | public Message beforeSend(Producer producer, Message message) { 54 | return message; 55 | } 56 | 57 | @Override 58 | public void onSendAcknowledgement( 59 | Producer producer, Message message, MessageId msgId, Throwable exception) { 60 | if (exception != null) { 61 | numRecordsOutErrors.inc(1); 62 | } else { 63 | numRecordsOutCounter.inc(1); 64 | numBytesOutCounter.inc(message.size()); 65 | } 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/BytesSchema.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.schema; 20 | 21 | import org.apache.pulsar.client.api.Schema; 22 | import org.apache.pulsar.common.schema.SchemaInfo; 23 | 24 | import java.io.Serializable; 25 | import java.nio.ByteBuffer; 26 | 27 | import static org.apache.pulsar.client.internal.PulsarClientImplementationBinding.getBytes; 28 | 29 | /** 30 | * This schema is a wrapper for the original schema. It will send the schema info to Pulsar for 31 | * compatibility check. And didn't deserialize messages. 32 | */ 33 | public class BytesSchema implements Schema, Serializable { 34 | private static final long serialVersionUID = -539752264675729127L; 35 | 36 | private final PulsarSchema schema; 37 | 38 | public BytesSchema(PulsarSchema schema) { 39 | this.schema = schema; 40 | } 41 | 42 | @Override 43 | public void validate(byte[] message) { 44 | schema.getPulsarSchema().validate(message); 45 | } 46 | 47 | @Override 48 | public byte[] encode(byte[] message) { 49 | return message; 50 | } 51 | 52 | @Override 53 | public boolean supportSchemaVersioning() { 54 | return schema.getPulsarSchema().supportSchemaVersioning(); 55 | } 56 | 57 | @Override 58 | public byte[] decode(byte[] bytes) { 59 | return bytes; 60 | } 61 | 62 | @Override 63 | public byte[] decode(byte[] bytes, byte[] schemaVersion) { 64 | // None of Pulsar's schema implementations have implemented this method. 65 | return bytes; 66 | } 67 | 68 | @Override 69 | public byte[] decode(ByteBuffer data, byte[] schemaVersion) { 70 | return getBytes(data); 71 | } 72 | 73 | @Override 74 | public SchemaInfo getSchemaInfo() { 75 | return schema.getSchemaInfo(); 76 | } 77 | 78 | @Override 79 | public Schema clone() { 80 | return this; 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchemaFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.schema; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.api.common.typeinfo.TypeInformation; 23 | 24 | import org.apache.pulsar.client.api.Schema; 25 | import org.apache.pulsar.common.schema.SchemaInfo; 26 | import org.apache.pulsar.common.schema.SchemaType; 27 | 28 | import static org.apache.flink.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; 29 | 30 | /** 31 | * The schema factory for a specified {@link SchemaType}. We add this factory because of pulsar 32 | * don't provide a serializable schema and we can't create it directly from {@link SchemaInfo}. So 33 | * we have to implement this creation logic. 34 | */ 35 | @Internal 36 | public interface PulsarSchemaFactory { 37 | 38 | /** The supported schema type for this factory. We would classify the factory by the type. */ 39 | SchemaType type(); 40 | 41 | /** Create the schema by the given info. */ 42 | Schema createSchema(SchemaInfo info); 43 | 44 | /** Create the flink type information by the given schema info. */ 45 | default TypeInformation createTypeInfo(SchemaInfo info) { 46 | Schema pulsarSchema = createSchema(info); 47 | Class typeClass = decodeClassInfo(info); 48 | PulsarSchema schema = new PulsarSchema<>(pulsarSchema, typeClass); 49 | 50 | return new PulsarSchemaTypeInformation<>(schema); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchemaTypeInformation.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.schema; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.api.common.ExecutionConfig; 23 | import org.apache.flink.api.common.typeinfo.TypeInformation; 24 | import org.apache.flink.api.common.typeutils.TypeSerializer; 25 | 26 | import java.util.Objects; 27 | 28 | /** Wrap the pulsar {@code Schema} into a flink {@code TypeInformation}. */ 29 | @Internal 30 | public class PulsarSchemaTypeInformation extends TypeInformation { 31 | private static final long serialVersionUID = 7284667318651333519L; 32 | 33 | private final PulsarSchema schema; 34 | 35 | public PulsarSchemaTypeInformation(PulsarSchema schema) { 36 | this.schema = schema; 37 | } 38 | 39 | @Override 40 | public boolean isBasicType() { 41 | return false; 42 | } 43 | 44 | @Override 45 | public boolean isTupleType() { 46 | return false; 47 | } 48 | 49 | @Override 50 | public int getArity() { 51 | return 1; 52 | } 53 | 54 | @Override 55 | public int getTotalFields() { 56 | return 1; 57 | } 58 | 59 | @Override 60 | public Class getTypeClass() { 61 | return schema.getRecordClass(); 62 | } 63 | 64 | @Override 65 | public boolean isKeyType() { 66 | return false; 67 | } 68 | 69 | @Override 70 | public TypeSerializer createSerializer(ExecutionConfig config) { 71 | return new PulsarSchemaTypeSerializer<>(schema); 72 | } 73 | 74 | @Override 75 | public String toString() { 76 | return schema.toString(); 77 | } 78 | 79 | @Override 80 | public boolean equals(Object obj) { 81 | if (obj instanceof PulsarSchemaTypeInformation) { 82 | PulsarSchemaTypeInformation that = (PulsarSchemaTypeInformation) obj; 83 | return Objects.equals(schema, that.schema); 84 | } else { 85 | return false; 86 | } 87 | } 88 | 89 | @Override 90 | public int hashCode() { 91 | return schema.hashCode(); 92 | } 93 | 94 | @Override 95 | public boolean canEqual(Object obj) { 96 | return obj instanceof PulsarSchemaTypeInformation; 97 | } 98 | } 99 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.schema.factories; 20 | 21 | import org.apache.flink.api.common.typeinfo.TypeInformation; 22 | import org.apache.flink.api.java.typeutils.AvroUtils; 23 | import org.apache.flink.connector.pulsar.common.schema.PulsarSchemaFactory; 24 | 25 | import org.apache.pulsar.client.api.Schema; 26 | import org.apache.pulsar.client.api.schema.SchemaDefinition; 27 | import org.apache.pulsar.client.impl.schema.AvroSchema; 28 | import org.apache.pulsar.common.schema.SchemaInfo; 29 | import org.apache.pulsar.common.schema.SchemaType; 30 | 31 | import static org.apache.flink.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; 32 | 33 | /** The schema factory for pulsar's {@link AvroSchema}. */ 34 | public class AvroSchemaFactory implements PulsarSchemaFactory { 35 | 36 | @Override 37 | public SchemaType type() { 38 | return SchemaType.AVRO; 39 | } 40 | 41 | @Override 42 | public Schema createSchema(SchemaInfo info) { 43 | Class typeClass = decodeClassInfo(info); 44 | SchemaDefinition definition = 45 | SchemaDefinition.builder() 46 | .withPojo(typeClass) 47 | .withProperties(info.getProperties()) 48 | .build(); 49 | 50 | return AvroSchema.of(definition); 51 | } 52 | 53 | @Override 54 | public TypeInformation createTypeInfo(SchemaInfo info) { 55 | try { 56 | Class decodeClassInfo = decodeClassInfo(info); 57 | return AvroUtils.getAvroUtils().createAvroTypeInfo(decodeClassInfo); 58 | } catch (Exception e) { 59 | return PulsarSchemaFactory.super.createTypeInfo(info); 60 | } 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.schema.factories; 20 | 21 | import org.apache.flink.api.common.typeinfo.TypeInformation; 22 | import org.apache.flink.api.java.typeutils.AvroUtils; 23 | import org.apache.flink.connector.pulsar.common.schema.PulsarSchemaFactory; 24 | 25 | import org.apache.pulsar.client.api.Schema; 26 | import org.apache.pulsar.client.impl.schema.JSONSchema; 27 | import org.apache.pulsar.common.schema.SchemaInfo; 28 | import org.apache.pulsar.common.schema.SchemaType; 29 | 30 | import static org.apache.flink.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; 31 | 32 | /** The schema factory for pulsar's {@link JSONSchema}. */ 33 | public class JSONSchemaFactory implements PulsarSchemaFactory { 34 | 35 | @Override 36 | public SchemaType type() { 37 | return SchemaType.JSON; 38 | } 39 | 40 | @Override 41 | public Schema createSchema(SchemaInfo info) { 42 | Class typeClass = decodeClassInfo(info); 43 | return JSONSchema.of(typeClass, info.getProperties()); 44 | } 45 | 46 | @Override 47 | public TypeInformation createTypeInfo(SchemaInfo info) { 48 | try { 49 | Class decodeClassInfo = decodeClassInfo(info); 50 | return AvroUtils.getAvroUtils().createAvroTypeInfo(decodeClassInfo); 51 | } catch (Exception e) { 52 | return PulsarSchemaFactory.super.createTypeInfo(info); 53 | } 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/ProtobufNativeSchemaFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.schema.factories; 20 | 21 | import org.apache.flink.connector.pulsar.common.schema.PulsarSchemaFactory; 22 | 23 | import com.google.protobuf.GeneratedMessageV3; 24 | import org.apache.pulsar.client.api.Schema; 25 | import org.apache.pulsar.client.impl.schema.ProtobufNativeSchema; 26 | import org.apache.pulsar.common.schema.SchemaInfo; 27 | import org.apache.pulsar.common.schema.SchemaType; 28 | 29 | import static org.apache.flink.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; 30 | 31 | /** The schema factory for pulsar's {@link ProtobufNativeSchema}. */ 32 | public class ProtobufNativeSchemaFactory 33 | implements PulsarSchemaFactory { 34 | 35 | @Override 36 | public SchemaType type() { 37 | return SchemaType.PROTOBUF_NATIVE; 38 | } 39 | 40 | @Override 41 | public Schema createSchema(SchemaInfo info) { 42 | Class typeClass = decodeClassInfo(info); 43 | return ProtobufNativeSchema.of(typeClass, info.getProperties()); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/ProtobufSchemaFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.schema.factories; 20 | 21 | import org.apache.flink.connector.pulsar.common.schema.PulsarSchemaFactory; 22 | 23 | import com.google.protobuf.GeneratedMessageV3; 24 | import org.apache.pulsar.client.api.Schema; 25 | import org.apache.pulsar.client.impl.schema.ProtobufSchema; 26 | import org.apache.pulsar.common.schema.SchemaInfo; 27 | import org.apache.pulsar.common.schema.SchemaType; 28 | 29 | import static org.apache.flink.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; 30 | 31 | /** The schema factory for pulsar's {@link ProtobufSchema}. */ 32 | public class ProtobufSchemaFactory implements PulsarSchemaFactory { 33 | 34 | @Override 35 | public SchemaType type() { 36 | return SchemaType.PROTOBUF; 37 | } 38 | 39 | @Override 40 | public Schema createSchema(SchemaInfo info) { 41 | Class typeClass = decodeClassInfo(info); 42 | return ProtobufSchema.of(typeClass, info.getProperties()); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/StringSchemaFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.schema.factories; 20 | 21 | import org.apache.flink.api.common.typeinfo.Types; 22 | 23 | import org.apache.pulsar.client.api.Schema; 24 | import org.apache.pulsar.client.impl.schema.StringSchema; 25 | import org.apache.pulsar.common.schema.SchemaInfo; 26 | 27 | /** The schema factory for pulsar's {@link StringSchema}. */ 28 | public class StringSchemaFactory extends PrimitiveSchemaFactory { 29 | 30 | public StringSchemaFactory() { 31 | super(Schema.STRING, Types.STRING); 32 | } 33 | 34 | @Override 35 | public Schema createSchema(SchemaInfo info) { 36 | // SchemaInfo contains the string encode type. 37 | return StringSchema.fromSchemaInfo(info); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarTransactionUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.utils; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | 23 | import org.apache.pulsar.client.api.PulsarClient; 24 | import org.apache.pulsar.client.api.PulsarClientException; 25 | import org.apache.pulsar.client.api.transaction.Transaction; 26 | import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; 27 | import org.apache.pulsar.client.api.transaction.TxnID; 28 | import org.apache.pulsar.client.impl.PulsarClientImpl; 29 | 30 | import java.util.concurrent.TimeUnit; 31 | 32 | import static org.apache.flink.util.Preconditions.checkNotNull; 33 | 34 | /** A suit of workarounds for the Pulsar Transaction. */ 35 | @Internal 36 | public final class PulsarTransactionUtils { 37 | 38 | private PulsarTransactionUtils() { 39 | // No public constructor 40 | } 41 | 42 | /** Create transaction with given timeout millis. */ 43 | public static Transaction createTransaction(PulsarClient pulsarClient, long timeoutMs) 44 | throws PulsarClientException { 45 | try { 46 | return pulsarClient 47 | .newTransaction() 48 | .withTransactionTimeout(timeoutMs, TimeUnit.MILLISECONDS) 49 | .build() 50 | .get(); 51 | } catch (InterruptedException e) { 52 | Thread.currentThread().interrupt(); 53 | throw new PulsarClientException(e); 54 | } catch (Exception e) { 55 | throw PulsarClientException.unwrap(e); 56 | } 57 | } 58 | 59 | /** 60 | * {@link PulsarClient} didn't expose the internal {@link TransactionCoordinatorClient} to the 61 | * end user. But the connector needs it to manually commit/abort the transaction by {@link 62 | * TxnID}. 63 | */ 64 | public static TransactionCoordinatorClient getTcClient(PulsarClient pulsarClient) { 65 | TransactionCoordinatorClient coordinatorClient = 66 | ((PulsarClientImpl) pulsarClient).getTcClient(); 67 | // Ensure you have enabled transaction. 68 | checkNotNull(coordinatorClient, "You haven't enable transaction in Pulsar client."); 69 | 70 | return coordinatorClient; 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/committer/PulsarCommittable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.committer; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | 23 | import org.apache.pulsar.client.api.transaction.TxnID; 24 | 25 | import java.util.Objects; 26 | 27 | /** The writer state for Pulsar connector. We would used in Pulsar committer. */ 28 | @Internal 29 | public class PulsarCommittable { 30 | 31 | /** The transaction id. */ 32 | private final TxnID txnID; 33 | 34 | /** The topic name with partition information. */ 35 | private final String topic; 36 | 37 | public PulsarCommittable(TxnID txnID, String topic) { 38 | this.txnID = txnID; 39 | this.topic = topic; 40 | } 41 | 42 | public TxnID getTxnID() { 43 | return txnID; 44 | } 45 | 46 | public String getTopic() { 47 | return topic; 48 | } 49 | 50 | @Override 51 | public boolean equals(Object o) { 52 | if (this == o) { 53 | return true; 54 | } 55 | if (o == null || getClass() != o.getClass()) { 56 | return false; 57 | } 58 | PulsarCommittable that = (PulsarCommittable) o; 59 | return Objects.equals(txnID, that.txnID) && Objects.equals(topic, that.topic); 60 | } 61 | 62 | @Override 63 | public int hashCode() { 64 | return Objects.hash(txnID, topic); 65 | } 66 | 67 | @Override 68 | public String toString() { 69 | return "PulsarCommittable{" + "txnID=" + txnID + ", topic='" + topic + '\'' + '}'; 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/committer/PulsarCommittableSerializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.committer; 20 | 21 | import org.apache.flink.core.io.SimpleVersionedSerializer; 22 | 23 | import org.apache.pulsar.client.api.transaction.TxnID; 24 | 25 | import java.io.ByteArrayInputStream; 26 | import java.io.ByteArrayOutputStream; 27 | import java.io.DataInputStream; 28 | import java.io.DataOutputStream; 29 | import java.io.IOException; 30 | 31 | /** A serializer used to serialize {@link PulsarCommittable}. */ 32 | public class PulsarCommittableSerializer implements SimpleVersionedSerializer { 33 | 34 | private static final int CURRENT_VERSION = 1; 35 | 36 | @Override 37 | public int getVersion() { 38 | return CURRENT_VERSION; 39 | } 40 | 41 | @Override 42 | public byte[] serialize(PulsarCommittable obj) throws IOException { 43 | try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); 44 | final DataOutputStream out = new DataOutputStream(baos)) { 45 | TxnID txnID = obj.getTxnID(); 46 | out.writeLong(txnID.getMostSigBits()); 47 | out.writeLong(txnID.getLeastSigBits()); 48 | out.writeUTF(obj.getTopic()); 49 | out.flush(); 50 | return baos.toByteArray(); 51 | } 52 | } 53 | 54 | @Override 55 | public PulsarCommittable deserialize(int version, byte[] serialized) throws IOException { 56 | try (final ByteArrayInputStream bais = new ByteArrayInputStream(serialized); 57 | final DataInputStream in = new DataInputStream(bais)) { 58 | long mostSigBits = in.readLong(); 59 | long leastSigBits = in.readLong(); 60 | TxnID txnID = new TxnID(mostSigBits, leastSigBits); 61 | String topic = in.readUTF(); 62 | return new PulsarCommittable(txnID, topic); 63 | } 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/context/PulsarSinkContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.writer.context; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.connector.pulsar.sink.PulsarSinkOptions; 23 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; 24 | 25 | import java.util.Optional; 26 | 27 | /** This context provides information on the pulsar record target location. */ 28 | @PublicEvolving 29 | public interface PulsarSinkContext { 30 | 31 | /** 32 | * Get the number of the subtask that PulsarSink is running on. The numbering starts from 0 and 33 | * goes up to parallelism-1. (parallelism as returned by {@link #getNumberOfParallelInstances()} 34 | * 35 | * @return index of subtask 36 | */ 37 | int getParallelInstanceId(); 38 | 39 | /** @return number of parallel PulsarSink tasks. */ 40 | int getNumberOfParallelInstances(); 41 | 42 | /** 43 | * Pulsar can check the schema and upgrade the schema automatically. If you enable this option, 44 | * we wouldn't serialize the record into bytes, we send and serialize it in the client. 45 | */ 46 | boolean isEnableSchemaEvolution(); 47 | 48 | /** Returns the current process time in flink. */ 49 | long processTime(); 50 | 51 | /** 52 | * Query the topic metadata from Pulsar. The query result will be cached in {@link 53 | * PulsarSinkOptions#PULSAR_TOPIC_METADATA_REFRESH_INTERVAL} interval. 54 | * 55 | * @return Return {@link Optional#empty()} if the topic doesn't exist. 56 | */ 57 | Optional topicMetadata(String topic); 58 | } 59 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/context/PulsarSinkContextImpl.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.writer.context; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.api.common.operators.ProcessingTimeService; 23 | import org.apache.flink.api.connector.sink2.Sink.InitContext; 24 | import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; 25 | import org.apache.flink.connector.pulsar.sink.writer.topic.MetadataListener; 26 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; 27 | 28 | import java.util.Optional; 29 | 30 | /** An implementation that would contain all the required context. */ 31 | @Internal 32 | public class PulsarSinkContextImpl implements PulsarSinkContext { 33 | 34 | private final int numberOfParallelSubtasks; 35 | private final int parallelInstanceId; 36 | private final ProcessingTimeService processingTimeService; 37 | private final boolean enableSchemaEvolution; 38 | private final MetadataListener metadataListener; 39 | 40 | public PulsarSinkContextImpl( 41 | InitContext initContext, 42 | SinkConfiguration sinkConfiguration, 43 | MetadataListener metadataListener) { 44 | this.parallelInstanceId = initContext.getSubtaskId(); 45 | this.numberOfParallelSubtasks = initContext.getNumberOfParallelSubtasks(); 46 | this.processingTimeService = initContext.getProcessingTimeService(); 47 | this.enableSchemaEvolution = sinkConfiguration.isEnableSchemaEvolution(); 48 | this.metadataListener = metadataListener; 49 | } 50 | 51 | @Override 52 | public int getParallelInstanceId() { 53 | return parallelInstanceId; 54 | } 55 | 56 | @Override 57 | public int getNumberOfParallelInstances() { 58 | return numberOfParallelSubtasks; 59 | } 60 | 61 | @Override 62 | public boolean isEnableSchemaEvolution() { 63 | return enableSchemaEvolution; 64 | } 65 | 66 | @Override 67 | public long processTime() { 68 | return processingTimeService.getCurrentProcessingTime(); 69 | } 70 | 71 | @Override 72 | public Optional topicMetadata(String topic) { 73 | return metadataListener.queryTopicMetadata(topic); 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/delayer/FixedMessageDelayer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.writer.delayer; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; 23 | 24 | /** A delayer for making sure all the messages would be sent in a fixed delay duration. */ 25 | @PublicEvolving 26 | public class FixedMessageDelayer implements MessageDelayer { 27 | private static final long serialVersionUID = -7550834520312097614L; 28 | 29 | private final long delayDuration; 30 | 31 | public FixedMessageDelayer(long delayDuration) { 32 | this.delayDuration = delayDuration; 33 | } 34 | 35 | @Override 36 | public long deliverAt(IN message, PulsarSinkContext sinkContext) { 37 | if (delayDuration > 0) { 38 | return sinkContext.processTime() + delayDuration; 39 | } else { 40 | return delayDuration; 41 | } 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/delayer/MessageDelayer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.writer.delayer; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; 23 | import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; 24 | 25 | import org.apache.pulsar.client.api.SubscriptionType; 26 | 27 | import java.io.Serializable; 28 | import java.time.Duration; 29 | 30 | /** 31 | * A delayer for Pulsar broker passing the sent message to the downstream consumer. This is only 32 | * works in {@link SubscriptionType#Shared} subscription. 33 | * 34 | *

Read delayed 36 | * message delivery for better understanding this feature. 37 | */ 38 | @PublicEvolving 39 | public interface MessageDelayer extends Serializable { 40 | 41 | /** 42 | * Return the send time for this message. You should calculate the timestamp by using {@link 43 | * PulsarSinkContext#processTime()} and the non-positive value indicate this message should be 44 | * sent immediately. 45 | */ 46 | long deliverAt(IN message, PulsarSinkContext sinkContext); 47 | 48 | /** Implement this method if you have some non-serializable field. */ 49 | default void open(SinkConfiguration sinkConfiguration) { 50 | // Nothing to do by default. 51 | } 52 | 53 | /** All the messages should be consumed immediately. */ 54 | static FixedMessageDelayer never() { 55 | return new FixedMessageDelayer<>(-1L); 56 | } 57 | 58 | /** All the messages should be consumed in a fixed duration. */ 59 | static FixedMessageDelayer fixed(Duration duration) { 60 | return new FixedMessageDelayer<>(duration.toMillis()); 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/router/KeyHashTopicRouter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.writer.router; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; 23 | import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; 24 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; 25 | 26 | import org.apache.pulsar.client.impl.Hash; 27 | import org.apache.pulsar.shade.com.google.common.base.Strings; 28 | 29 | import java.util.List; 30 | import java.util.concurrent.ThreadLocalRandom; 31 | 32 | import static org.apache.flink.util.Preconditions.checkArgument; 33 | import static org.apache.pulsar.client.util.MathUtils.signSafeMod; 34 | 35 | /** 36 | * If you choose the {@link TopicRoutingMode#MESSAGE_KEY_HASH} policy, we would use this 37 | * implementation. We would pick the topic by the message key's hash code. If no message key was 38 | * provided, we would randomly pick one. 39 | * 40 | * @param The message type which should write to Pulsar. 41 | */ 42 | @Internal 43 | public class KeyHashTopicRouter implements TopicRouter { 44 | private static final long serialVersionUID = 2475614648095079804L; 45 | 46 | private final MessageKeyHash messageKeyHash; 47 | 48 | public KeyHashTopicRouter(SinkConfiguration sinkConfiguration) { 49 | this.messageKeyHash = sinkConfiguration.getMessageKeyHash(); 50 | } 51 | 52 | @Override 53 | public TopicPartition route( 54 | IN in, String key, List partitions, PulsarSinkContext context) { 55 | checkArgument( 56 | !partitions.isEmpty(), 57 | "You should provide topics for routing topic by message key hash."); 58 | 59 | int topicIndex; 60 | if (Strings.isNullOrEmpty(key)) { 61 | // We would randomly pick one topic to write. 62 | topicIndex = ThreadLocalRandom.current().nextInt(partitions.size()); 63 | } else { 64 | // Hash the message key and choose the topic to write. 65 | Hash hash = messageKeyHash.getHash(); 66 | int code = hash.makeHash(key); 67 | topicIndex = signSafeMod(code, partitions.size()); 68 | } 69 | 70 | return partitions.get(topicIndex); 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/router/MessageKeyHash.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.writer.router; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.annotation.PublicEvolving; 23 | import org.apache.flink.configuration.DescribedEnum; 24 | import org.apache.flink.configuration.description.InlineElement; 25 | 26 | import org.apache.pulsar.client.impl.Hash; 27 | import org.apache.pulsar.client.impl.JavaStringHash; 28 | import org.apache.pulsar.client.impl.Murmur3Hash32; 29 | 30 | import static org.apache.flink.configuration.description.LinkElement.link; 31 | import static org.apache.flink.configuration.description.TextElement.code; 32 | import static org.apache.flink.configuration.description.TextElement.text; 33 | 34 | /** Predefined the available hash function for routing the message. */ 35 | @PublicEvolving 36 | public enum MessageKeyHash implements DescribedEnum { 37 | 38 | /** Use regular String.hashCode(). */ 39 | JAVA_HASH( 40 | "java-hash", 41 | text( 42 | "This hash would use %s to calculate the message key string's hash code.", 43 | code("String.hashCode()"))) { 44 | @Override 45 | public Hash getHash() { 46 | return JavaStringHash.getInstance(); 47 | } 48 | }, 49 | /** 50 | * Use Murmur3 hashing function. https://en.wikipedia.org/wiki/MurmurHash 52 | */ 53 | MURMUR3_32_HASH( 54 | "murmur-3-32-hash", 55 | text( 56 | "This hash would calculate message key's hash code by using %s algorithm.", 57 | link("https://en.wikipedia.org/wiki/MurmurHash", "Murmur3"))) { 58 | @Override 59 | public Hash getHash() { 60 | return Murmur3Hash32.getInstance(); 61 | } 62 | }; 63 | 64 | private final String name; 65 | private final transient InlineElement desc; 66 | 67 | MessageKeyHash(String name, InlineElement desc) { 68 | this.name = name; 69 | this.desc = desc; 70 | } 71 | 72 | @Internal 73 | public abstract Hash getHash(); 74 | 75 | @Override 76 | public String toString() { 77 | return name; 78 | } 79 | 80 | @Internal 81 | @Override 82 | public InlineElement getDescription() { 83 | return desc; 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/router/RoundRobinTopicRouter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.writer.router; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; 23 | import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; 24 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; 25 | import org.apache.flink.util.Preconditions; 26 | 27 | import java.util.List; 28 | import java.util.concurrent.atomic.AtomicLong; 29 | 30 | import static org.apache.flink.util.Preconditions.checkArgument; 31 | 32 | /** 33 | * If you choose the {@link TopicRoutingMode#ROUND_ROBIN} policy, we would use this implementation. 34 | * We would pick the topic one by one in a fixed batch size. 35 | * 36 | * @param The message type which should write to Pulsar. 37 | */ 38 | @Internal 39 | public class RoundRobinTopicRouter implements TopicRouter { 40 | private static final long serialVersionUID = -1160533263474038206L; 41 | 42 | /** The internal counter for counting the messages. */ 43 | private final AtomicLong counter = new AtomicLong(0); 44 | 45 | /** The size when we switch to another topic. */ 46 | private final int partitionSwitchSize; 47 | 48 | public RoundRobinTopicRouter(SinkConfiguration configuration) { 49 | this.partitionSwitchSize = configuration.getPartitionSwitchSize(); 50 | Preconditions.checkArgument(partitionSwitchSize > 0); 51 | } 52 | 53 | @Override 54 | public TopicPartition route( 55 | IN in, String key, List partitions, PulsarSinkContext context) { 56 | checkArgument( 57 | !partitions.isEmpty(), 58 | "You should provide topics for routing topic by message key hash."); 59 | 60 | long counts = counter.getAndAdd(1); 61 | long index = (counts / partitionSwitchSize) % partitions.size(); 62 | // Avoid digit overflow for message counter. 63 | int topicIndex = (int) (Math.abs(index) % Integer.MAX_VALUE); 64 | 65 | return partitions.get(topicIndex); 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/router/TopicRouter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.writer.router; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.connector.pulsar.sink.PulsarSinkBuilder; 23 | import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; 24 | import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; 25 | import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder; 26 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; 27 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; 28 | 29 | import java.io.Serializable; 30 | import java.util.List; 31 | 32 | /** 33 | * The router for choosing the desired topic to write the Flink records. The user can implement this 34 | * router for complex requirements. We have provided some easy-to-use implementations. 35 | * 36 | *

This topic router is stateless and doesn't have any initialize logic. Make sure you don't 37 | * require some dynamic state. 38 | * 39 | * @param The record type needs to be written to Pulsar. 40 | */ 41 | @PublicEvolving 42 | public interface TopicRouter extends Serializable { 43 | 44 | /** 45 | * Choose the topic by given record & available partition list. You can return a new topic name 46 | * if you need it. 47 | * 48 | * @param in The record instance which need to be written to Pulsar. 49 | * @param key The key of the message from {@link PulsarMessageBuilder#key(String)}. It could be 50 | * null, if message doesn't have a key. 51 | * @param partitions The available partition list. This could be empty if you don't provide any 52 | * topics in {@link PulsarSinkBuilder#setTopics(String...)}. You can return a custom topic, 53 | * but make sure it should contain a partition index in naming. Using {@link 54 | * TopicNameUtils#topicNameWithPartition(String, int)} can easily create a topic name with 55 | * partition index. 56 | * @param context The context contains useful information for determining the topic. 57 | * @return The topic name to use. 58 | */ 59 | TopicPartition route( 60 | IN in, String key, List partitions, PulsarSinkContext context); 61 | 62 | /** Implement this method if you have some non-serializable field. */ 63 | default void open(SinkConfiguration sinkConfiguration) { 64 | // Nothing to do by default. 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/serializer/PulsarSchemaWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.writer.serializer; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; 23 | import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; 24 | import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessage; 25 | 26 | import org.apache.pulsar.client.api.Schema; 27 | import org.apache.pulsar.common.schema.KeyValue; 28 | 29 | /** 30 | * Wrap the Pulsar's Schema into PulsarSerializationSchema. We support schema evolution out of box 31 | * by this implementation. 32 | */ 33 | @Internal 34 | public class PulsarSchemaWrapper implements PulsarSerializationSchema { 35 | private static final long serialVersionUID = -2567052498398184194L; 36 | 37 | private final PulsarSchema pulsarSchema; 38 | 39 | public PulsarSchemaWrapper(Schema schema) { 40 | this(new PulsarSchema<>(schema)); 41 | } 42 | 43 | public PulsarSchemaWrapper(Schema schema, Class clazz) { 44 | this(new PulsarSchema<>(schema, clazz)); 45 | } 46 | 47 | public PulsarSchemaWrapper( 48 | Schema> schema, Class keyClass, Class valueClass) { 49 | this(new PulsarSchema<>(schema, keyClass, valueClass)); 50 | } 51 | 52 | public PulsarSchemaWrapper(PulsarSchema pulsarSchema) { 53 | this.pulsarSchema = pulsarSchema; 54 | } 55 | 56 | @Override 57 | public PulsarMessage serialize(IN element, PulsarSinkContext sinkContext) { 58 | Schema schema = pulsarSchema.getPulsarSchema(); 59 | if (sinkContext.isEnableSchemaEvolution()) { 60 | return PulsarMessage.builder(schema, element).build(); 61 | } else { 62 | return PulsarMessage.builder(schema.encode(element)).build(); 63 | } 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/serializer/PulsarSerializationSchemaWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.writer.serializer; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.api.common.serialization.SerializationSchema; 23 | import org.apache.flink.api.common.serialization.SerializationSchema.InitializationContext; 24 | import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; 25 | import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; 26 | import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessage; 27 | 28 | /** Wrap the Flink's SerializationSchema into PulsarSerializationSchema. */ 29 | @Internal 30 | public class PulsarSerializationSchemaWrapper implements PulsarSerializationSchema { 31 | private static final long serialVersionUID = 4948155843623161119L; 32 | 33 | private final SerializationSchema serializationSchema; 34 | 35 | public PulsarSerializationSchemaWrapper(SerializationSchema serializationSchema) { 36 | this.serializationSchema = serializationSchema; 37 | } 38 | 39 | @Override 40 | public void open( 41 | InitializationContext initializationContext, 42 | PulsarSinkContext sinkContext, 43 | SinkConfiguration sinkConfiguration) 44 | throws Exception { 45 | serializationSchema.open(initializationContext); 46 | } 47 | 48 | @Override 49 | public PulsarMessage serialize(IN element, PulsarSinkContext sinkContext) { 50 | byte[] value = serializationSchema.serialize(element); 51 | return PulsarMessage.builder(value).build(); 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/CursorVerification.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.config; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.annotation.PublicEvolving; 23 | import org.apache.flink.configuration.DescribedEnum; 24 | import org.apache.flink.configuration.description.InlineElement; 25 | 26 | import static org.apache.flink.configuration.description.TextElement.text; 27 | 28 | /** The enum class for defining the cursor verify behavior. */ 29 | @PublicEvolving 30 | public enum CursorVerification implements DescribedEnum { 31 | 32 | /** We would just fail the consuming. */ 33 | FAIL_ON_MISMATCH(text("Fail the consuming from Pulsar when we don't find the related cursor.")), 34 | 35 | /** Print a warn message and start consuming from the valid offset. */ 36 | WARN_ON_MISMATCH(text("Print a warn message and start consuming from the valid offset.")); 37 | 38 | private final transient InlineElement desc; 39 | 40 | CursorVerification(InlineElement desc) { 41 | this.desc = desc; 42 | } 43 | 44 | @Internal 45 | @Override 46 | public InlineElement getDescription() { 47 | return desc; 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator; 20 | 21 | import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; 22 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; 23 | 24 | import java.util.HashSet; 25 | import java.util.Set; 26 | 27 | /** 28 | * The state class for pulsar source enumerator, used for storing the split state. This class is 29 | * managed and controlled by {@link SplitAssigner}. 30 | */ 31 | public class PulsarSourceEnumState { 32 | 33 | /** The topic partitions that have been appended to this source. */ 34 | private final Set appendedPartitions; 35 | 36 | public PulsarSourceEnumState(Set appendedPartitions) { 37 | this.appendedPartitions = appendedPartitions; 38 | } 39 | 40 | public Set getAppendedPartitions() { 41 | return appendedPartitions; 42 | } 43 | 44 | /** The initial assignment state for Pulsar. */ 45 | public static PulsarSourceEnumState initialState() { 46 | return new PulsarSourceEnumState(new HashSet<>()); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator.cursor.start; 20 | 21 | import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; 22 | import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; 23 | 24 | import org.apache.pulsar.client.api.ConsumerBuilder; 25 | import org.apache.pulsar.client.api.MessageId; 26 | 27 | import java.util.Objects; 28 | 29 | /** This cursor would leave pulsar start consuming from a specific message id. */ 30 | public class MessageIdStartCursor implements StartCursor { 31 | private static final long serialVersionUID = -8057345435887170111L; 32 | 33 | private final MessageId messageId; 34 | private final boolean inclusive; 35 | 36 | /** 37 | * The default {@code inclusive} behavior should be controlled in {@link 38 | * ConsumerBuilder#startMessageIdInclusive}. But pulsar has a bug and doesn't support this 39 | * feature currently. So we have to implement this feature by ourselves. 40 | * 41 | * @param messageId The message id for start position. 42 | * @param inclusive Whether we include the start message id in the consuming result. This works 43 | * only if we provide a specified message id instead of {@link MessageId#earliest} or {@link 44 | * MessageId#latest}. 45 | */ 46 | public MessageIdStartCursor(MessageId messageId, boolean inclusive) { 47 | this.messageId = messageId; 48 | this.inclusive = inclusive; 49 | } 50 | 51 | @Override 52 | public CursorPosition position(String topic, int partitionId) { 53 | return new CursorPosition(messageId, inclusive); 54 | } 55 | 56 | @Override 57 | public boolean equals(Object o) { 58 | if (this == o) { 59 | return true; 60 | } 61 | if (o == null || getClass() != o.getClass()) { 62 | return false; 63 | } 64 | MessageIdStartCursor that = (MessageIdStartCursor) o; 65 | return Objects.equals(messageId, that.messageId); 66 | } 67 | 68 | @Override 69 | public int hashCode() { 70 | return Objects.hash(messageId); 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator.cursor.start; 20 | 21 | import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; 22 | import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; 23 | 24 | import java.util.Objects; 25 | 26 | /** This cursor would left pulsar start consuming from a specific publish timestamp. */ 27 | public class TimestampStartCursor implements StartCursor { 28 | private static final long serialVersionUID = 5170578885838095320L; 29 | 30 | private final long timestamp; 31 | 32 | public TimestampStartCursor(long timestamp, boolean inclusive) { 33 | this.timestamp = inclusive ? timestamp : timestamp + 1; 34 | } 35 | 36 | @Override 37 | public CursorPosition position(String topic, int partitionId) { 38 | return new CursorPosition(timestamp); 39 | } 40 | 41 | @Override 42 | public boolean equals(Object o) { 43 | if (this == o) { 44 | return true; 45 | } 46 | if (o == null || getClass() != o.getClass()) { 47 | return false; 48 | } 49 | TimestampStartCursor that = (TimestampStartCursor) o; 50 | return timestamp == that.timestamp; 51 | } 52 | 53 | @Override 54 | public int hashCode() { 55 | return Objects.hash(timestamp); 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator.cursor.stop; 20 | 21 | import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; 22 | 23 | import org.apache.pulsar.client.api.Message; 24 | 25 | import java.util.Objects; 26 | 27 | /** Stop consuming message at the given event time. */ 28 | public class EventTimestampStopCursor implements StopCursor { 29 | private static final long serialVersionUID = 2391576769339369027L; 30 | 31 | private final long timestamp; 32 | private final boolean inclusive; 33 | 34 | public EventTimestampStopCursor(long timestamp, boolean inclusive) { 35 | this.timestamp = timestamp; 36 | this.inclusive = inclusive; 37 | } 38 | 39 | @Override 40 | public StopCondition shouldStop(Message message) { 41 | long eventTime = message.getEventTime(); 42 | return StopCondition.compare(timestamp, eventTime, inclusive); 43 | } 44 | 45 | @Override 46 | public boolean equals(Object o) { 47 | if (this == o) { 48 | return true; 49 | } 50 | if (o == null || getClass() != o.getClass()) { 51 | return false; 52 | } 53 | EventTimestampStopCursor that = (EventTimestampStopCursor) o; 54 | return timestamp == that.timestamp && inclusive == that.inclusive; 55 | } 56 | 57 | @Override 58 | public int hashCode() { 59 | return Objects.hash(timestamp, inclusive); 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator.cursor.stop; 20 | 21 | import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; 22 | 23 | import org.apache.pulsar.client.api.Message; 24 | import org.apache.pulsar.client.api.MessageId; 25 | 26 | import java.util.Objects; 27 | 28 | import static org.apache.flink.util.Preconditions.checkArgument; 29 | import static org.apache.pulsar.client.api.MessageId.earliest; 30 | import static org.apache.pulsar.client.api.MessageId.latest; 31 | 32 | /** 33 | * Stop consuming message at a given message id. We use the {@link MessageId#compareTo(Object)} for 34 | * compare the consuming message with the given message id. 35 | */ 36 | public class MessageIdStopCursor implements StopCursor { 37 | private static final long serialVersionUID = -3990454110809274542L; 38 | 39 | private final MessageId messageId; 40 | 41 | private final boolean inclusive; 42 | 43 | public MessageIdStopCursor(MessageId messageId, boolean inclusive) { 44 | checkArgument(!earliest.equals(messageId), "MessageId.earliest is not supported."); 45 | checkArgument(!latest.equals(messageId), "Use LatestMessageStopCursor instead."); 46 | 47 | this.inclusive = inclusive; 48 | this.messageId = messageId; 49 | } 50 | 51 | @Override 52 | public StopCondition shouldStop(Message message) { 53 | MessageId current = message.getMessageId(); 54 | return StopCondition.compare(messageId, current, inclusive); 55 | } 56 | 57 | @Override 58 | public boolean equals(Object o) { 59 | if (this == o) { 60 | return true; 61 | } 62 | if (o == null || getClass() != o.getClass()) { 63 | return false; 64 | } 65 | MessageIdStopCursor that = (MessageIdStopCursor) o; 66 | return inclusive == that.inclusive && Objects.equals(messageId, that.messageId); 67 | } 68 | 69 | @Override 70 | public int hashCode() { 71 | return Objects.hash(messageId, inclusive); 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator.cursor.stop; 20 | 21 | import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; 22 | 23 | import org.apache.pulsar.client.api.Message; 24 | 25 | /** An implementation which wouldn't stop forever. */ 26 | public class NeverStopCursor implements StopCursor { 27 | private static final long serialVersionUID = -3113601090292771786L; 28 | 29 | @Override 30 | public StopCondition shouldStop(Message message) { 31 | return StopCondition.CONTINUE; 32 | } 33 | 34 | @Override 35 | public boolean equals(Object o) { 36 | if (this == o) { 37 | return true; 38 | } 39 | 40 | return o != null && getClass() == o.getClass(); 41 | } 42 | 43 | @Override 44 | public int hashCode() { 45 | return 31; 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator.cursor.stop; 20 | 21 | import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; 22 | 23 | import org.apache.pulsar.client.api.Message; 24 | 25 | import java.util.Objects; 26 | 27 | /** Stop consuming message at the given publish time. */ 28 | public class PublishTimestampStopCursor implements StopCursor { 29 | private static final long serialVersionUID = 4386276745339324527L; 30 | 31 | private final long timestamp; 32 | private final boolean inclusive; 33 | 34 | public PublishTimestampStopCursor(long timestamp, boolean inclusive) { 35 | this.timestamp = timestamp; 36 | this.inclusive = inclusive; 37 | } 38 | 39 | @Override 40 | public StopCondition shouldStop(Message message) { 41 | long publishTime = message.getPublishTime(); 42 | return StopCondition.compare(timestamp, publishTime, inclusive); 43 | } 44 | 45 | @Override 46 | public boolean equals(Object o) { 47 | if (this == o) { 48 | return true; 49 | } 50 | if (o == null || getClass() != o.getClass()) { 51 | return false; 52 | } 53 | PublishTimestampStopCursor that = (PublishTimestampStopCursor) o; 54 | return timestamp == that.timestamp && inclusive == that.inclusive; 55 | } 56 | 57 | @Override 58 | public int hashCode() { 59 | return Objects.hash(timestamp, inclusive); 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator.subscriber; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.pulsar.source.enumerator.subscriber.impl.TopicListSubscriber; 23 | import org.apache.flink.connector.pulsar.source.enumerator.subscriber.impl.TopicPatternSubscriber; 24 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; 25 | import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator; 26 | 27 | import org.apache.pulsar.client.api.PulsarClient; 28 | import org.apache.pulsar.client.api.RegexSubscriptionMode; 29 | 30 | import java.io.Serializable; 31 | import java.util.List; 32 | import java.util.Set; 33 | import java.util.regex.Pattern; 34 | 35 | /** 36 | * Pulsar consumer allows a few different ways to consume from the topics, including: 37 | * 38 | *

    39 | *
  1. Subscribe from a collection of topics. 40 | *
  2. Subscribe to a topic pattern using Java {@code Regex}. 41 | *
42 | * 43 | *

The PulsarSubscriber provides a unified interface for the Pulsar source to support all these 44 | * two types of subscribing mode. 45 | */ 46 | @Internal 47 | public interface PulsarSubscriber extends Serializable { 48 | 49 | /** 50 | * Get a set of subscribed {@link TopicPartition}s. The method could throw {@link 51 | * IllegalStateException}, an extra try catch is required. 52 | * 53 | * @param generator The range for different partitions. 54 | * @param parallelism The parallelism of flink source. 55 | * @return A subscribed {@link TopicPartition} for each pulsar topic partition. 56 | */ 57 | @SuppressWarnings("java:S112") 58 | Set getSubscribedTopicPartitions(RangeGenerator generator, int parallelism) 59 | throws Exception; 60 | 61 | /** 62 | * Initialize the topic subscriber. 63 | * 64 | * @param client The client interface for querying the topics by regex pattern. 65 | */ 66 | void open(PulsarClient client); 67 | 68 | // ----------------- factory methods -------------- 69 | 70 | static PulsarSubscriber getTopicListSubscriber(List topics) { 71 | return new TopicListSubscriber(topics); 72 | } 73 | 74 | static PulsarSubscriber getTopicPatternSubscriber( 75 | Pattern topicPattern, RegexSubscriptionMode subscriptionMode) { 76 | return new TopicPatternSubscriber(topicPattern, subscriptionMode); 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicMetadata.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator.topic; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | 23 | import static org.apache.flink.util.Preconditions.checkArgument; 24 | import static org.apache.pulsar.common.partition.PartitionedTopicMetadata.NON_PARTITIONED; 25 | 26 | /** The pojo class for pulsar topic metadata information. */ 27 | @PublicEvolving 28 | public final class TopicMetadata { 29 | 30 | /** 31 | * The name of the topic, it would be a {@link TopicNameUtils#topicName(String)} which don't 32 | * contain partition information. 33 | */ 34 | private final String name; 35 | 36 | /** The size for a partitioned topic. It would be zero for non-partitioned topic. */ 37 | private final int partitionSize; 38 | 39 | public TopicMetadata(String name, int partitionSize) { 40 | checkArgument(partitionSize >= 0); 41 | 42 | this.name = name; 43 | this.partitionSize = partitionSize; 44 | } 45 | 46 | public String getName() { 47 | return name; 48 | } 49 | 50 | public boolean isPartitioned() { 51 | return partitionSize != NON_PARTITIONED; 52 | } 53 | 54 | public int getPartitionSize() { 55 | return partitionSize; 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FullRangeGenerator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator.topic.range; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; 23 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; 24 | 25 | import org.apache.pulsar.client.api.SubscriptionType; 26 | 27 | import java.util.List; 28 | 29 | import static java.util.Collections.singletonList; 30 | 31 | /** 32 | * Default implementation for {@link SubscriptionType#Shared}, {@link SubscriptionType#Failover} and 33 | * {@link SubscriptionType#Exclusive} subscription. 34 | */ 35 | @PublicEvolving 36 | public class FullRangeGenerator implements RangeGenerator { 37 | private static final long serialVersionUID = -4571731955155036216L; 38 | 39 | @Override 40 | public List range(TopicMetadata metadata, int parallelism) { 41 | return singletonList(TopicRange.createFullRange()); 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/RangeGenerator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator.topic.range; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; 23 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; 24 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; 25 | 26 | import org.apache.pulsar.client.api.KeySharedPolicy; 27 | import org.apache.pulsar.client.api.SubscriptionType; 28 | 29 | import java.io.Serializable; 30 | import java.util.List; 31 | 32 | /** 33 | * A generator for generating the {@link TopicRange} for given topic. It was used for pulsar's 34 | * {@link SubscriptionType#Key_Shared} mode. {@link TopicRange} would be used in {@link 35 | * KeySharedPolicy} for different pulsar source readers. 36 | * 37 | *

If you implement this interface, make sure that each {@link TopicRange} would be assigned to a 38 | * specified source reader. Since flink parallelism is provided, make sure the pulsar message key's 39 | * hashcode is evenly distributed among these topic ranges. 40 | */ 41 | @PublicEvolving 42 | @FunctionalInterface 43 | public interface RangeGenerator extends Serializable { 44 | 45 | /** 46 | * Generate range for the given topic. 47 | * 48 | * @param metadata The metadata of the topic. 49 | * @param parallelism The reader size for this topic. 50 | */ 51 | List range(TopicMetadata metadata, int parallelism); 52 | 53 | /** Initialize some extra resources when bootstrap the source. */ 54 | default void open(SourceConfiguration sourceConfiguration) { 55 | // This method is used for user implementation. 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/GenericRecordDeserializationSchema.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.pulsar.source.reader.deserializer; 2 | 3 | import org.apache.flink.annotation.Internal; 4 | import org.apache.flink.api.common.typeinfo.TypeInformation; 5 | import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; 6 | import org.apache.flink.util.Collector; 7 | 8 | import org.apache.pulsar.client.api.Message; 9 | import org.apache.pulsar.client.api.schema.GenericRecord; 10 | import org.apache.pulsar.client.impl.PulsarClientImpl; 11 | import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; 12 | import org.apache.pulsar.client.impl.schema.generic.MultiVersionSchemaInfoProvider; 13 | import org.apache.pulsar.common.naming.TopicName; 14 | 15 | import java.util.HashMap; 16 | import java.util.Map; 17 | 18 | /** 19 | * The internal implementation for supporting consuming the messages from Pulsar without a 20 | * predefined schema. 21 | */ 22 | @Internal 23 | public class GenericRecordDeserializationSchema implements PulsarDeserializationSchema { 24 | private static final long serialVersionUID = 1133225716807307498L; 25 | 26 | private transient PulsarClientImpl client; 27 | private transient Map schemaMap; 28 | 29 | private final GenericRecordDeserializer deserializer; 30 | 31 | public GenericRecordDeserializationSchema(GenericRecordDeserializer deserializer) { 32 | this.deserializer = deserializer; 33 | } 34 | 35 | @Override 36 | public void deserialize(Message message, Collector out) throws Exception { 37 | AutoConsumeSchema schema = getSchema(message); 38 | GenericRecord element = schema.decode(message.getData(), message.getSchemaVersion()); 39 | T msg = deserializer.deserialize(element); 40 | 41 | out.collect(msg); 42 | } 43 | 44 | @Override 45 | public TypeInformation getProducedType() { 46 | return deserializer.getProducedType(); 47 | } 48 | 49 | @Override 50 | public void open(PulsarInitializationContext context, SourceConfiguration configuration) 51 | throws Exception { 52 | this.client = (PulsarClientImpl) context.getPulsarClient(); 53 | this.schemaMap = new HashMap<>(); 54 | } 55 | 56 | /** Get or create an auto schema instance for this topic partition. */ 57 | private AutoConsumeSchema getSchema(Message message) { 58 | String topic = message.getTopicName(); 59 | AutoConsumeSchema schema = schemaMap.get(topic); 60 | if (schema != null) { 61 | return schema; 62 | } 63 | 64 | schema = new AutoConsumeSchema(); 65 | 66 | // Set dynamic schema info provider. 67 | TopicName topicName = TopicName.get(topic); 68 | MultiVersionSchemaInfoProvider provider = 69 | new MultiVersionSchemaInfoProvider(topicName, client); 70 | schema.setSchemaInfoProvider(provider); 71 | 72 | schemaMap.put(topic, schema); 73 | return schema; 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/GenericRecordDeserializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.reader.deserializer; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.api.java.typeutils.ResultTypeQueryable; 23 | 24 | import org.apache.pulsar.client.api.schema.GenericRecord; 25 | 26 | import java.io.Serializable; 27 | 28 | /** 29 | * This is deserializer for deserialize the Pulsar's {@link GenericRecord} in a result and send it 30 | * to the downstream. 31 | * 32 | * @param The output message type for sinking to downstream flink operator. 33 | */ 34 | @PublicEvolving 35 | @SuppressWarnings("java:S112") 36 | public interface GenericRecordDeserializer extends Serializable, ResultTypeQueryable { 37 | 38 | /** 39 | * GenericRecord is an interface represents a message with schema. It was created from the 40 | * client with the native object included and the corresponding schema information. 41 | */ 42 | T deserialize(GenericRecord message) throws Exception; 43 | } 44 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaInitializationContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.reader.deserializer; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.api.connector.source.SourceReaderContext; 23 | import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.PulsarInitializationContext; 24 | import org.apache.flink.metrics.MetricGroup; 25 | import org.apache.flink.util.UserCodeClassLoader; 26 | 27 | import org.apache.pulsar.client.api.PulsarClient; 28 | 29 | /** 30 | * Convert the {@link SourceReaderContext} into a {@link PulsarInitializationContext}, we would use 31 | * a pulsar named metric group for this content. 32 | */ 33 | @Internal 34 | public class PulsarDeserializationSchemaInitializationContext 35 | implements PulsarInitializationContext { 36 | 37 | private final SourceReaderContext readerContext; 38 | private final PulsarClient pulsarClient; 39 | 40 | public PulsarDeserializationSchemaInitializationContext( 41 | SourceReaderContext readerContext, PulsarClient pulsarClient) { 42 | this.readerContext = readerContext; 43 | this.pulsarClient = pulsarClient; 44 | } 45 | 46 | @Override 47 | public MetricGroup getMetricGroup() { 48 | return readerContext.metricGroup().addGroup("pulsarDeserializer"); 49 | } 50 | 51 | @Override 52 | public UserCodeClassLoader getUserCodeClassLoader() { 53 | return readerContext.getUserCodeClassLoader(); 54 | } 55 | 56 | @Override 57 | public PulsarClient getPulsarClient() { 58 | return pulsarClient; 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.reader.deserializer; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.api.common.serialization.DeserializationSchema; 23 | import org.apache.flink.api.common.typeinfo.TypeInformation; 24 | import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; 25 | import org.apache.flink.util.Collector; 26 | 27 | import org.apache.pulsar.client.api.Message; 28 | 29 | /** 30 | * A {@link PulsarDeserializationSchema} implementation which based on the given flink's {@link 31 | * DeserializationSchema}. We would consume the message as a byte array from pulsar and deserialize 32 | * it by using flink serialization logic. 33 | * 34 | * @param The output type of the message. 35 | */ 36 | @Internal 37 | public class PulsarDeserializationSchemaWrapper implements PulsarDeserializationSchema { 38 | private static final long serialVersionUID = -630646912412751300L; 39 | 40 | private final DeserializationSchema deserializationSchema; 41 | 42 | public PulsarDeserializationSchemaWrapper(DeserializationSchema deserializationSchema) { 43 | this.deserializationSchema = deserializationSchema; 44 | } 45 | 46 | @Override 47 | public void open(PulsarInitializationContext context, SourceConfiguration configuration) 48 | throws Exception { 49 | // Initialize it for some custom logic. 50 | deserializationSchema.open(context); 51 | } 52 | 53 | @Override 54 | public void deserialize(Message message, Collector out) throws Exception { 55 | byte[] bytes = message.getData(); 56 | T instance = deserializationSchema.deserialize(bytes); 57 | 58 | out.collect(instance); 59 | } 60 | 61 | @Override 62 | public TypeInformation getProducedType() { 63 | return deserializationSchema.getProducedType(); 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.pulsar.source.reader.deserializer; 19 | 20 | import org.apache.flink.annotation.Internal; 21 | import org.apache.flink.api.common.typeinfo.TypeInformation; 22 | import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; 23 | import org.apache.flink.util.Collector; 24 | 25 | import org.apache.pulsar.client.api.Message; 26 | import org.apache.pulsar.client.api.Schema; 27 | import org.apache.pulsar.common.schema.KeyValue; 28 | import org.apache.pulsar.common.schema.SchemaInfo; 29 | 30 | import static org.apache.flink.connector.pulsar.common.schema.PulsarSchemaUtils.createTypeInformation; 31 | 32 | /** 33 | * The deserialization schema wrapper for pulsar original {@link Schema}. Pulsar would deserialize 34 | * the message and pass it to flink with an auto generated {@link TypeInformation}. 35 | * 36 | * @param The output type of the message. 37 | */ 38 | @Internal 39 | public class PulsarSchemaWrapper implements PulsarDeserializationSchema { 40 | private static final long serialVersionUID = -4864701207257059158L; 41 | 42 | /** The serializable pulsar schema, it wrap the schema with type class. */ 43 | private final PulsarSchema pulsarSchema; 44 | 45 | public PulsarSchemaWrapper(Schema schema) { 46 | this(new PulsarSchema<>(schema)); 47 | } 48 | 49 | public PulsarSchemaWrapper(Schema schema, Class clazz) { 50 | this(new PulsarSchema<>(schema, clazz)); 51 | } 52 | 53 | public PulsarSchemaWrapper( 54 | Schema> schema, Class keyClass, Class valueClass) { 55 | this(new PulsarSchema<>(schema, keyClass, valueClass)); 56 | } 57 | 58 | public PulsarSchemaWrapper(PulsarSchema pulsarSchema) { 59 | this.pulsarSchema = pulsarSchema; 60 | } 61 | 62 | @Override 63 | public void deserialize(Message message, Collector out) throws Exception { 64 | Schema schema = this.pulsarSchema.getPulsarSchema(); 65 | byte[] bytes = message.getData(); 66 | T instance = schema.decode(bytes); 67 | 68 | out.collect(instance); 69 | } 70 | 71 | @Override 72 | public TypeInformation getProducedType() { 73 | SchemaInfo info = pulsarSchema.getSchemaInfo(); 74 | return createTypeInformation(info); 75 | } 76 | 77 | public PulsarSchema pulsarSchema() { 78 | return pulsarSchema; 79 | } 80 | } 81 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.pulsar.source.reader.deserializer; 19 | 20 | import org.apache.flink.annotation.Internal; 21 | import org.apache.flink.api.common.ExecutionConfig; 22 | import org.apache.flink.api.common.typeinfo.TypeInformation; 23 | import org.apache.flink.api.common.typeutils.TypeSerializer; 24 | import org.apache.flink.api.java.ExecutionEnvironment; 25 | import org.apache.flink.core.memory.DataInputDeserializer; 26 | import org.apache.flink.util.Collector; 27 | 28 | import org.apache.pulsar.client.api.Message; 29 | 30 | /** 31 | * Wrap the flink TypeInformation into a {@code PulsarDeserializationSchema}. We would create a 32 | * flink {@code TypeSerializer} by using given ExecutionConfig. This execution config could be 33 | * {@link ExecutionEnvironment#getConfig()}. 34 | */ 35 | @Internal 36 | public class PulsarTypeInformationWrapper implements PulsarDeserializationSchema { 37 | private static final long serialVersionUID = 6647084180084963022L; 38 | 39 | /** 40 | * PulsarDeserializationSchema would be shared for multiple SplitReaders in different fetcher 41 | * thread. Use a thread-local DataInputDeserializer would be better. 42 | */ 43 | @SuppressWarnings("java:S5164") 44 | private static final ThreadLocal DESERIALIZER = 45 | ThreadLocal.withInitial(DataInputDeserializer::new); 46 | 47 | private final TypeInformation information; 48 | private final TypeSerializer serializer; 49 | 50 | public PulsarTypeInformationWrapper(TypeInformation information, ExecutionConfig config) { 51 | this.information = information; 52 | this.serializer = information.createSerializer(config); 53 | } 54 | 55 | @Override 56 | public void deserialize(Message message, Collector out) throws Exception { 57 | DataInputDeserializer dis = DESERIALIZER.get(); 58 | dis.setBuffer(message.getData()); 59 | T instance = serializer.deserialize(dis); 60 | 61 | out.collect(instance); 62 | } 63 | 64 | @Override 65 | public TypeInformation getProducedType() { 66 | return information; 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplitState.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.split; 20 | 21 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; 22 | 23 | import org.apache.pulsar.client.api.MessageId; 24 | import org.apache.pulsar.client.api.transaction.TxnID; 25 | 26 | import javax.annotation.Nullable; 27 | 28 | /** Pulsar partition split state. */ 29 | public class PulsarPartitionSplitState { 30 | 31 | private final PulsarPartitionSplit split; 32 | 33 | @Nullable private TxnID uncommittedTransactionId; 34 | 35 | @Nullable private MessageId latestConsumedId; 36 | 37 | public PulsarPartitionSplitState(PulsarPartitionSplit split) { 38 | this.split = split; 39 | } 40 | 41 | /** 42 | * Create a partition split which contains the latest consumed message id as the start position. 43 | */ 44 | public PulsarPartitionSplit toPulsarPartitionSplit() { 45 | return new PulsarPartitionSplit( 46 | split.getPartition(), 47 | split.getStopCursor(), 48 | latestConsumedId, 49 | uncommittedTransactionId); 50 | } 51 | 52 | public TopicPartition getPartition() { 53 | return split.getPartition(); 54 | } 55 | 56 | @Nullable 57 | public TxnID getUncommittedTransactionId() { 58 | return uncommittedTransactionId; 59 | } 60 | 61 | public void setUncommittedTransactionId(@Nullable TxnID uncommittedTransactionId) { 62 | this.uncommittedTransactionId = uncommittedTransactionId; 63 | } 64 | 65 | @Nullable 66 | public MessageId getLatestConsumedId() { 67 | return latestConsumedId; 68 | } 69 | 70 | public void setLatestConsumedId(@Nullable MessageId latestConsumedId) { 71 | this.latestConsumedId = latestConsumedId; 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.pulsar.table.PulsarTableFactory 17 | org.apache.flink.connector.pulsar.table.UpsertPulsarTableFactory 18 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/architecture/ProductionCodeArchitectureTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.architecture; 20 | 21 | import org.apache.flink.architecture.common.ImportOptions; 22 | 23 | import com.tngtech.archunit.core.importer.ImportOption; 24 | import com.tngtech.archunit.junit.AnalyzeClasses; 25 | import com.tngtech.archunit.junit.ArchTest; 26 | import com.tngtech.archunit.junit.ArchTests; 27 | 28 | /** production code for Architecture tests. */ 29 | @AnalyzeClasses( 30 | packages = "org.apache.flink.connector.pulsar", 31 | importOptions = { 32 | ImportOption.DoNotIncludeTests.class, 33 | ImportOption.DoNotIncludeArchives.class, 34 | ImportOptions.ExcludeScalaImportOption.class, 35 | ImportOptions.ExcludeShadedImportOption.class 36 | }) 37 | public class ProductionCodeArchitectureTest { 38 | 39 | @ArchTest 40 | public static final ArchTests COMMON_TESTS = ArchTests.in(ProductionCodeArchitectureBase.class); 41 | } 42 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.architecture; 20 | 21 | import org.apache.flink.architecture.common.ImportOptions; 22 | 23 | import com.tngtech.archunit.core.importer.ImportOption; 24 | import com.tngtech.archunit.junit.AnalyzeClasses; 25 | import com.tngtech.archunit.junit.ArchTest; 26 | import com.tngtech.archunit.junit.ArchTests; 27 | 28 | /** Architecture tests for test code. */ 29 | @AnalyzeClasses( 30 | packages = "org.apache.flink.connector.pulsar", 31 | importOptions = { 32 | ImportOption.OnlyIncludeTests.class, 33 | ImportOptions.ExcludeScalaImportOption.class, 34 | ImportOptions.ExcludeShadedImportOption.class 35 | }) 36 | public class TestCodeArchitectureTest { 37 | 38 | @ArchTest 39 | public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); 40 | } 41 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/config/PulsarConfigValidatorTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.config; 20 | 21 | import org.apache.flink.configuration.ConfigOption; 22 | import org.apache.flink.configuration.ConfigOptions; 23 | import org.apache.flink.configuration.Configuration; 24 | 25 | import org.junit.jupiter.api.Test; 26 | 27 | import static org.assertj.core.api.Assertions.assertThatCode; 28 | import static org.assertj.core.api.Assertions.assertThatThrownBy; 29 | 30 | /** Unit tests for {@link PulsarConfigValidator}. */ 31 | class PulsarConfigValidatorTest { 32 | 33 | @Test 34 | void conflictKeysAndRequiredKeysValidation() { 35 | ConfigOption required = ConfigOptions.key("required").stringType().noDefaultValue(); 36 | ConfigOption c1 = ConfigOptions.key("conflict1").stringType().noDefaultValue(); 37 | ConfigOption c2 = ConfigOptions.key("conflict2").stringType().noDefaultValue(); 38 | 39 | PulsarConfigValidator validator = 40 | PulsarConfigValidator.builder() 41 | .requiredOption(required) 42 | .conflictOptions(c1, c2) 43 | .build(); 44 | Configuration configuration = new Configuration(); 45 | 46 | // Required options 47 | assertThatThrownBy(() -> validator.validate(configuration)) 48 | .isInstanceOf(IllegalArgumentException.class); 49 | 50 | configuration.set(required, "required"); 51 | assertThatCode(() -> validator.validate(configuration)).doesNotThrowAnyException(); 52 | 53 | // Conflict options 54 | configuration.set(c1, "c1"); 55 | assertThatCode(() -> validator.validate(configuration)).doesNotThrowAnyException(); 56 | 57 | configuration.set(c2, "c2"); 58 | assertThatThrownBy(() -> validator.validate(configuration)) 59 | .isInstanceOf(IllegalArgumentException.class); 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/config/PulsarConfigurationTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.config; 20 | 21 | import org.apache.flink.configuration.ConfigOption; 22 | import org.apache.flink.configuration.ConfigOptions; 23 | import org.apache.flink.configuration.Configuration; 24 | 25 | import org.junit.jupiter.api.Test; 26 | 27 | import java.util.Map; 28 | import java.util.Properties; 29 | 30 | import static java.util.Collections.emptyMap; 31 | import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; 32 | import static org.assertj.core.api.Assertions.assertThat; 33 | 34 | /** Unit tests for {@link PulsarConfiguration}. */ 35 | class PulsarConfigurationTest { 36 | 37 | private static final ConfigOption> PROP_OP = 38 | ConfigOptions.key("some.config").mapType().defaultValue(emptyMap()); 39 | 40 | @Test 41 | void pulsarConfigurationCanGetMapWithPrefix() { 42 | Properties expectProp = new Properties(); 43 | for (int i = 0; i < 10; i++) { 44 | expectProp.put(randomAlphabetic(10), randomAlphabetic(10)); 45 | } 46 | 47 | Configuration configuration = new Configuration(); 48 | 49 | for (String name : expectProp.stringPropertyNames()) { 50 | configuration.setString(PROP_OP.key() + "." + name, expectProp.getProperty(name)); 51 | } 52 | 53 | TestConfiguration configuration1 = new TestConfiguration(configuration); 54 | Map properties = configuration1.getProperties(PROP_OP); 55 | 56 | assertThat(properties).isEqualTo(expectProp); 57 | } 58 | 59 | private static final class TestConfiguration extends PulsarConfiguration { 60 | private static final long serialVersionUID = 944689984000450917L; 61 | 62 | private TestConfiguration(Configuration config) { 63 | super(config); 64 | } 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchemaTypeInformationTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.schema; 20 | 21 | import org.apache.flink.api.common.ExecutionConfig; 22 | import org.apache.flink.connector.pulsar.testutils.SampleData.Bar; 23 | import org.apache.flink.util.InstantiationUtil; 24 | 25 | import org.apache.pulsar.client.api.Schema; 26 | import org.junit.jupiter.api.Test; 27 | 28 | import static org.assertj.core.api.Assertions.assertThat; 29 | import static org.assertj.core.api.Assertions.assertThatCode; 30 | 31 | /** Unit tests for {@link PulsarSchemaTypeInformation}. */ 32 | class PulsarSchemaTypeInformationTest { 33 | 34 | @Test 35 | void pulsarTypeInfoSerializationAndCreation() throws Exception { 36 | PulsarSchema schema = new PulsarSchema<>(Schema.AVRO(Bar.class), Bar.class); 37 | PulsarSchemaTypeInformation info = new PulsarSchemaTypeInformation<>(schema); 38 | assertThatCode(() -> InstantiationUtil.clone(info)).doesNotThrowAnyException(); 39 | 40 | PulsarSchemaTypeInformation clonedInfo = InstantiationUtil.clone(info); 41 | assertThat(clonedInfo).isEqualTo(info).isNotSameAs(info); 42 | 43 | assertThatCode(() -> info.createSerializer(new ExecutionConfig())) 44 | .doesNotThrowAnyException(); 45 | 46 | assertThat(clonedInfo.getTypeClass()).isEqualTo(info.getTypeClass()); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.schema.factories; 20 | 21 | import org.apache.flink.api.common.typeinfo.TypeInformation; 22 | import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; 23 | import org.apache.flink.connector.pulsar.testutils.SampleData.FL; 24 | import org.apache.flink.connector.pulsar.testutils.SampleData.Foo; 25 | import org.apache.flink.formats.avro.typeutils.AvroTypeInfo; 26 | import org.apache.flink.util.InstantiationUtil; 27 | 28 | import org.apache.pulsar.client.api.Schema; 29 | import org.apache.pulsar.client.api.schema.SchemaDefinition; 30 | import org.apache.pulsar.client.impl.schema.JSONSchema; 31 | import org.junit.jupiter.api.Test; 32 | 33 | import static org.assertj.core.api.Assertions.assertThat; 34 | import static org.assertj.core.api.Assertions.assertThatCode; 35 | 36 | /** Unit tests for {@link JSONSchemaFactory}. */ 37 | class JSONSchemaFactoryTest { 38 | 39 | @Test 40 | void createJSONSchemaFromSchemaInfo() { 41 | JSONSchema schema = 42 | JSONSchema.of( 43 | SchemaDefinition.builder() 44 | .withPojo(Foo.class) 45 | .withAlwaysAllowNull(false) 46 | .build()); 47 | PulsarSchema pulsarSchema = new PulsarSchema<>(schema, Foo.class); 48 | JSONSchemaFactory factory = new JSONSchemaFactory<>(); 49 | Schema decodedSchema = factory.createSchema(pulsarSchema.getSchemaInfo()); 50 | 51 | assertThat(decodedSchema) 52 | .isInstanceOf(JSONSchema.class) 53 | .hasFieldOrPropertyWithValue("schemaInfo", pulsarSchema.getSchemaInfo()) 54 | .hasFieldOrPropertyWithValue("pojo", Foo.class); 55 | } 56 | 57 | @Test 58 | void createJSONTypeInformationFromSchemaInfo() { 59 | JSONSchema schema = JSONSchema.of(FL.class); 60 | PulsarSchema pulsarSchema = new PulsarSchema<>(schema, FL.class); 61 | JSONSchemaFactory factory = new JSONSchemaFactory<>(); 62 | TypeInformation typeInfo = factory.createTypeInfo(pulsarSchema.getSchemaInfo()); 63 | 64 | assertThat(typeInfo) 65 | .isInstanceOf(AvroTypeInfo.class) 66 | .hasFieldOrPropertyWithValue("typeClass", FL.class); 67 | 68 | // TypeInformation serialization. 69 | assertThatCode(() -> InstantiationUtil.clone(typeInfo)).doesNotThrowAnyException(); 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/ProtobufSchemaFactoryTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.common.schema.factories; 20 | 21 | import org.apache.flink.api.common.typeinfo.TypeInformation; 22 | import org.apache.flink.connector.pulsar.SampleMessage.SubMessage; 23 | import org.apache.flink.connector.pulsar.SampleMessage.TestMessage; 24 | import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; 25 | import org.apache.flink.connector.pulsar.common.schema.PulsarSchemaTypeInformation; 26 | import org.apache.flink.util.InstantiationUtil; 27 | 28 | import org.apache.pulsar.client.api.Schema; 29 | import org.apache.pulsar.client.impl.schema.ProtobufSchema; 30 | import org.junit.jupiter.api.Test; 31 | 32 | import static org.apache.flink.connector.pulsar.common.schema.PulsarSchemaUtils.decodeClassInfo; 33 | import static org.assertj.core.api.Assertions.assertThat; 34 | import static org.assertj.core.api.Assertions.assertThatCode; 35 | 36 | /** Unit tests for {@link ProtobufSchemaFactory}. */ 37 | class ProtobufSchemaFactoryTest { 38 | 39 | @Test 40 | void createProtobufSchemaFromSchemaInfo() { 41 | ProtobufSchema schema = ProtobufSchema.of(SubMessage.class); 42 | PulsarSchema pulsarSchema = new PulsarSchema<>(schema, SubMessage.class); 43 | ProtobufSchemaFactory factory = new ProtobufSchemaFactory<>(); 44 | 45 | Schema schema2 = factory.createSchema(pulsarSchema.getSchemaInfo()); 46 | assertThat(schema2) 47 | .hasFieldOrPropertyWithValue("schemaInfo", pulsarSchema.getSchemaInfo()) 48 | .isInstanceOf(ProtobufSchema.class); 49 | 50 | assertThat(decodeClassInfo(schema2.getSchemaInfo())).isEqualTo(SubMessage.class); 51 | assertThat(schema2).isNotSameAs(schema); 52 | } 53 | 54 | @Test 55 | void createProtobufTypeInfoFromSchemaInfo() { 56 | ProtobufSchema schema = ProtobufSchema.of(TestMessage.class); 57 | PulsarSchema pulsarSchema = new PulsarSchema<>(schema, TestMessage.class); 58 | ProtobufSchemaFactory factory = new ProtobufSchemaFactory<>(); 59 | 60 | TypeInformation typeInfo = 61 | factory.createTypeInfo(pulsarSchema.getSchemaInfo()); 62 | assertThatCode(() -> InstantiationUtil.clone(typeInfo)).doesNotThrowAnyException(); 63 | assertThat(typeInfo) 64 | .isInstanceOf(PulsarSchemaTypeInformation.class) 65 | .hasFieldOrPropertyWithValue("typeClass", TestMessage.class); 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/committer/PulsarCommittableSerializerTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.sink.committer; 20 | 21 | import org.apache.pulsar.client.api.transaction.TxnID; 22 | import org.junit.jupiter.api.Test; 23 | 24 | import java.io.IOException; 25 | import java.util.concurrent.ThreadLocalRandom; 26 | 27 | import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; 28 | import static org.assertj.core.api.Assertions.assertThat; 29 | 30 | /** 31 | * Tests for serializing and deserializing {@link PulsarCommittable} with {@link 32 | * PulsarCommittableSerializer}. 33 | */ 34 | class PulsarCommittableSerializerTest { 35 | 36 | private static final PulsarCommittableSerializer INSTANCE = new PulsarCommittableSerializer(); 37 | 38 | @Test 39 | void committableSerDe() throws IOException { 40 | String topic = randomAlphabetic(10); 41 | TxnID txnID = 42 | new TxnID( 43 | ThreadLocalRandom.current().nextLong(), 44 | ThreadLocalRandom.current().nextLong()); 45 | 46 | PulsarCommittable committable = new PulsarCommittable(txnID, topic); 47 | 48 | byte[] bytes = INSTANCE.serialize(committable); 49 | PulsarCommittable committable1 = INSTANCE.deserialize(INSTANCE.getVersion(), bytes); 50 | 51 | assertThat(committable1).isEqualTo(committable); 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilderTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source; 20 | 21 | import org.apache.pulsar.client.api.Schema; 22 | import org.junit.jupiter.api.Test; 23 | 24 | import static org.assertj.core.api.Assertions.assertThatThrownBy; 25 | 26 | /** Unit tests for {@link PulsarSourceBuilder}. */ 27 | class PulsarSourceBuilderTest { 28 | 29 | @Test 30 | void someSetterMethodCouldOnlyBeCalledOnce() { 31 | PulsarSourceBuilder builder = new PulsarSourceBuilder<>(); 32 | fillRequiredFields(builder); 33 | 34 | assertThatThrownBy(() -> builder.setServiceUrl("service-url2")) 35 | .isInstanceOf(IllegalArgumentException.class); 36 | 37 | assertThatThrownBy(() -> builder.setSubscriptionName("set_subscription_name2")) 38 | .isInstanceOf(IllegalArgumentException.class); 39 | } 40 | 41 | @Test 42 | void topicPatternAndListCouldChooseOnlyOne() { 43 | PulsarSourceBuilder builder = new PulsarSourceBuilder<>(); 44 | fillRequiredFields(builder); 45 | assertThatThrownBy(() -> builder.setTopicPattern("a-a-a")) 46 | .isInstanceOf(IllegalStateException.class); 47 | } 48 | 49 | private void fillRequiredFields(PulsarSourceBuilder builder) { 50 | builder.setServiceUrl("service-url"); 51 | builder.setSubscriptionName("subscription-name"); 52 | builder.setTopics("topic"); 53 | builder.setDeserializationSchema(Schema.STRING); 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartitionTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator.topic; 20 | 21 | import org.junit.jupiter.api.Test; 22 | 23 | import static org.assertj.core.api.Assertions.assertThat; 24 | 25 | /** Unit tests for {@link TopicPartition}. */ 26 | class TopicPartitionTest { 27 | 28 | @Test 29 | void topicNameForPartitionedAndNonPartitionedTopic() { 30 | // For partitioned topic 31 | TopicPartition partition = new TopicPartition("test-name", 12); 32 | assertThat(partition.getFullTopicName()) 33 | .isEqualTo("persistent://public/default/test-name-partition-12"); 34 | 35 | // For non-partitioned topic 36 | TopicPartition partition1 = new TopicPartition("test-topic", -1); 37 | assertThat(partition1.getFullTopicName()) 38 | .isEqualTo("persistent://public/default/test-topic"); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRangeTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.source.enumerator.topic; 20 | 21 | import org.apache.flink.util.InstantiationUtil; 22 | 23 | import org.junit.jupiter.api.Test; 24 | 25 | import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.MAX_RANGE; 26 | import static org.assertj.core.api.Assertions.assertThat; 27 | import static org.assertj.core.api.Assertions.assertThatCode; 28 | import static org.assertj.core.api.Assertions.assertThatThrownBy; 29 | 30 | /** Unit tests for {@link TopicRange}. */ 31 | class TopicRangeTest { 32 | 33 | @Test 34 | void topicRangeIsSerializable() throws Exception { 35 | final TopicRange range = new TopicRange(1, 5); 36 | final TopicRange cloneRange = InstantiationUtil.clone(range); 37 | assertThat(cloneRange).isEqualTo(range); 38 | } 39 | 40 | @Test 41 | void negativeStart() { 42 | assertThatThrownBy(() -> new TopicRange(-1, 1)) 43 | .isInstanceOf(IllegalArgumentException.class); 44 | } 45 | 46 | @Test 47 | void endBelowTheMaximum() { 48 | assertThatCode(() -> new TopicRange(1, MAX_RANGE - 1)).doesNotThrowAnyException(); 49 | } 50 | 51 | @Test 52 | void endOnTheMaximum() { 53 | assertThatCode(() -> new TopicRange(1, MAX_RANGE)).doesNotThrowAnyException(); 54 | } 55 | 56 | @Test 57 | void endAboveTheMaximum() { 58 | assertThatThrownBy(() -> new TopicRange(1, MAX_RANGE + 1)) 59 | .isInstanceOf(IllegalArgumentException.class); 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.table.testutils; 20 | 21 | import org.apache.pulsar.client.api.Authentication; 22 | import org.apache.pulsar.client.api.AuthenticationDataProvider; 23 | import org.apache.pulsar.client.api.PulsarClientException; 24 | import org.apache.pulsar.client.impl.auth.AuthenticationDataNull; 25 | 26 | import java.io.IOException; 27 | import java.util.HashSet; 28 | import java.util.Map; 29 | import java.util.Objects; 30 | import java.util.Set; 31 | import java.util.concurrent.CompletableFuture; 32 | 33 | /** A class to verify Pulsar authentication auth params map is created properly. way to do this */ 34 | public class MockPulsarAuthentication implements Authentication { 35 | public static String key1 = "key1"; 36 | public static String key2 = "key2"; 37 | public static String value1 = "value1"; 38 | public static String value2 = "value2"; 39 | 40 | @Override 41 | public String getAuthMethodName() { 42 | return "custom authentication"; 43 | } 44 | 45 | @Override 46 | public AuthenticationDataProvider getAuthData() { 47 | return new AuthenticationDataNull(); 48 | } 49 | 50 | @Override 51 | public AuthenticationDataProvider getAuthData(String brokerHostName) { 52 | return new AuthenticationDataNull(); 53 | } 54 | 55 | @Override 56 | public void authenticationStage( 57 | String requestUrl, 58 | AuthenticationDataProvider authData, 59 | Map previousResHeaders, 60 | CompletableFuture> authFuture) { 61 | Authentication.super.authenticationStage( 62 | requestUrl, authData, previousResHeaders, authFuture); 63 | } 64 | 65 | @Override 66 | public Set> newRequestHeader( 67 | String hostName, 68 | AuthenticationDataProvider authData, 69 | Map previousResHeaders) { 70 | return new HashSet<>(); 71 | } 72 | 73 | @Override 74 | public void configure(Map authParams) { 75 | assert Objects.equals(authParams.get(key1), value1); 76 | assert Objects.equals(authParams.get(key2), value2); 77 | } 78 | 79 | @Override 80 | public void start() throws PulsarClientException {} 81 | 82 | @Override 83 | public void close() throws IOException {} 84 | } 85 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.table.testutils; 20 | 21 | import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; 22 | import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; 23 | import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; 24 | import org.apache.flink.table.data.RowData; 25 | 26 | import java.util.List; 27 | 28 | /** A mock topic Router for testing purposes only. */ 29 | public class MockTopicRouter implements TopicRouter { 30 | 31 | private static final long serialVersionUID = 1316133122715449818L; 32 | 33 | @Override 34 | public TopicPartition route( 35 | RowData rowData, 36 | String key, 37 | List partitions, 38 | PulsarSinkContext context) { 39 | return new TopicPartition("never-exist-topic"); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.pulsar.table.testutils; 2 | 3 | import java.io.Serializable; 4 | import java.util.Objects; 5 | import java.util.concurrent.ThreadLocalRandom; 6 | 7 | import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; 8 | 9 | /** A test POJO class. */ 10 | public class TestingUser implements Serializable { 11 | private static final long serialVersionUID = -1123545861004770003L; 12 | public String name; 13 | public Integer age; 14 | 15 | public String getName() { 16 | return name; 17 | } 18 | 19 | public void setName(String name) { 20 | this.name = name; 21 | } 22 | 23 | public Integer getAge() { 24 | return age; 25 | } 26 | 27 | public void setAge(Integer age) { 28 | this.age = age; 29 | } 30 | 31 | @Override 32 | public boolean equals(Object o) { 33 | if (this == o) { 34 | return true; 35 | } 36 | if (o == null || getClass() != o.getClass()) { 37 | return false; 38 | } 39 | TestingUser that = (TestingUser) o; 40 | return Objects.equals(name, that.name) && Objects.equals(age, that.age); 41 | } 42 | 43 | @Override 44 | public int hashCode() { 45 | return Objects.hash(name, age); 46 | } 47 | 48 | public static TestingUser createUser(String name, Integer age) { 49 | TestingUser user = new TestingUser(); 50 | user.setAge(age); 51 | user.setName(name); 52 | return user; 53 | } 54 | 55 | public static TestingUser createRandomUser() { 56 | TestingUser user = new TestingUser(); 57 | user.setName(randomAlphabetic(5)); 58 | user.setAge(ThreadLocalRandom.current().nextInt(0, Integer.MAX_VALUE)); 59 | return user; 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.testutils; 20 | 21 | import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; 22 | import org.apache.flink.connector.testframe.external.ExternalContext; 23 | 24 | import org.apache.pulsar.client.api.Schema; 25 | 26 | import java.net.URL; 27 | import java.util.ArrayList; 28 | import java.util.List; 29 | 30 | /** 31 | * The implementation for Flink connector test tools. Providing the common test case writing 32 | * constraint for both source, sink and table API. 33 | */ 34 | public abstract class PulsarTestContext implements ExternalContext { 35 | 36 | protected final PulsarRuntimeOperator operator; 37 | // The schema used for consuming and producing messages between Pulsar and tests. 38 | protected final Schema schema; 39 | 40 | private final List connectorJarPaths = new ArrayList<>(); 41 | 42 | protected PulsarTestContext(PulsarTestEnvironment environment, Schema schema) { 43 | this.operator = environment.operator(); 44 | this.schema = schema; 45 | } 46 | 47 | /** Implement this method for providing a more friendly test name in IDE. */ 48 | protected abstract String displayName(); 49 | 50 | @Override 51 | public String toString() { 52 | return displayName(); 53 | } 54 | 55 | @Override 56 | public List getConnectorJarPaths() { 57 | return connectorJarPaths; 58 | } 59 | 60 | public void addConnectorJarPaths(List urls) { 61 | this.connectorJarPaths.addAll(urls); 62 | } 63 | 64 | /** 65 | * This method is called after finishing executing a test case. Check the {@code 66 | * TestResourceProvidingInvocationContext.getAdditionalExtensions()} for detailed information. 67 | */ 68 | @Override 69 | public void close() throws Exception { 70 | // All the topics would be deleted in the PulsarRuntime. No need to manually close them. 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.testutils; 20 | 21 | import org.apache.flink.connector.pulsar.testutils.source.PulsarSourceTestContext; 22 | import org.apache.flink.connector.testframe.external.ExternalContextFactory; 23 | 24 | import java.util.function.Function; 25 | 26 | /** 27 | * Factory for creating all the test context that extends {@link PulsarSourceTestContext}. Test 28 | * context class should have a constructor with single {@link PulsarTestEnvironment} arg. 29 | */ 30 | public class PulsarTestContextFactory> 31 | implements ExternalContextFactory { 32 | 33 | private final PulsarTestEnvironment environment; 34 | private final Function contextFactory; 35 | 36 | public PulsarTestContextFactory( 37 | PulsarTestEnvironment environment, Function contextFactory) { 38 | this.environment = environment; 39 | this.contextFactory = contextFactory; 40 | } 41 | 42 | @Override 43 | public T createExternalContext(String testName) { 44 | return contextFactory.apply(environment); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.testutils; 20 | 21 | import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; 22 | import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; 23 | import org.apache.flink.util.TestLoggerExtension; 24 | 25 | import org.junit.jupiter.api.TestInstance; 26 | import org.junit.jupiter.api.extension.ExtendWith; 27 | import org.junit.jupiter.api.extension.RegisterExtension; 28 | 29 | import java.util.Collections; 30 | import java.util.Map; 31 | 32 | /** 33 | * The base class for the all Pulsar related test sites. It brings up: 34 | * 35 | *

    36 | *
  • A Pulsar Broker with memory based local metastore. 37 | *
  • A standalone Bookkeeper with memory based local metastore. 38 | *
39 | * 40 | *

You just need to write a JUnit 5 test class and extends this suite class. All the helper 41 | * method list below would be ready. 42 | * 43 | *

{@code PulsarSourceEnumeratorTest} would be a test example for how to use this base class. If 44 | * you have some setup logic, such as create topic or send message, just place then in a setup 45 | * method with annotation {@code @BeforeAll}. This setup method would not require {@code static}. 46 | * 47 | * @see PulsarRuntimeOperator for how to use the helper methods in this class. 48 | */ 49 | @ExtendWith(TestLoggerExtension.class) 50 | @TestInstance(TestInstance.Lifecycle.PER_CLASS) 51 | public abstract class PulsarTestSuiteBase { 52 | 53 | @RegisterExtension 54 | final PulsarTestEnvironment environment = 55 | new PulsarTestEnvironment(runtime().withConfigs(runtimeConfigs())); 56 | 57 | /** 58 | * Choose the desired pulsar runtime as the test backend. The default test backend is a 59 | * singleton pulsar instance. Override this method when needed. 60 | */ 61 | protected PulsarRuntime runtime() { 62 | return PulsarRuntime.singletonContainer(); 63 | } 64 | 65 | /** Operate pulsar by acquiring a runtime operator. */ 66 | protected PulsarRuntimeOperator operator() { 67 | return environment.operator(); 68 | } 69 | 70 | /** Override the default runtime configuration. */ 71 | protected Map runtimeConfigs() { 72 | return Collections.emptyMap(); 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/remote/PulsarRemoteRuntime.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.pulsar.testutils.runtime.remote; 2 | 3 | import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; 4 | import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; 5 | 6 | import java.util.Map; 7 | 8 | import static org.apache.flink.util.Preconditions.checkNotNull; 9 | import static org.testcontainers.containers.PulsarContainer.BROKER_HTTP_PORT; 10 | import static org.testcontainers.containers.PulsarContainer.BROKER_PORT; 11 | 12 | /** The runtime which will connect to a remote instance. It's always used for local debugging. */ 13 | public class PulsarRemoteRuntime implements PulsarRuntime { 14 | 15 | private final String serviceUrl; 16 | private final String adminUrl; 17 | 18 | private PulsarRuntimeOperator operator; 19 | 20 | public PulsarRemoteRuntime(String host) { 21 | this("pulsar://" + host + ":" + BROKER_PORT, "http://" + host + ":" + BROKER_HTTP_PORT); 22 | } 23 | 24 | public PulsarRemoteRuntime(String serviceUrl, String adminUrl) { 25 | this.serviceUrl = serviceUrl; 26 | this.adminUrl = adminUrl; 27 | } 28 | 29 | @Override 30 | public PulsarRuntime withConfigs(Map configs) { 31 | if (!configs.isEmpty()) { 32 | throw new UnsupportedOperationException( 33 | "We can't change the broker configs on a running instance."); 34 | } 35 | 36 | return this; 37 | } 38 | 39 | @Override 40 | public void startUp() throws Exception { 41 | this.operator = new PulsarRuntimeOperator(serviceUrl, adminUrl); 42 | } 43 | 44 | @Override 45 | public void tearDown() { 46 | // Nothing to do here. 47 | } 48 | 49 | @Override 50 | public PulsarRuntimeOperator operator() { 51 | return checkNotNull(operator, "You should start this pulsar runtime first."); 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/cases/AutoCreateTopicProducingContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.testutils.sink.cases; 20 | 21 | import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; 22 | import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestContext; 23 | 24 | /** 25 | * Sink the messages into a non-existed topic and test the connector could auto create it. This test 26 | * case only gets passed when the {@code allowAutoTopicCreation} is enabled on the Pulsar runtime. 27 | */ 28 | public class AutoCreateTopicProducingContext extends PulsarSinkTestContext { 29 | 30 | public AutoCreateTopicProducingContext(PulsarTestEnvironment environment) { 31 | super(environment); 32 | } 33 | 34 | @Override 35 | protected boolean creatTopic() { 36 | return false; 37 | } 38 | 39 | @Override 40 | protected String displayName() { 41 | return "write messages into a non-existed topic in Pulsar"; 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/cases/EncryptedMessageProducingContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.testutils.sink.cases; 20 | 21 | import org.apache.flink.connector.pulsar.common.crypto.PulsarCrypto; 22 | import org.apache.flink.connector.pulsar.sink.PulsarSinkBuilder; 23 | import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; 24 | import org.apache.flink.connector.pulsar.testutils.PulsarTestKeyReader; 25 | import org.apache.flink.connector.pulsar.testutils.PulsarTestKeyReader.MessageCryptoBcSupplier; 26 | import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestContext; 27 | import org.apache.flink.connector.pulsar.testutils.sink.reader.PulsarEncryptDataReader; 28 | import org.apache.flink.connector.pulsar.testutils.sink.reader.PulsarPartitionDataReader; 29 | 30 | import java.util.List; 31 | 32 | import static org.apache.flink.connector.pulsar.testutils.PulsarTestKeyReader.ENCRYPT_KEY; 33 | import static org.apache.pulsar.client.api.ProducerCryptoFailureAction.FAIL; 34 | import static org.apache.pulsar.client.api.Schema.STRING; 35 | 36 | /** Sink the encrypted messages into a topic and try to decrypt them. */ 37 | public class EncryptedMessageProducingContext extends PulsarSinkTestContext { 38 | 39 | public EncryptedMessageProducingContext(PulsarTestEnvironment environment) { 40 | super(environment); 41 | } 42 | 43 | @Override 44 | protected void setSinkBuilder(PulsarSinkBuilder builder) { 45 | PulsarCrypto pulsarCrypto = 46 | PulsarCrypto.builder() 47 | .cryptoKeyReader(new PulsarTestKeyReader()) 48 | .addEncryptKeys(ENCRYPT_KEY) 49 | .messageCrypto(new MessageCryptoBcSupplier(true)) 50 | .build(); 51 | builder.setPulsarCrypto(pulsarCrypto, FAIL); 52 | } 53 | 54 | @Override 55 | protected PulsarPartitionDataReader createSinkDataReader(List topics) { 56 | PulsarCrypto pulsarCrypto = 57 | PulsarCrypto.builder() 58 | .cryptoKeyReader(new PulsarTestKeyReader()) 59 | .addEncryptKeys(ENCRYPT_KEY) 60 | .messageCrypto(new MessageCryptoBcSupplier(false)) 61 | .build(); 62 | return new PulsarEncryptDataReader<>(operator, topics, STRING, pulsarCrypto); 63 | } 64 | 65 | @Override 66 | protected String displayName() { 67 | return "write messages into one topic by end-to-end encryption"; 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/cases/MultipleTopicsProducingContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.testutils.sink.cases; 20 | 21 | import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; 22 | import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestContext; 23 | 24 | import java.util.List; 25 | import java.util.stream.Stream; 26 | 27 | import static java.util.stream.Collectors.toList; 28 | import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; 29 | 30 | /** Sink the message into multiple topics and test all the messages have been written. */ 31 | public class MultipleTopicsProducingContext extends PulsarSinkTestContext { 32 | 33 | private static final String TOPIC_NAME_PREFIX = "sink-multiple-topic-"; 34 | 35 | public MultipleTopicsProducingContext(PulsarTestEnvironment environment) { 36 | super(environment); 37 | } 38 | 39 | @Override 40 | protected String displayName() { 41 | return "write messages into multiple topics in Pulsar"; 42 | } 43 | 44 | @Override 45 | protected List generateTopics() { 46 | return Stream.generate(() -> TOPIC_NAME_PREFIX + randomAlphanumeric(10)) 47 | .limit(4) 48 | .collect(toList()); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/cases/SingleTopicProducingContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.testutils.sink.cases; 20 | 21 | import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; 22 | import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestContext; 23 | 24 | /** Sink the message into a topic and test all the messages have been written. */ 25 | public class SingleTopicProducingContext extends PulsarSinkTestContext { 26 | 27 | public SingleTopicProducingContext(PulsarTestEnvironment environment) { 28 | super(environment); 29 | } 30 | 31 | @Override 32 | protected String displayName() { 33 | return "write messages into one topic in Pulsar"; 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/reader/PulsarEncryptDataReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.testutils.sink.reader; 20 | 21 | import org.apache.flink.connector.pulsar.common.crypto.PulsarCrypto; 22 | import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; 23 | 24 | import org.apache.pulsar.client.api.Schema; 25 | 26 | import java.util.List; 27 | 28 | /** The data reader for reading encrypted messages from Pulsar. */ 29 | public class PulsarEncryptDataReader extends PulsarPartitionDataReader { 30 | 31 | public PulsarEncryptDataReader( 32 | PulsarRuntimeOperator operator, 33 | List topics, 34 | Schema schema, 35 | PulsarCrypto pulsarCrypto) { 36 | super(operator, topics, schema, pulsarCrypto); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicsConsumingContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.testutils.source.cases; 20 | 21 | import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; 22 | import org.apache.flink.connector.pulsar.testutils.source.PulsarSourceTestContext; 23 | import org.apache.flink.util.FlinkRuntimeException; 24 | 25 | import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; 26 | import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; 27 | 28 | /** 29 | * Pulsar external context that will create multiple topics with only one partitions as source 30 | * splits. 31 | */ 32 | public class MultipleTopicsConsumingContext extends PulsarSourceTestContext { 33 | 34 | private final String topicPrefix = 35 | "public/default/flink-multiple-topic-" + randomAlphabetic(8) + "-"; 36 | 37 | private int index = 0; 38 | 39 | public MultipleTopicsConsumingContext(PulsarTestEnvironment environment) { 40 | super(environment); 41 | } 42 | 43 | @Override 44 | protected String displayName() { 45 | return "consume message on multiple topic"; 46 | } 47 | 48 | @Override 49 | protected String topicPattern() { 50 | return topicPrefix + "\\d+"; 51 | } 52 | 53 | @Override 54 | protected String subscriptionName() { 55 | return "flink-multiple-topic-test"; 56 | } 57 | 58 | @Override 59 | protected String generatePartitionName() { 60 | String topic = topicPrefix + index; 61 | try { 62 | operator.createTopic(topic, 1); 63 | } catch (Exception e) { 64 | throw new FlinkRuntimeException(e); 65 | } 66 | index++; 67 | 68 | return topicNameWithPartition(topic, 0); 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SingleTopicConsumingContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.testutils.source.cases; 20 | 21 | import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; 22 | import org.apache.flink.connector.pulsar.testutils.source.PulsarSourceTestContext; 23 | import org.apache.flink.util.FlinkRuntimeException; 24 | 25 | import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; 26 | import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; 27 | 28 | /** 29 | * A Pulsar external context that will create only one topic and use partitions in that topic as the 30 | * source splits. 31 | */ 32 | public class SingleTopicConsumingContext extends PulsarSourceTestContext { 33 | 34 | private final String topicName = "public/default/pulsar-single-topic-" + randomAlphanumeric(8); 35 | 36 | private int index = 0; 37 | 38 | public SingleTopicConsumingContext(PulsarTestEnvironment environment) { 39 | super(environment); 40 | } 41 | 42 | @Override 43 | protected String displayName() { 44 | return "consume message on single topic"; 45 | } 46 | 47 | @Override 48 | protected String topicPattern() { 49 | return topicName; 50 | } 51 | 52 | @Override 53 | protected String subscriptionName() { 54 | return "pulsar-single-topic-test"; 55 | } 56 | 57 | @Override 58 | protected String generatePartitionName() { 59 | try { 60 | if (index == 0) { 61 | operator.createTopic(topicName, index + 1); 62 | } else { 63 | operator.increaseTopicPartitions(topicName, index + 1); 64 | } 65 | } catch (Exception e) { 66 | throw new FlinkRuntimeException(e); 67 | } 68 | 69 | return topicNameWithPartition(topicName, index++); 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/writer/KeyedPulsarPartitionDataWriter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.testutils.source.writer; 20 | 21 | import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; 22 | import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; 23 | import org.apache.flink.util.FlinkRuntimeException; 24 | 25 | import org.apache.pulsar.client.api.Schema; 26 | 27 | import java.util.List; 28 | 29 | import static java.util.stream.Collectors.toList; 30 | 31 | /** 32 | * Source split data writer for writing test data into a Pulsar topic partition. It will write the 33 | * message with two keys. 34 | */ 35 | public class KeyedPulsarPartitionDataWriter implements ExternalSystemSplitDataWriter { 36 | 37 | private final PulsarRuntimeOperator operator; 38 | private final String fullTopicName; 39 | private final String keyToRead; 40 | private final String keyToExclude; 41 | 42 | public KeyedPulsarPartitionDataWriter( 43 | PulsarRuntimeOperator operator, 44 | String fullTopicName, 45 | String keyToRead, 46 | String keyToExclude) { 47 | this.operator = operator; 48 | this.fullTopicName = fullTopicName; 49 | this.keyToRead = keyToRead; 50 | this.keyToExclude = keyToExclude; 51 | } 52 | 53 | @Override 54 | public void writeRecords(List records) { 55 | try { 56 | // Send messages with the key we don't need. 57 | List newRecords = records.stream().map(a -> a + keyToRead).collect(toList()); 58 | operator.sendMessages(fullTopicName, Schema.STRING, keyToExclude, newRecords, false); 59 | 60 | // Send messages with the given key. 61 | operator.sendMessages(fullTopicName, Schema.STRING, keyToRead, records, false); 62 | } catch (Exception e) { 63 | throw new FlinkRuntimeException(e); 64 | } 65 | } 66 | 67 | @Override 68 | public void close() { 69 | // Nothing to do. 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/writer/PulsarPartitionDataWriter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.pulsar.testutils.source.writer; 20 | 21 | import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; 22 | import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; 23 | import org.apache.flink.util.FlinkRuntimeException; 24 | 25 | import org.apache.pulsar.client.api.Schema; 26 | 27 | import java.util.List; 28 | 29 | /** 30 | * Source split data writer for writing test data into a Pulsar topic partition. This writer doesn't 31 | * need to be closed. 32 | */ 33 | public class PulsarPartitionDataWriter implements ExternalSystemSplitDataWriter { 34 | 35 | private final PulsarRuntimeOperator operator; 36 | private final String fullTopicName; 37 | private final Schema schema; 38 | 39 | public PulsarPartitionDataWriter( 40 | PulsarRuntimeOperator operator, String fullTopicName, Schema schema) { 41 | this.operator = operator; 42 | this.fullTopicName = fullTopicName; 43 | this.schema = schema; 44 | } 45 | 46 | @Override 47 | public void writeRecords(List records) { 48 | try { 49 | operator.sendMessages(fullTopicName, schema, records, false); 50 | } catch (Exception e) { 51 | throw new FlinkRuntimeException(e); 52 | } 53 | } 54 | 55 | @Override 56 | public void close() { 57 | // Nothing to do. 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/resources/archunit.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | # 18 | 19 | # By default we allow removing existing violations, but fail when new violations are added. 20 | freeze.store.default.allowStoreUpdate=true 21 | 22 | # Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations. 23 | #freeze.store.default.allowStoreCreation=true 24 | 25 | # Enable this to add allow new violations to be recorded. 26 | # NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new 27 | # violation, please try to avoid creating the violation. If the violation was created due to a 28 | # shortcoming of the rule, file a JIRA issue so the rule can be improved. 29 | #freeze.refreeze=true 30 | #freeze.store.default.allowStoreCreation=true 31 | 32 | freeze.store.default.path=archunit-violations 33 | 34 | archRule.failOnEmptyShould=false 35 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level=INFO 22 | rootLogger.appenderRef.test.ref=TestLogger 23 | appender.testlogger.name=TestLogger 24 | appender.testlogger.type=CONSOLE 25 | appender.testlogger.target=SYSTEM_ERR 26 | appender.testlogger.layout.type=PatternLayout 27 | appender.testlogger.layout.pattern=%-4r [%t] %-5p %c %x - %m%n 28 | 29 | # Logger for pulsar. 30 | logger.pulsar.name=org.apache.pulsar 31 | logger.pulsar.level=INFO 32 | 33 | # Logger for pulsar connector. 34 | logger.pulsar.connector.name=org.apache.flink.connector.pulsar 35 | logger.pulsar.connector.level=INFO 36 | -------------------------------------------------------------------------------- /flink-connector-pulsar/src/test/resources/protobuf/sample_message.proto: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | syntax = "proto3"; 20 | 21 | package org.apache.flink.connector.pulsar; 22 | 23 | enum TestEnum { 24 | SHARED = 0; 25 | FAILOVER = 1; 26 | } 27 | 28 | message SubMessage { 29 | string foo = 1; 30 | double bar = 2; 31 | message NestedMessage { 32 | string url = 1; 33 | string title = 2; 34 | repeated string snippets = 3; 35 | } 36 | } 37 | 38 | message TestMessage { 39 | string stringField = 1; 40 | double doubleField = 2; 41 | int32 intField = 6; 42 | TestEnum testEnum = 4; 43 | SubMessage nestedField = 5; 44 | repeated string repeatedField = 10; 45 | } 46 | -------------------------------------------------------------------------------- /flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE: -------------------------------------------------------------------------------- 1 | flink-sql-connector-pulsar 2 | Copyright 2014-2024 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | 7 | This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) 8 | 9 | - com.fasterxml.jackson.core:jackson-annotations:2.13.4 10 | - org.apache.pulsar:pulsar-client-admin-api:3.0.2 11 | - org.apache.pulsar:pulsar-client-all:3.0.2 12 | - org.apache.pulsar:pulsar-client-api:3.0.2 13 | 14 | This project bundles the following dependencies under the Bouncy Castle license. 15 | See bundled license files for details. 16 | 17 | - org.bouncycastle:bcpkix-jdk15on:1.69 18 | - org.bouncycastle:bcprov-ext-jdk15on:1.69 19 | - org.bouncycastle:bcprov-jdk15on:1.69 20 | - org.bouncycastle:bcutil-jdk15on:1.69 21 | -------------------------------------------------------------------------------- /flink-sql-connector-pulsar/src/main/resources/META-INF/licences/LICENSE.bouncycastle: -------------------------------------------------------------------------------- 1 | Copyright (c) 2000 - 2021 The Legion of the Bouncy Castle Inc. (https://www.bouncycastle.org) 2 | 3 | Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: 4 | 5 | The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. 6 | 7 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. -------------------------------------------------------------------------------- /tools/ci/log4j.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | rootLogger.level = INFO 20 | rootLogger.appenderRef.out.ref = ConsoleAppender 21 | 22 | # ----------------------------------------------------------------------------- 23 | # Console (use 'console') 24 | # ----------------------------------------------------------------------------- 25 | 26 | appender.console.name = ConsoleAppender 27 | appender.console.type = CONSOLE 28 | appender.console.layout.type = PatternLayout 29 | appender.console.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n 30 | 31 | # ----------------------------------------------------------------------------- 32 | # File (use 'file') 33 | # ----------------------------------------------------------------------------- 34 | appender.file.name = FileAppender 35 | appender.file.type = FILE 36 | appender.file.fileName = ${sys:log.dir}/mvn-${sys:mvn.forkNumber:-output}.log 37 | appender.file.layout.type = PatternLayout 38 | appender.file.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n 39 | appender.file.createOnDemand = true 40 | 41 | # suppress the irrelevant (wrong) warnings from the netty channel handler 42 | logger.netty.name = org.jboss.netty.channel.DefaultChannelPipeline 43 | logger.netty.level = ERROR 44 | -------------------------------------------------------------------------------- /tools/maven/suppressions.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 24 | 25 | 26 | 27 | 30 | 31 | --------------------------------------------------------------------------------