├── NOTICE ├── .gitignore ├── src ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── flink │ │ │ └── connector │ │ │ └── rocketmq │ │ │ ├── source │ │ │ ├── enumerator │ │ │ │ ├── offset │ │ │ │ │ ├── OffsetsValidator.java │ │ │ │ │ ├── OffsetsSelectorNoStopping.java │ │ │ │ │ ├── OffsetsSelectorByStrategy.java │ │ │ │ │ ├── OffsetsSelectorByTimestamp.java │ │ │ │ │ └── OffsetsSelectorBySpecified.java │ │ │ │ ├── RocketMQSourceEnumState.java │ │ │ │ ├── allocate │ │ │ │ │ ├── AverageAllocateStrategy.java │ │ │ │ │ ├── BroadcastAllocateStrategy.java │ │ │ │ │ ├── ConsistentHashAllocateStrategy.java │ │ │ │ │ ├── AllocateStrategy.java │ │ │ │ │ └── AllocateStrategyFactory.java │ │ │ │ └── RocketMQSourceEnumStateSerializer.java │ │ │ ├── reader │ │ │ │ ├── deserializer │ │ │ │ │ ├── DirtyDataStrategy.java │ │ │ │ │ ├── RocketMQSchemaWrapper.java │ │ │ │ │ ├── BytesMessage.java │ │ │ │ │ ├── RocketMQDeserializationSchemaWrapper.java │ │ │ │ │ ├── QueryableSchema.java │ │ │ │ │ └── RocketMQDeserializationSchema.java │ │ │ │ ├── RocketMQRecordEmitter.java │ │ │ │ └── MessageView.java │ │ │ ├── util │ │ │ │ └── UtilAll.java │ │ │ ├── config │ │ │ │ └── OffsetVerification.java │ │ │ ├── split │ │ │ │ ├── RocketMQSourceSplitState.java │ │ │ │ └── RocketMQPartitionSplitSerializer.java │ │ │ └── metrics │ │ │ │ └── RocketMQSourceReaderMetrics.java │ │ │ ├── MetricUtil.java │ │ │ ├── legacy │ │ │ ├── common │ │ │ │ ├── config │ │ │ │ │ ├── StartupMode.java │ │ │ │ │ └── OffsetResetStrategy.java │ │ │ │ ├── selector │ │ │ │ │ ├── TopicSelector.java │ │ │ │ │ ├── MessageQueueSelector.java │ │ │ │ │ ├── HashMessageQueueSelector.java │ │ │ │ │ ├── RandomMessageQueueSelector.java │ │ │ │ │ ├── DefaultTopicSelector.java │ │ │ │ │ └── SimpleTopicSelector.java │ │ │ │ ├── serialization │ │ │ │ │ ├── KeyValueSerializationSchema.java │ │ │ │ │ ├── KeyValueDeserializationSchema.java │ │ │ │ │ ├── ForwardMessageExtDeserialization.java │ │ │ │ │ ├── MessageExtDeserializationScheme.java │ │ │ │ │ ├── SimpleStringDeserializationSchema.java │ │ │ │ │ ├── SimpleTupleDeserializationSchema.java │ │ │ │ │ ├── SimpleKeyValueSerializationSchema.java │ │ │ │ │ └── SimpleKeyValueDeserializationSchema.java │ │ │ │ ├── watermark │ │ │ │ │ ├── WaterMarkForAll.java │ │ │ │ │ ├── TimeLagWatermarkGenerator.java │ │ │ │ │ ├── BoundedOutOfOrdernessGenerator.java │ │ │ │ │ ├── PunctuatedAssigner.java │ │ │ │ │ ├── WaterMarkPerQueue.java │ │ │ │ │ └── BoundedOutOfOrdernessGeneratorPerQueue.java │ │ │ │ └── util │ │ │ │ │ ├── TestUtils.java │ │ │ │ │ ├── RetryUtil.java │ │ │ │ │ └── RocketMQUtils.java │ │ │ ├── RunningChecker.java │ │ │ └── function │ │ │ │ ├── SourceMapFunction.java │ │ │ │ └── SinkMapFunction.java │ │ │ ├── common │ │ │ ├── constant │ │ │ │ ├── SchemaRegistryConstant.java │ │ │ │ └── RocketMqCatalogConstant.java │ │ │ ├── lock │ │ │ │ └── SpinLock.java │ │ │ ├── event │ │ │ │ ├── SourceDetectEvent.java │ │ │ │ ├── SourceInitAssignEvent.java │ │ │ │ ├── SourceCheckEvent.java │ │ │ │ └── SourceReportOffsetEvent.java │ │ │ └── config │ │ │ │ └── RocketMQConfiguration.java │ │ │ ├── sink │ │ │ ├── writer │ │ │ │ ├── serializer │ │ │ │ │ ├── RocketMQSerializerWrapper.java │ │ │ │ │ └── RocketMQSerializationSchema.java │ │ │ │ └── context │ │ │ │ │ ├── RocketMQSinkContext.java │ │ │ │ │ └── RocketMQSinkContextImpl.java │ │ │ ├── TransactionResult.java │ │ │ ├── table │ │ │ │ └── RocketMQRowDataSink.java │ │ │ ├── committer │ │ │ │ └── SendCommittableSerializer.java │ │ │ ├── RocketMQSink.java │ │ │ └── InnerProducer.java │ │ │ └── catalog │ │ │ ├── RocketMQCatalogFactoryOptions.java │ │ │ └── RocketMQCatalogFactory.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.flink.table.factories.Factory └── test │ ├── resources │ └── logback-test.xml │ └── java │ └── org │ └── apache │ └── flink │ └── connector │ └── rocketmq │ ├── legacy │ ├── common │ │ ├── selector │ │ │ ├── DefaultTopicSelectorTest.java │ │ │ ├── SimpleTopicSelectorTest.java │ │ │ ├── HashMessageQueueSelectorTest.java │ │ │ └── RandomMessageQueueSelectorTest.java │ │ ├── serialization │ │ │ ├── SimpleKeyValueSerializationSchemaTest.java │ │ │ └── RowKeyValueDeserializationSchemaTest.java │ │ └── util │ │ │ └── RetryUtilTest.java │ └── RocketMQSinkTest.java │ ├── source │ ├── enumerator │ │ ├── allocate │ │ │ ├── ConsistentHashAllocateStrategyTest.java │ │ │ └── BroadcastAllocateStrategyTest.java │ │ └── RocketMQSourceEnumStateSerializerTest.java │ └── reader │ │ └── RocketMQRecordEmitterTest.java │ ├── example │ ├── ConnectorConfig.java │ ├── SimpleConsumer.java │ └── SimpleProducer.java │ ├── common │ └── config │ │ └── RocketMQConfigBuilderTest.java │ └── catalog │ └── RocketMQCatalogFactoryTest.java ├── .asf.yaml ├── .travis.yml └── style └── copyright ├── Apache.xml └── profiles_settings.xml /NOTICE: -------------------------------------------------------------------------------- 1 | Apache RocketMQ 2 | Copyright 2016-2021 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Compiled class file 2 | *.class 3 | 4 | # Log file 5 | *.log 6 | 7 | # Package Files # 8 | *.jar 9 | *.war 10 | *.nar 11 | *.ear 12 | *.zip 13 | *.tar.gz 14 | *.rar 15 | 16 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml 17 | hs_err_pid* 18 | 19 | *.iml 20 | 21 | .idea/ 22 | 23 | target/ 24 | 25 | .DS_Store 26 | nohup.out 27 | 28 | # VSCode 29 | *.project 30 | *.settings/ 31 | *.classpath 32 | *.factorypath 33 | .vscode/ 34 | 35 | # Ignore it while not disable for some reason. 36 | dependency-reduced-pom.xml 37 | /logs/output.log.* -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsValidator.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.rocketmq.source.enumerator.offset; 2 | 3 | import org.apache.flink.annotation.Internal; 4 | 5 | import java.util.Properties; 6 | 7 | @Internal 8 | public interface OffsetsValidator { 9 | 10 | /** 11 | * Validate offsets initializer with properties of RocketMQ source. 12 | * 13 | * @param properties Properties of RocketMQ source 14 | * @throws IllegalStateException if validation fails 15 | */ 16 | void validate(Properties properties) throws IllegalStateException; 17 | } 18 | -------------------------------------------------------------------------------- /.asf.yaml: -------------------------------------------------------------------------------- 1 | github: 2 | description: "RocketMQ integration for Apache Flink. This module includes the RocketMQ source and sink that allows a flink job to either write messages into a topic or read from topics in a flink job." 3 | homepage: https://rocketmq.apache.org/ 4 | labels: 5 | - streaming 6 | - cloud-native 7 | - rocketmq 8 | features: 9 | # Enable issue management 10 | issues: true 11 | # Enable wiki 12 | wiki: true 13 | protected_branches: 14 | 15 | notifications: 16 | commits: commits@rocketmq.apache.org 17 | issues: commits@rocketmq.apache.org 18 | pullrequests: commits@rocketmq.apache.org 19 | jobs: commits@rocketmq.apache.org 20 | discussions: dev@rocketmq.apache.org 21 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | dist: trusty 2 | 3 | notifications: 4 | email: 5 | recipients: 6 | - dev@rocketmq.apache.org 7 | on_success: change 8 | on_failure: always 9 | 10 | language: java 11 | 12 | jdk: 13 | - oraclejdk8 14 | - oraclejdk9 15 | - oraclejdk11 16 | - openjdk8 17 | 18 | # before_install: 19 | # - echo 'MAVEN_OPTS="$MAVEN_OPTS -Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal"' >> ~/.mavenrc 20 | # - cat ~/.mavenrc 21 | # - if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi 22 | # - if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "$CUSTOM_JDK"; fi 23 | 24 | script: 25 | - travis_retry mvn -B clean apache-rat:check -debug 26 | - travis_retry mvn -B clean install jacoco:report coveralls:report 27 | 28 | after_success: 29 | - bash <(curl -s https://codecov.io/bash) || echo 'Codecov failed to upload' 30 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/MetricUtil.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.rocketmq; 19 | 20 | public class MetricUtil {} 21 | -------------------------------------------------------------------------------- /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.rocketmq.source.table.RocketMQDynamicTableSourceFactory 17 | org.apache.flink.connector.rocketmq.sink.table.RocketMQDynamicTableSinkFactory 18 | org.apache.flink.connector.rocketmq.catalog.RocketMQCatalogFactory 19 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/config/StartupMode.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.rocketmq.legacy.common.config; 19 | 20 | /** RocketMQ startup mode. */ 21 | public enum StartupMode { 22 | EARLIEST, 23 | LATEST, 24 | GROUP_OFFSETS, 25 | TIMESTAMP, 26 | SPECIFIC_OFFSETS 27 | } 28 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/TopicSelector.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 | package org.apache.flink.connector.rocketmq.legacy.common.selector; 18 | 19 | import java.io.Serializable; 20 | 21 | public interface TopicSelector extends Serializable { 22 | 23 | String getTopic(T tuple); 24 | 25 | String getTag(T tuple); 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/MessageQueueSelector.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.rocketmq.legacy.common.selector; 20 | 21 | import java.io.Serializable; 22 | 23 | public interface MessageQueueSelector 24 | extends org.apache.rocketmq.client.producer.MessageQueueSelector, Serializable {} 25 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/DirtyDataStrategy.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.rocketmq.source.reader.deserializer; 19 | 20 | /** Dirty data process strategy. */ 21 | public enum DirtyDataStrategy { 22 | SKIP, 23 | SKIP_SILENT, 24 | CUT, 25 | PAD, 26 | NULL, 27 | EXCEPTION 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/KeyValueSerializationSchema.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 | package org.apache.flink.connector.rocketmq.legacy.common.serialization; 18 | 19 | import java.io.Serializable; 20 | 21 | public interface KeyValueSerializationSchema extends Serializable { 22 | 23 | byte[] serializeKey(T tuple); 24 | 25 | byte[] serializeValue(T tuple); 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/common/constant/SchemaRegistryConstant.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.rocketmq.common.constant; 20 | 21 | /** SchemaRegistryConstant. */ 22 | public class SchemaRegistryConstant { 23 | 24 | public static final String SCHEMA_REGISTRY_BASE_URL = "http://localhost:8080"; 25 | 26 | public static final String SCHEMA_REGISTRY_BASE_URL_KEY = "schema.registry.base.url"; 27 | } 28 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/sink/writer/serializer/RocketMQSerializerWrapper.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.rocketmq.sink.writer.serializer; 19 | 20 | import org.apache.flink.annotation.Internal; 21 | 22 | /** 23 | * Wrap the RocketMQ Schema into RocketMQSerializationSchema. We support schema evolution out of box 24 | * by this implementation. 25 | */ 26 | @Internal 27 | public class RocketMQSerializerWrapper {} 28 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/KeyValueDeserializationSchema.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 | package org.apache.flink.connector.rocketmq.legacy.common.serialization; 18 | 19 | import org.apache.flink.api.java.typeutils.ResultTypeQueryable; 20 | 21 | import java.io.Serializable; 22 | 23 | public interface KeyValueDeserializationSchema extends ResultTypeQueryable, Serializable { 24 | T deserializeKeyAndValue(byte[] key, byte[] value); 25 | } 26 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/RunningChecker.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 | package org.apache.flink.connector.rocketmq.legacy; 18 | 19 | import java.io.Serializable; 20 | 21 | public class RunningChecker implements Serializable { 22 | private volatile boolean isRunning = false; 23 | 24 | public boolean isRunning() { 25 | return isRunning; 26 | } 27 | 28 | public void setRunning(boolean running) { 29 | isRunning = running; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/config/OffsetResetStrategy.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.rocketmq.legacy.common.config; 19 | 20 | /** Config for #{@link StartupMode#GROUP_OFFSETS}. */ 21 | public enum OffsetResetStrategy { 22 | /** If group offsets is not found,the latest offset would be set to start consumer */ 23 | LATEST, 24 | 25 | /** If group offsets is not found,the earliest offset would be set to start consumer */ 26 | EARLIEST 27 | } 28 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/common/constant/RocketMqCatalogConstant.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.rocketmq.common.constant; 20 | 21 | /** RocketMqCatalogConstant. */ 22 | public class RocketMqCatalogConstant { 23 | public static final String CONNECTOR = "connector"; 24 | 25 | public static final String TOPIC = "topic"; 26 | public static final String NAME_SERVER_ADDRESS = "nameServerAddress"; 27 | public static final String ROCKETMQ_CONNECTOR = "rocketmq"; 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/common/lock/SpinLock.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.rocketmq.common.lock; 19 | 20 | import java.util.concurrent.atomic.AtomicBoolean; 21 | 22 | public class SpinLock { 23 | private AtomicBoolean lock = new AtomicBoolean(false); 24 | 25 | public void lock() { 26 | boolean lock = false; 27 | do { 28 | lock = this.lock.compareAndSet(false, true); 29 | } while (!lock); 30 | } 31 | 32 | public void unlock() { 33 | this.lock.set(false); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/sink/TransactionResult.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.rocketmq.sink; 19 | 20 | public enum TransactionResult { 21 | 22 | /** Notify server that current transaction should be committed. */ 23 | COMMIT, 24 | 25 | /** Notify server that current transaction should be roll-backed. */ 26 | ROLLBACK, 27 | 28 | /** 29 | * Notify the server that the state of this transaction is not sure. You should be cautious 30 | * before return unknown because the examination from the server will be performed periodically. 31 | */ 32 | UNKNOWN 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/common/event/SourceDetectEvent.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.rocketmq.common.event; 20 | 21 | import org.apache.flink.api.connector.source.SourceEvent; 22 | 23 | public class SourceDetectEvent implements SourceEvent { 24 | // Request to resend the initial allocation result 25 | private boolean reSendInitAssign = true; 26 | 27 | public boolean getReSendInitAssign() { 28 | return reSendInitAssign; 29 | } 30 | 31 | public void setReSendInitAssign(boolean reSendInitAssign) { 32 | this.reSendInitAssign = reSendInitAssign; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/common/event/SourceInitAssignEvent.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.rocketmq.common.event; 20 | 21 | import org.apache.flink.api.connector.source.SourceEvent; 22 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit; 23 | 24 | import java.util.List; 25 | 26 | public class SourceInitAssignEvent implements SourceEvent { 27 | private List splits; 28 | 29 | public void setSplits(List splits) { 30 | this.splits = splits; 31 | } 32 | 33 | public List getSplits() { 34 | return splits; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/HashMessageQueueSelector.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.rocketmq.legacy.common.selector; 20 | 21 | import org.apache.rocketmq.common.message.Message; 22 | import org.apache.rocketmq.common.message.MessageQueue; 23 | 24 | import java.util.List; 25 | 26 | public class HashMessageQueueSelector implements MessageQueueSelector { 27 | @Override 28 | public MessageQueue select(List mqs, Message msg, Object arg) { 29 | int value = arg.hashCode() % mqs.size(); 30 | if (value < 0) { 31 | value = Math.abs(value); 32 | } 33 | return mqs.get(value); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/function/SourceMapFunction.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.rocketmq.legacy.function; 19 | 20 | import org.apache.flink.api.java.tuple.Tuple2; 21 | import org.apache.flink.streaming.api.functions.ProcessFunction; 22 | import org.apache.flink.util.Collector; 23 | 24 | public class SourceMapFunction 25 | extends ProcessFunction, Tuple2> { 26 | 27 | @Override 28 | public void processElement( 29 | Tuple2 value, Context ctx, Collector> out) 30 | throws Exception { 31 | out.collect(new Tuple2<>(value.f0, value.f1)); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQSchemaWrapper.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.rocketmq.source.reader.deserializer; 19 | 20 | import org.apache.flink.api.common.typeinfo.TypeInformation; 21 | import org.apache.flink.connector.rocketmq.source.reader.MessageView; 22 | import org.apache.flink.util.Collector; 23 | 24 | import java.io.IOException; 25 | 26 | public class RocketMQSchemaWrapper implements RocketMQDeserializationSchema { 27 | @Override 28 | public void deserialize(MessageView messageView, Collector out) throws IOException {} 29 | 30 | @Override 31 | public TypeInformation getProducedType() { 32 | return null; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/RandomMessageQueueSelector.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.rocketmq.legacy.common.selector; 20 | 21 | import org.apache.rocketmq.common.message.Message; 22 | import org.apache.rocketmq.common.message.MessageQueue; 23 | 24 | import java.util.List; 25 | import java.util.Random; 26 | 27 | public class RandomMessageQueueSelector implements MessageQueueSelector { 28 | private Random random = new Random(System.currentTimeMillis()); 29 | 30 | @Override 31 | public MessageQueue select(List mqs, Message msg, Object arg) { 32 | int value = random.nextInt(mqs.size()); 33 | return mqs.get(value); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /src/test/resources/logback-test.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | 8 | 9 | ${SIMPLE_LOG_PATTERN} 10 | utf-8 11 | 12 | 13 | 14 | 15 | 16 | ${SIMPLE_LOG_PATTERN} 17 | utf-8 18 | 19 | 20 | ${user.home}${file.separator}logs${file.separator}flink-connector-rocketmq${file.separator}connector.log 21 | 22 | true 23 | 24 | 25 | ${user.home}${file.separator}logs${file.separator}flink-connector-rocketmq${file.separator}others${file.separator}connector_log.%i.log.gz 26 | 27 | 28 | 29 | 100MB 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/common/event/SourceCheckEvent.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.rocketmq.common.event; 20 | 21 | import org.apache.flink.api.connector.source.SourceEvent; 22 | import org.apache.flink.api.java.tuple.Tuple2; 23 | 24 | import org.apache.rocketmq.common.message.MessageQueue; 25 | 26 | import java.util.Map; 27 | 28 | public class SourceCheckEvent implements SourceEvent { 29 | private Map> assignedMq; 30 | 31 | public Map> getAssignedMq() { 32 | return assignedMq; 33 | } 34 | 35 | public void setAssignedMq(Map> assignedMq) { 36 | this.assignedMq = assignedMq; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/DefaultTopicSelectorTest.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.rocketmq.legacy.common.selector; 20 | 21 | import org.junit.Test; 22 | 23 | import static org.junit.Assert.assertEquals; 24 | 25 | public class DefaultTopicSelectorTest { 26 | @Test 27 | public void getTopic() throws Exception { 28 | DefaultTopicSelector selector = new DefaultTopicSelector("rocket"); 29 | assertEquals("rocket", selector.getTopic(null)); 30 | assertEquals("", selector.getTag(null)); 31 | 32 | selector = new DefaultTopicSelector("rocket", "tg"); 33 | assertEquals("rocket", selector.getTopic(null)); 34 | assertEquals("tg", selector.getTag(null)); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/ForwardMessageExtDeserialization.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.rocketmq.legacy.common.serialization; 19 | 20 | import org.apache.flink.api.common.typeinfo.TypeInformation; 21 | 22 | import org.apache.rocketmq.common.message.MessageExt; 23 | 24 | /** A Forward messageExt deserialization. */ 25 | public class ForwardMessageExtDeserialization 26 | implements MessageExtDeserializationScheme { 27 | 28 | @Override 29 | public MessageExt deserializeMessageExt(MessageExt messageExt) { 30 | return messageExt; 31 | } 32 | 33 | @Override 34 | public TypeInformation getProducedType() { 35 | return TypeInformation.of(MessageExt.class); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumState.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.rocketmq.source.enumerator; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | 23 | import org.apache.rocketmq.common.message.MessageQueue; 24 | 25 | import java.util.Set; 26 | 27 | /** The state of RocketMQ source enumerator. */ 28 | @Internal 29 | public class RocketMQSourceEnumState { 30 | 31 | private final Set currentSplitAssignment; 32 | 33 | public RocketMQSourceEnumState(Set currentSplitAssignment) { 34 | this.currentSplitAssignment = currentSplitAssignment; 35 | } 36 | 37 | public Set getCurrentSplitAssignment() { 38 | return currentSplitAssignment; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/DefaultTopicSelector.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 | package org.apache.flink.connector.rocketmq.legacy.common.selector; 18 | 19 | public class DefaultTopicSelector implements TopicSelector { 20 | private final String topicName; 21 | private final String tagName; 22 | 23 | public DefaultTopicSelector(final String topicName, final String tagName) { 24 | this.topicName = topicName; 25 | this.tagName = tagName; 26 | } 27 | 28 | public DefaultTopicSelector(final String topicName) { 29 | this(topicName, ""); 30 | } 31 | 32 | @Override 33 | public String getTopic(T tuple) { 34 | return topicName; 35 | } 36 | 37 | @Override 38 | public String getTag(T tuple) { 39 | return tagName; 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/MessageExtDeserializationScheme.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.rocketmq.legacy.common.serialization; 19 | 20 | import org.apache.flink.api.java.typeutils.ResultTypeQueryable; 21 | 22 | import org.apache.rocketmq.common.message.MessageExt; 23 | 24 | import java.io.Serializable; 25 | 26 | /** 27 | * The interface Message ext deserialization scheme. 28 | * 29 | * @param the type parameter 30 | */ 31 | public interface MessageExtDeserializationScheme extends ResultTypeQueryable, Serializable { 32 | /** 33 | * Deserialize messageExt to type T you want to output. 34 | * 35 | * @param messageExt the messageExt 36 | * @return the t 37 | */ 38 | T deserializeMessageExt(MessageExt messageExt); 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/WaterMarkForAll.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.rocketmq.legacy.common.watermark; 19 | 20 | import org.apache.flink.streaming.api.watermark.Watermark; 21 | 22 | public class WaterMarkForAll { 23 | 24 | private long maxOutOfOrderness = 5000L; // 5 seconds 25 | 26 | private long maxTimestamp = 0L; 27 | 28 | public WaterMarkForAll() {} 29 | 30 | public WaterMarkForAll(long maxOutOfOrderness) { 31 | this.maxOutOfOrderness = maxOutOfOrderness; 32 | } 33 | 34 | public void extractTimestamp(long timestamp) { 35 | maxTimestamp = Math.max(timestamp, maxTimestamp); 36 | } 37 | 38 | public Watermark getCurrentWatermark() { 39 | return new Watermark(maxTimestamp - maxOutOfOrderness); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/TestUtils.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 | package org.apache.flink.connector.rocketmq.legacy.common.util; 18 | 19 | import java.lang.reflect.Field; 20 | 21 | public class TestUtils { 22 | public static void setFieldValue(Object obj, String fieldName, Object value) { 23 | try { 24 | Field field = obj.getClass().getDeclaredField(fieldName); 25 | field.setAccessible(true); 26 | field.set(obj, value); 27 | } catch (Exception e) { 28 | e.printStackTrace(); 29 | } 30 | } 31 | 32 | public static Object getFieldValue(Object obj, String fieldName) 33 | throws NoSuchFieldException, IllegalAccessException { 34 | Field field = obj.getClass().getDeclaredField(fieldName); 35 | field.setAccessible(true); 36 | return field.get(obj); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/ConsistentHashAllocateStrategyTest.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.rocketmq.source.enumerator.allocate; 2 | 3 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit; 4 | 5 | import org.junit.Assert; 6 | import org.junit.Test; 7 | 8 | import java.util.ArrayList; 9 | import java.util.Collection; 10 | import java.util.HashSet; 11 | import java.util.Map; 12 | import java.util.Set; 13 | 14 | public class ConsistentHashAllocateStrategyTest { 15 | 16 | private static final String BROKER_NAME = "brokerName"; 17 | private static final String PREFIX_TOPIC = "test-topic-"; 18 | private static final int NUM_SPLITS = 3; 19 | private static final int[] SPLIT_SIZE = {1000, 2000, 3000}; 20 | 21 | @Test 22 | public void consistentHashAllocateStrategyTest() { 23 | AllocateStrategy allocateStrategy = new ConsistentHashAllocateStrategy(); 24 | Collection mqAll = new ArrayList<>(); 25 | for (int i = 0; i < NUM_SPLITS; i++) { 26 | mqAll.add( 27 | new RocketMQSourceSplit( 28 | PREFIX_TOPIC + (i + 1), BROKER_NAME, i, 0, SPLIT_SIZE[i])); 29 | } 30 | int parallelism = 2; 31 | Map> result = 32 | allocateStrategy.allocate(mqAll, parallelism); 33 | for (int i = 0; i < parallelism; i++) { 34 | Set splits = result.getOrDefault(i, new HashSet<>()); 35 | for (RocketMQSourceSplit split : splits) { 36 | mqAll.remove(split); 37 | } 38 | } 39 | Assert.assertEquals(0, mqAll.size()); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleStringDeserializationSchema.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 | package org.apache.flink.connector.rocketmq.legacy.common.serialization; 19 | 20 | import org.apache.flink.api.common.typeinfo.TypeInformation; 21 | 22 | import java.nio.charset.StandardCharsets; 23 | 24 | /** deserialize the message body to string */ 25 | public class SimpleStringDeserializationSchema implements KeyValueDeserializationSchema { 26 | 27 | private static final long serialVersionUID = 1L; 28 | 29 | @Override 30 | public String deserializeKeyAndValue(byte[] key, byte[] value) { 31 | String v = value != null ? new String(value, StandardCharsets.UTF_8) : null; 32 | return v; 33 | } 34 | 35 | @Override 36 | public TypeInformation getProducedType() { 37 | return TypeInformation.of(String.class); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/util/UtilAll.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.rocketmq.source.util; 19 | 20 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit; 21 | 22 | import org.apache.rocketmq.common.message.MessageQueue; 23 | 24 | public class UtilAll { 25 | 26 | public static final String SEPARATOR = "#"; 27 | 28 | public static String getSplitId(MessageQueue mq) { 29 | return mq.getTopic() + SEPARATOR + mq.getBrokerName() + SEPARATOR + mq.getQueueId(); 30 | } 31 | 32 | public static String getQueueDescription(MessageQueue mq) { 33 | return String.format( 34 | "(Topic: %s, BrokerName: %s, QueueId: %d)", 35 | mq.getTopic(), mq.getBrokerName(), mq.getQueueId()); 36 | } 37 | 38 | public static MessageQueue getMessageQueue(RocketMQSourceSplit split) { 39 | return new MessageQueue(split.getTopic(), split.getBrokerName(), split.getQueueId()); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/AverageAllocateStrategy.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.rocketmq.source.enumerator.allocate; 2 | 3 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit; 4 | 5 | import java.util.Collection; 6 | import java.util.HashMap; 7 | import java.util.HashSet; 8 | import java.util.Map; 9 | import java.util.Set; 10 | 11 | public class AverageAllocateStrategy implements AllocateStrategy { 12 | @Override 13 | public String getStrategyName() { 14 | return AllocateStrategyFactory.STRATEGY_NAME_AVERAGE; 15 | } 16 | 17 | @Override 18 | public Map> allocate( 19 | Collection mqAll, int parallelism) { 20 | return null; 21 | } 22 | 23 | @Override 24 | public Map> allocate( 25 | Collection mqAll, int parallelism, int globalAssignedNumber) { 26 | Map> result = new HashMap<>(); 27 | for (RocketMQSourceSplit mq : mqAll) { 28 | int readerIndex = 29 | this.getSplitOwner(mq.getTopic(), globalAssignedNumber++, parallelism); 30 | result.computeIfAbsent(readerIndex, k -> new HashSet<>()).add(mq); 31 | } 32 | return result; 33 | } 34 | 35 | private int getSplitOwner(String topic, int partition, int numReaders) { 36 | int startIndex = ((topic.hashCode() * 31) & 0x7FFFFFFF) % numReaders; 37 | 38 | // here, the assumption is that the id of RocketMQ partitions are always ascending 39 | // starting from 0, and therefore can be used directly as the offset clockwise from the 40 | // start index 41 | return (startIndex + partition) % numReaders; 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/BytesMessage.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.rocketmq.source.reader.deserializer; 20 | 21 | import java.util.HashMap; 22 | import java.util.Map; 23 | 24 | /** Message contains byte array. */ 25 | public class BytesMessage { 26 | 27 | private byte[] data; 28 | 29 | private Map properties = new HashMap<>(); 30 | 31 | public byte[] getData() { 32 | return data; 33 | } 34 | 35 | public void setData(byte[] data) { 36 | this.data = data; 37 | } 38 | 39 | public Map getProperties() { 40 | return properties; 41 | } 42 | 43 | public void setProperties(Map props) { 44 | this.properties = props; 45 | } 46 | 47 | public Object getProperty(String key) { 48 | return properties.get(key); 49 | } 50 | 51 | public void setProperty(String key, String value) { 52 | properties.put(key, value); 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /style/copyright/Apache.xml: -------------------------------------------------------------------------------- 1 | 17 | 18 | 19 | 20 | 24 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/SimpleTopicSelectorTest.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.rocketmq.legacy.common.selector; 20 | 21 | import org.junit.Test; 22 | 23 | import java.util.HashMap; 24 | import java.util.Map; 25 | 26 | import static org.junit.Assert.assertEquals; 27 | 28 | public class SimpleTopicSelectorTest { 29 | @Test 30 | public void getTopic() throws Exception { 31 | SimpleTopicSelector selector = new SimpleTopicSelector("tpc", "dtpc", "tg", "dtg"); 32 | Map tuple = new HashMap(); 33 | tuple.put("id", "x001"); 34 | tuple.put("name", "vesense"); 35 | tuple.put("tpc", "tpc1"); 36 | tuple.put("tg", "tg1"); 37 | 38 | assertEquals("tpc1", selector.getTopic(tuple)); 39 | assertEquals("tg1", selector.getTag(tuple)); 40 | 41 | tuple = new HashMap(); 42 | tuple.put("id", "x001"); 43 | tuple.put("name", "vesense"); 44 | 45 | assertEquals("dtpc", selector.getTopic(tuple)); 46 | assertEquals("dtg", selector.getTag(tuple)); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleTupleDeserializationSchema.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.rocketmq.legacy.common.serialization; 19 | 20 | import org.apache.flink.api.common.typeinfo.TypeHint; 21 | import org.apache.flink.api.common.typeinfo.TypeInformation; 22 | import org.apache.flink.api.java.tuple.Tuple2; 23 | 24 | import java.nio.charset.StandardCharsets; 25 | 26 | public class SimpleTupleDeserializationSchema 27 | implements KeyValueDeserializationSchema> { 28 | 29 | @Override 30 | public Tuple2 deserializeKeyAndValue(byte[] key, byte[] value) { 31 | String keyString = key != null ? new String(key, StandardCharsets.UTF_8) : null; 32 | String valueString = value != null ? new String(value, StandardCharsets.UTF_8) : null; 33 | return new Tuple2<>(keyString, valueString); 34 | } 35 | 36 | @Override 37 | public TypeInformation> getProducedType() { 38 | return TypeInformation.of(new TypeHint>() {}); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorNoStopping.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.rocketmq.source.enumerator.offset; 19 | 20 | import org.apache.flink.annotation.Internal; 21 | import org.apache.flink.connector.rocketmq.legacy.common.config.OffsetResetStrategy; 22 | 23 | import org.apache.rocketmq.common.message.MessageQueue; 24 | 25 | import java.util.Collection; 26 | import java.util.Collections; 27 | import java.util.Map; 28 | 29 | @Internal 30 | public class OffsetsSelectorNoStopping implements OffsetsSelector { 31 | 32 | @Override 33 | public Map getMessageQueueOffsets( 34 | Collection messageQueues, MessageQueueOffsetsRetriever offsetsRetriever) { 35 | 36 | return Collections.emptyMap(); 37 | } 38 | 39 | @Override 40 | public OffsetResetStrategy getAutoOffsetResetStrategy() { 41 | 42 | throw new UnsupportedOperationException( 43 | "The OffsetsSelectorNoStopping does not have an OffsetResetStrategy. " 44 | + "It should only be used to end offset."); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/common/event/SourceReportOffsetEvent.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.rocketmq.common.event; 20 | 21 | import org.apache.flink.api.connector.source.SourceEvent; 22 | 23 | public class SourceReportOffsetEvent implements SourceEvent { 24 | private String topic; 25 | private String broker; 26 | private int queueId; 27 | private long checkpoint = -1; 28 | 29 | public void setBroker(String broker) { 30 | this.broker = broker; 31 | } 32 | 33 | public void setCheckpoint(long checkpoint) { 34 | this.checkpoint = checkpoint; 35 | } 36 | 37 | public void setQueueId(int queueId) { 38 | this.queueId = queueId; 39 | } 40 | 41 | public void setTopic(String topic) { 42 | this.topic = topic; 43 | } 44 | 45 | public long getCheckpoint() { 46 | return checkpoint; 47 | } 48 | 49 | public int getQueueId() { 50 | return queueId; 51 | } 52 | 53 | public String getBroker() { 54 | return broker; 55 | } 56 | 57 | public String getTopic() { 58 | return topic; 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/BroadcastAllocateStrategyTest.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.rocketmq.source.enumerator.allocate; 2 | 3 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit; 4 | 5 | import org.junit.Test; 6 | 7 | import java.util.ArrayList; 8 | import java.util.Collection; 9 | import java.util.Map; 10 | import java.util.Set; 11 | 12 | import static org.junit.Assert.assertEquals; 13 | import static org.junit.Assert.assertTrue; 14 | 15 | public class BroadcastAllocateStrategyTest { 16 | 17 | private static final String BROKER_NAME = "brokerName"; 18 | private static final String PREFIX_TOPIC = "test-topic-"; 19 | private static final int NUM_SPLITS = 3; 20 | private static final int[] SPLIT_SIZE = {1000, 2000, 3000}; 21 | 22 | @Test 23 | public void broadcastAllocateStrategyTest() { 24 | AllocateStrategy allocateStrategy = new BroadcastAllocateStrategy(); 25 | Collection mqAll = new ArrayList<>(); 26 | for (int i = 0; i < NUM_SPLITS; i++) { 27 | mqAll.add( 28 | new RocketMQSourceSplit( 29 | PREFIX_TOPIC + (i + 1), BROKER_NAME, i, 0, SPLIT_SIZE[i])); 30 | } 31 | int parallelism = 3; 32 | Map> result = 33 | allocateStrategy.allocate(mqAll, parallelism); 34 | assertEquals(parallelism, result.size()); 35 | for (int i = 0; i < parallelism; i++) { 36 | Set splits = result.get(i); 37 | assertEquals(NUM_SPLITS, splits.size()); 38 | 39 | for (int j = 0; j < NUM_SPLITS; j++) { 40 | assertTrue( 41 | splits.contains( 42 | new RocketMQSourceSplit( 43 | PREFIX_TOPIC + (i + 1), BROKER_NAME, i, 0, SPLIT_SIZE[i]))); 44 | } 45 | } 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleKeyValueSerializationSchemaTest.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.rocketmq.legacy.common.serialization; 20 | 21 | import org.junit.Test; 22 | 23 | import java.util.HashMap; 24 | import java.util.Map; 25 | 26 | import static org.junit.Assert.assertEquals; 27 | 28 | public class SimpleKeyValueSerializationSchemaTest { 29 | @Test 30 | public void serializeKeyAndValue() throws Exception { 31 | SimpleKeyValueSerializationSchema serializationSchema = 32 | new SimpleKeyValueSerializationSchema("id", "name"); 33 | SimpleKeyValueDeserializationSchema deserializationSchema = 34 | new SimpleKeyValueDeserializationSchema("id", "name"); 35 | 36 | Map tuple = new HashMap<>(); 37 | tuple.put("id", "x001"); 38 | tuple.put("name", "vesense"); 39 | 40 | assertEquals( 41 | tuple, 42 | deserializationSchema.deserializeKeyAndValue( 43 | serializationSchema.serializeKey(tuple), 44 | serializationSchema.serializeValue(tuple))); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/sink/writer/serializer/RocketMQSerializationSchema.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.rocketmq.sink.writer.serializer; 2 | 3 | import org.apache.flink.annotation.PublicEvolving; 4 | import org.apache.flink.api.common.serialization.SerializationSchema; 5 | import org.apache.flink.connector.rocketmq.sink.writer.context.RocketMQSinkContext; 6 | 7 | import org.apache.rocketmq.common.message.Message; 8 | import org.apache.rocketmq.common.message.MessageExt; 9 | 10 | import java.io.Serializable; 11 | 12 | /** 13 | * The serialization schema for how to serialize records into RocketMQ. A serialization schema which 14 | * defines how to convert a value of type {@code T} to {@link MessageExt}. 15 | * 16 | * @param the type of values being serialized 17 | */ 18 | @PublicEvolving 19 | public interface RocketMQSerializationSchema extends Serializable { 20 | 21 | /** 22 | * Initialization method for the schema. It is called before the actual working methods {@link 23 | * #serialize(Object, RocketMQSinkContext, Long)} and thus suitable for one time setup work. 24 | * 25 | *

The provided {@link SerializationSchema.InitializationContext} can be used to access 26 | * additional features such as e.g. registering user metrics. 27 | * 28 | * @param context Contextual information that can be used during initialization. 29 | * @param sinkContext runtime information i.e. partitions, subtaskId 30 | */ 31 | default void open( 32 | SerializationSchema.InitializationContext context, RocketMQSinkContext sinkContext) 33 | throws Exception {} 34 | 35 | /** 36 | * Serializes given element and returns it as a {@link MessageExt}. 37 | * 38 | * @param element element to be serialized 39 | * @param context context to possibly determine target partition 40 | * @param timestamp timestamp 41 | * @return RocketMQ {@link MessageExt} 42 | */ 43 | Message serialize(T element, RocketMQSinkContext context, Long timestamp); 44 | } 45 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/function/SinkMapFunction.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.rocketmq.legacy.function; 19 | 20 | import org.apache.flink.api.java.tuple.Tuple2; 21 | import org.apache.flink.streaming.api.functions.ProcessFunction; 22 | import org.apache.flink.util.Collector; 23 | 24 | import org.apache.commons.lang.Validate; 25 | import org.apache.rocketmq.common.message.Message; 26 | 27 | public class SinkMapFunction extends ProcessFunction, Message> { 28 | 29 | private String topic; 30 | 31 | private String tag; 32 | 33 | public SinkMapFunction() {} 34 | 35 | public SinkMapFunction(String topic, String tag) { 36 | this.topic = topic; 37 | this.tag = tag; 38 | } 39 | 40 | @Override 41 | public void processElement(Tuple2 tuple, Context ctx, Collector out) 42 | throws Exception { 43 | Validate.notNull(topic, "the message topic is null"); 44 | Validate.notNull(tuple.f1.getBytes(), "the message body is null"); 45 | 46 | Message message = new Message(topic, tag, tuple.f0, tuple.f1.getBytes()); 47 | out.collect(message); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/config/OffsetVerification.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.rocketmq.source.config; 19 | 20 | import org.apache.flink.annotation.Internal; 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.configuration.DescribedEnum; 23 | import org.apache.flink.configuration.description.InlineElement; 24 | 25 | import static org.apache.flink.configuration.description.TextElement.text; 26 | 27 | /** The enum class for defining the offset verify behavior. */ 28 | @PublicEvolving 29 | public enum OffsetVerification implements DescribedEnum { 30 | 31 | /** We would just fail the consuming. */ 32 | FAIL_ON_MISMATCH( 33 | text("Fail the consuming from RocketMQ when we don't find the related offset.")), 34 | 35 | /** Print one 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 InlineElement desc; 39 | 40 | OffsetVerification(InlineElement desc) { 41 | this.desc = desc; 42 | } 43 | 44 | @Internal 45 | @Override 46 | public InlineElement getDescription() { 47 | return desc; 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/HashMessageQueueSelectorTest.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.rocketmq.legacy.common.selector; 20 | 21 | import org.apache.rocketmq.common.message.Message; 22 | import org.apache.rocketmq.common.message.MessageQueue; 23 | import org.junit.Assert; 24 | import org.junit.Test; 25 | 26 | import java.nio.charset.StandardCharsets; 27 | import java.util.ArrayList; 28 | import java.util.List; 29 | 30 | public class HashMessageQueueSelectorTest { 31 | 32 | @Test 33 | public void testSelect() { 34 | MessageQueueSelector hash = new HashMessageQueueSelector(); 35 | List queues = new ArrayList<>(); 36 | MessageQueue queue0 = new MessageQueue("test", "broker-a", 0); 37 | MessageQueue queue1 = new MessageQueue("test", "broker-b", 1); 38 | MessageQueue queue2 = new MessageQueue("test", "broker-c", 2); 39 | queues.add(queue0); 40 | queues.add(queue1); 41 | queues.add(queue2); 42 | 43 | Message message = new Message("test", "*", "1", "body".getBytes(StandardCharsets.UTF_8)); 44 | 45 | MessageQueue messageQueue = hash.select(queues, message, 1); 46 | Assert.assertEquals(messageQueue.getQueueId(), 1); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/BroadcastAllocateStrategy.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 | package org.apache.flink.connector.rocketmq.source.enumerator.allocate; 18 | 19 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit; 20 | 21 | import java.util.Collection; 22 | import java.util.HashMap; 23 | import java.util.HashSet; 24 | import java.util.Map; 25 | import java.util.Set; 26 | 27 | public class BroadcastAllocateStrategy implements AllocateStrategy { 28 | 29 | @Override 30 | public String getStrategyName() { 31 | return AllocateStrategyFactory.STRATEGY_NAME_BROADCAST; 32 | } 33 | 34 | @Override 35 | public Map> allocate( 36 | final Collection mqAll, final int parallelism) { 37 | Map> result = new HashMap<>(parallelism); 38 | for (int i = 0; i < parallelism; i++) { 39 | result.computeIfAbsent(i, k -> new HashSet<>()).addAll(mqAll); 40 | } 41 | return result; 42 | } 43 | 44 | @Override 45 | public Map> allocate( 46 | Collection mqAll, int parallelism, int globalAssignedNumber) { 47 | return allocate(mqAll, parallelism); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/TimeLagWatermarkGenerator.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.rocketmq.legacy.common.watermark; 19 | 20 | import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; 21 | import org.apache.flink.streaming.api.watermark.Watermark; 22 | 23 | import org.apache.rocketmq.common.message.MessageExt; 24 | 25 | /** 26 | * This generator generates watermarks that are lagging behind processing time by a certain amount. 27 | * It assumes that elements arrive in Flink after at most a certain time. 28 | */ 29 | public class TimeLagWatermarkGenerator implements AssignerWithPeriodicWatermarks { 30 | private long maxTimeLag = 5000; // 5 seconds 31 | 32 | TimeLagWatermarkGenerator() {} 33 | 34 | TimeLagWatermarkGenerator(long maxTimeLag) { 35 | this.maxTimeLag = maxTimeLag; 36 | } 37 | 38 | @Override 39 | public long extractTimestamp(MessageExt element, long previousElementTimestamp) { 40 | return element.getBornTimestamp(); 41 | } 42 | 43 | @Override 44 | public Watermark getCurrentWatermark() { 45 | // return the watermark as current time minus the maximum time lag 46 | return new Watermark(System.currentTimeMillis() - maxTimeLag); 47 | } 48 | 49 | @Override 50 | public String toString() { 51 | return "TimeLagWatermarkGenerator{" + "maxTimeLag=" + maxTimeLag + '}'; 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/sink/writer/context/RocketMQSinkContext.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.rocketmq.sink.writer.context; 19 | 20 | import org.apache.flink.annotation.Experimental; 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.api.common.operators.MailboxExecutor; 23 | 24 | /** 25 | * This context provides information on the rocketmq record target location. An implementation that 26 | * would contain all the required context. 27 | */ 28 | @PublicEvolving 29 | public interface RocketMQSinkContext { 30 | 31 | /** 32 | * Get the number of the subtask that RocketMQSink is running on. The numbering starts from 0 33 | * and goes up to parallelism-1. (parallelism as returned by {@link 34 | * #getNumberOfParallelInstances()} 35 | * 36 | * @return number of subtask 37 | */ 38 | int getParallelInstanceId(); 39 | 40 | /** @return number of parallel RocketMQSink tasks. */ 41 | int getNumberOfParallelInstances(); 42 | 43 | /** 44 | * RocketMQ can check the schema and upgrade the schema automatically. If you enable this 45 | * option, we wouldn't serialize the record into bytes, we send and serialize it in the client. 46 | */ 47 | @Experimental 48 | boolean isEnableSchemaEvolution(); 49 | 50 | /** Returns the current process time in flink. */ 51 | long processTime(); 52 | 53 | MailboxExecutor getMailboxExecutor(); 54 | } 55 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/RowKeyValueDeserializationSchemaTest.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.rocketmq.legacy.common.serialization; 20 | 21 | import org.apache.flink.table.api.DataTypes; 22 | import org.apache.flink.table.api.TableSchema; 23 | import org.apache.flink.table.data.RowData; 24 | 25 | import org.apache.rocketmq.common.message.MessageExt; 26 | import org.junit.Test; 27 | 28 | import java.util.HashMap; 29 | 30 | import static org.junit.Assert.assertEquals; 31 | 32 | /** Test for {@link RowKeyValueDeserializationSchema}. */ 33 | public class RowKeyValueDeserializationSchemaTest { 34 | 35 | @Test 36 | public void testDeserializeKeyAndValue() { 37 | TableSchema tableSchema = 38 | new TableSchema.Builder().field("varchar", DataTypes.VARCHAR(100)).build(); 39 | RowKeyValueDeserializationSchema deserializationSchema = 40 | new RowKeyValueDeserializationSchema.Builder() 41 | .setTableSchema(tableSchema) 42 | .setProperties(new HashMap<>()) 43 | .build(); 44 | MessageExt messageExt = new MessageExt(); 45 | messageExt.setBody("test_deserialize_key_and_value".getBytes()); 46 | RowData rowData = deserializationSchema.deserializeKeyAndValue(null, messageExt.getBody()); 47 | assertEquals(new String(messageExt.getBody()), rowData.getString(0).toString()); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/example/ConnectorConfig.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.rocketmq.example; 19 | 20 | import org.apache.rocketmq.acl.common.AclClientRPCHook; 21 | import org.apache.rocketmq.acl.common.SessionCredentials; 22 | import org.apache.rocketmq.remoting.RPCHook; 23 | 24 | public class ConnectorConfig { 25 | 26 | // for rocketmq cluster 27 | public static final String ENDPOINTS = "127.0.0.1:9876"; 28 | public static final String CLUSTER_NAME = "flink-cluster"; 29 | 30 | // for admin 31 | public static final String ADMIN_TOOL_GROUP = "rocketmq-tools"; 32 | 33 | // for producer and consumer 34 | public static final String PRODUCER_GROUP = "PID-flink"; 35 | public static final String CONSUMER_GROUP = "GID-flink"; 36 | public static final String SOURCE_TOPIC_1 = "flink-source-1"; 37 | public static final String SOURCE_TOPIC_2 = "flink-source-2"; 38 | public static final String SINK_TOPIC_1 = "flink-sink-1"; 39 | public static final String SINK_TOPIC_2 = "flink-sink-2"; 40 | 41 | // for message 42 | public static final String TAGS = "*"; 43 | public static final String KEY_PREFIX = "key_"; 44 | public static final int MESSAGE_NUM = 100; 45 | 46 | // for authentication 47 | public static final String ACCESS_KEY = "accessKey"; 48 | public static final String SECRET_KEY = "secretKey"; 49 | 50 | public static RPCHook getAclRpcHook() { 51 | return new AclClientRPCHook(new SessionCredentials(ACCESS_KEY, SECRET_KEY)); 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/split/RocketMQSourceSplitState.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.rocketmq.source.split; 20 | 21 | /** This class extends RocketMQSourceSplit to track a mutable current offset. */ 22 | public class RocketMQSourceSplitState extends RocketMQSourceSplit { 23 | 24 | private long currentOffset; 25 | 26 | public RocketMQSourceSplitState(RocketMQSourceSplit partitionSplit) { 27 | super( 28 | partitionSplit.getTopic(), 29 | partitionSplit.getBrokerName(), 30 | partitionSplit.getQueueId(), 31 | partitionSplit.getStartingOffset(), 32 | partitionSplit.getStoppingOffset()); 33 | this.currentOffset = partitionSplit.getStartingOffset(); 34 | } 35 | 36 | public long getCurrentOffset() { 37 | return currentOffset; 38 | } 39 | 40 | public void setCurrentOffset(long currentOffset) { 41 | this.currentOffset = currentOffset; 42 | } 43 | 44 | /** 45 | * Use the current offset as the starting offset to create a new RocketMQSourceSplit. 46 | * 47 | * @return a new RocketMQSourceSplit which uses the current offset as its starting offset. 48 | */ 49 | public RocketMQSourceSplit getSourceSplit() { 50 | return new RocketMQSourceSplit( 51 | getTopic(), 52 | getBrokerName(), 53 | getQueueId(), 54 | getCurrentOffset(), 55 | getStoppingOffset(), 56 | getIsIncrease()); 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/legacy/common/selector/RandomMessageQueueSelectorTest.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.rocketmq.legacy.common.selector; 20 | 21 | import org.apache.rocketmq.common.message.Message; 22 | import org.apache.rocketmq.common.message.MessageQueue; 23 | import org.junit.Assert; 24 | import org.junit.Test; 25 | 26 | import java.nio.charset.StandardCharsets; 27 | import java.util.ArrayList; 28 | import java.util.List; 29 | 30 | public class RandomMessageQueueSelectorTest { 31 | 32 | @Test 33 | public void testSelect() { 34 | MessageQueueSelector hash = new RandomMessageQueueSelector(); 35 | List queueIds = 36 | new ArrayList() { 37 | { 38 | add(0); 39 | add(1); 40 | add(2); 41 | } 42 | }; 43 | List queues = new ArrayList<>(); 44 | MessageQueue queue0 = new MessageQueue("test", "broker-a", 0); 45 | MessageQueue queue1 = new MessageQueue("test", "broker-b", 1); 46 | MessageQueue queue2 = new MessageQueue("test", "broker-c", 2); 47 | queues.add(queue0); 48 | queues.add(queue1); 49 | queues.add(queue2); 50 | 51 | Message message = new Message("test", "*", "1", "body".getBytes(StandardCharsets.UTF_8)); 52 | 53 | MessageQueue messageQueue = hash.select(queues, message, 1); 54 | int queueId = messageQueue.getQueueId(); 55 | 56 | Assert.assertTrue(queueIds.contains(queueId)); 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfigBuilderTest.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.rocketmq.common.config; 2 | 3 | import org.apache.flink.configuration.ConfigOption; 4 | import org.apache.flink.configuration.ConfigOptions; 5 | import org.apache.flink.configuration.Configuration; 6 | 7 | import org.junit.jupiter.api.Test; 8 | 9 | import java.util.Properties; 10 | 11 | import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; 12 | import static org.junit.jupiter.api.Assertions.assertEquals; 13 | import static org.junit.jupiter.api.Assertions.assertThrows; 14 | 15 | public class RocketMQConfigBuilderTest { 16 | 17 | @Test 18 | void canNotSetSameOptionTwiceWithDifferentValue() { 19 | ConfigOption option = ConfigOptions.key("some.key").stringType().noDefaultValue(); 20 | RocketMQConfigBuilder builder = new RocketMQConfigBuilder(); 21 | builder.set(option, "value1"); 22 | 23 | assertDoesNotThrow(() -> builder.set(option, "value1")); 24 | assertThrows(IllegalArgumentException.class, () -> builder.set(option, "value2")); 25 | } 26 | 27 | @Test 28 | void setConfigurationCanNotOverrideExistedKeysWithNewValue() { 29 | ConfigOption option = ConfigOptions.key("string.k1").stringType().noDefaultValue(); 30 | RocketMQConfigBuilder builder = new RocketMQConfigBuilder(); 31 | 32 | Configuration configuration = new Configuration(); 33 | configuration.set(option, "value1"); 34 | 35 | builder.set(option, "value1"); 36 | assertDoesNotThrow(() -> builder.set(configuration)); 37 | 38 | configuration.set(option, "value2"); 39 | assertThrows(IllegalArgumentException.class, () -> builder.set(configuration)); 40 | } 41 | 42 | @Test 43 | void setPropertiesCanNotOverrideExistedKeysWithNewValueAndSupportTypeConversion() { 44 | ConfigOption option = ConfigOptions.key("int.type").intType().defaultValue(3); 45 | RocketMQConfigBuilder builder = new RocketMQConfigBuilder(); 46 | 47 | Properties properties = new Properties(); 48 | properties.put("int.type", "6"); 49 | assertDoesNotThrow(() -> builder.set(properties)); 50 | 51 | properties.put("int.type", "1"); 52 | assertThrows(IllegalArgumentException.class, () -> builder.set(properties)); 53 | 54 | Integer value = builder.get(option); 55 | assertEquals(value, 6); 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/BoundedOutOfOrdernessGenerator.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.rocketmq.legacy.common.watermark; 19 | 20 | import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; 21 | import org.apache.flink.streaming.api.watermark.Watermark; 22 | 23 | import org.apache.rocketmq.common.message.MessageExt; 24 | 25 | public class BoundedOutOfOrdernessGenerator implements AssignerWithPeriodicWatermarks { 26 | 27 | private long maxOutOfOrderness = 5000; // 5 seconds 28 | 29 | private long currentMaxTimestamp; 30 | 31 | public BoundedOutOfOrdernessGenerator() {} 32 | 33 | public BoundedOutOfOrdernessGenerator(long maxOutOfOrderness) { 34 | this.maxOutOfOrderness = maxOutOfOrderness; 35 | } 36 | 37 | @Override 38 | public long extractTimestamp(MessageExt element, long previousElementTimestamp) { 39 | long timestamp = element.getBornTimestamp(); 40 | currentMaxTimestamp = Math.max(timestamp, currentMaxTimestamp); 41 | return timestamp; 42 | } 43 | 44 | @Override 45 | public Watermark getCurrentWatermark() { 46 | // return the watermark as current highest timestamp minus the out-of-orderness bound 47 | return new Watermark(currentMaxTimestamp - maxOutOfOrderness); 48 | } 49 | 50 | @Override 51 | public String toString() { 52 | return "BoundedOutOfOrdernessGenerator{" 53 | + "maxOutOfOrderness=" 54 | + maxOutOfOrderness 55 | + ", currentMaxTimestamp=" 56 | + currentMaxTimestamp 57 | + '}'; 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/sink/table/RocketMQRowDataSink.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one or more contributor license 3 | * agreements. See the NOTICE file distributed with this work for additional information regarding 4 | * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the 5 | * "License"); you may not use this file except in compliance with the License. You may obtain a 6 | * 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 distributed under the 11 | * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either 12 | * express or implied. See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | package org.apache.flink.connector.rocketmq.sink.table; 16 | 17 | import org.apache.flink.api.common.functions.RuntimeContext; 18 | import org.apache.flink.configuration.Configuration; 19 | import org.apache.flink.connector.rocketmq.legacy.RocketMQSink; 20 | import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; 21 | import org.apache.flink.table.data.RowData; 22 | 23 | import org.apache.rocketmq.common.message.Message; 24 | 25 | /** RocketMQRowDataSink helps for writing the converted row data of table to RocketMQ messages. */ 26 | public class RocketMQRowDataSink extends RichSinkFunction { 27 | 28 | private static final long serialVersionUID = 1L; 29 | 30 | private final RocketMQSink sink; 31 | private final RocketMQRowDataConverter converter; 32 | 33 | public RocketMQRowDataSink(RocketMQSink sink, RocketMQRowDataConverter converter) { 34 | this.sink = sink; 35 | this.converter = converter; 36 | } 37 | 38 | @Override 39 | public void open(Configuration configuration) throws Exception { 40 | sink.open(configuration); 41 | converter.open(); 42 | } 43 | 44 | @Override 45 | public void setRuntimeContext(RuntimeContext runtimeContext) { 46 | sink.setRuntimeContext(runtimeContext); 47 | } 48 | 49 | @Override 50 | public void invoke(RowData rowData, Context context) throws Exception { 51 | Message message = converter.convert(rowData); 52 | if (message != null) { 53 | sink.invoke(message, context); 54 | } 55 | } 56 | 57 | @Override 58 | public void close() { 59 | sink.close(); 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleKeyValueSerializationSchema.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 | package org.apache.flink.connector.rocketmq.legacy.common.serialization; 18 | 19 | import java.nio.charset.StandardCharsets; 20 | import java.util.Map; 21 | 22 | public class SimpleKeyValueSerializationSchema implements KeyValueSerializationSchema { 23 | public static final String DEFAULT_KEY_FIELD = "key"; 24 | public static final String DEFAULT_VALUE_FIELD = "value"; 25 | 26 | private final String keyField; 27 | private final String valueField; 28 | 29 | public SimpleKeyValueSerializationSchema() { 30 | this(DEFAULT_KEY_FIELD, DEFAULT_VALUE_FIELD); 31 | } 32 | 33 | /** 34 | * SimpleKeyValueSerializationSchema Constructor. 35 | * 36 | * @param keyField tuple field for selecting the key 37 | * @param valueField tuple field for selecting the value 38 | */ 39 | public SimpleKeyValueSerializationSchema(String keyField, String valueField) { 40 | this.keyField = keyField; 41 | this.valueField = valueField; 42 | } 43 | 44 | @Override 45 | public byte[] serializeKey(Map tuple) { 46 | if (tuple == null || keyField == null) { 47 | return null; 48 | } 49 | Object key = tuple.get(keyField); 50 | return key != null ? key.toString().getBytes(StandardCharsets.UTF_8) : null; 51 | } 52 | 53 | @Override 54 | public byte[] serializeValue(Map tuple) { 55 | if (tuple == null || valueField == null) { 56 | return null; 57 | } 58 | Object value = tuple.get(valueField); 59 | return value != null ? value.toString().getBytes(StandardCharsets.UTF_8) : null; 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/metrics/RocketMQSourceReaderMetrics.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.rocketmq.source.metrics; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | import org.apache.flink.metrics.groups.SourceReaderMetricGroup; 22 | 23 | import org.apache.rocketmq.common.message.MessageQueue; 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | @PublicEvolving 28 | public class RocketMQSourceReaderMetrics { 29 | 30 | private static final Logger LOG = LoggerFactory.getLogger(RocketMQSourceReaderMetrics.class); 31 | 32 | public static final String ROCKETMQ_SOURCE_READER_METRIC_GROUP = "RocketmqSourceReader"; 33 | public static final String TOPIC_GROUP = "topic"; 34 | public static final String QUEUE_GROUP = "queue"; 35 | public static final String CURRENT_OFFSET_METRIC_GAUGE = "currentOffset"; 36 | public static final String COMMITTED_OFFSET_METRIC_GAUGE = "committedOffset"; 37 | public static final String COMMITS_SUCCEEDED_METRIC_COUNTER = "commitsSucceeded"; 38 | public static final String COMMITS_FAILED_METRIC_COUNTER = "commitsFailed"; 39 | public static final String ROCKETMQ_CONSUMER_METRIC_GROUP = "RocketMQConsumer"; 40 | 41 | public static final String CONSUMER_FETCH_MANAGER_GROUP = "consumer-fetch-manager-metrics"; 42 | public static final String BYTES_CONSUMED_TOTAL = "bytes-consumed-total"; 43 | public static final String RECORDS_LAG = "records-lag"; 44 | 45 | public RocketMQSourceReaderMetrics(SourceReaderMetricGroup sourceReaderMetricGroup) {} 46 | 47 | public void registerNewMessageQueue(MessageQueue messageQueue) {} 48 | 49 | public void unregisterMessageQueue(MessageQueue messageQueue) {} 50 | } 51 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQDeserializationSchemaWrapper.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.rocketmq.source.reader.deserializer; 19 | 20 | import org.apache.flink.annotation.Internal; 21 | import org.apache.flink.api.common.serialization.DeserializationSchema; 22 | import org.apache.flink.api.common.typeinfo.TypeInformation; 23 | import org.apache.flink.connector.rocketmq.source.reader.MessageView; 24 | import org.apache.flink.util.Collector; 25 | 26 | import java.io.IOException; 27 | 28 | /** 29 | * A {@link RocketMQDeserializationSchema} implementation which based on the given flink's {@link 30 | * DeserializationSchema}. We would consume the message as byte array from rocketmq and deserialize 31 | * it by using flink serialization logic. 32 | * 33 | * @param The output type of the message. 34 | */ 35 | @Internal 36 | public class RocketMQDeserializationSchemaWrapper implements RocketMQDeserializationSchema { 37 | 38 | private static final long serialVersionUID = 1L; 39 | private final DeserializationSchema deserializationSchema; 40 | 41 | RocketMQDeserializationSchemaWrapper(DeserializationSchema deserializationSchema) { 42 | this.deserializationSchema = deserializationSchema; 43 | } 44 | 45 | @Override 46 | public void open(DeserializationSchema.InitializationContext context) throws Exception { 47 | RocketMQDeserializationSchema.super.open(context); 48 | } 49 | 50 | @Override 51 | public void deserialize(MessageView messageView, Collector out) throws IOException {} 52 | 53 | @Override 54 | public TypeInformation getProducedType() { 55 | return deserializationSchema.getProducedType(); 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogFactoryOptions.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.rocketmq.catalog; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.configuration.ConfigOption; 23 | import org.apache.flink.configuration.ConfigOptions; 24 | import org.apache.flink.connector.rocketmq.common.constant.SchemaRegistryConstant; 25 | import org.apache.flink.connector.rocketmq.legacy.RocketMQConfig; 26 | import org.apache.flink.table.catalog.CommonCatalogOptions; 27 | 28 | /** {@link ConfigOption}s for {@link RocketMQCatalog}. */ 29 | @Internal 30 | public final class RocketMQCatalogFactoryOptions { 31 | 32 | public static final String IDENTIFIER = "rocketmq_catalog"; 33 | 34 | public static final ConfigOption DEFAULT_DATABASE = 35 | ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY) 36 | .stringType() 37 | .defaultValue(RocketMQCatalog.DEFAULT_DB); 38 | 39 | public static final ConfigOption NAME_SERVER_ADDR = 40 | ConfigOptions.key(RocketMQConfig.NAME_SERVER_ADDR) 41 | .stringType() 42 | .defaultValue("http://localhost:9876") 43 | .withDescription("Required rocketmq name server address"); 44 | 45 | public static final ConfigOption SCHEMA_REGISTRY_BASE_URL = 46 | ConfigOptions.key(SchemaRegistryConstant.SCHEMA_REGISTRY_BASE_URL_KEY) 47 | .stringType() 48 | .defaultValue(SchemaRegistryConstant.SCHEMA_REGISTRY_BASE_URL) 49 | .withDescription("Required schema registry server address"); 50 | 51 | private RocketMQCatalogFactoryOptions() {} 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/ConsistentHashAllocateStrategy.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 | package org.apache.flink.connector.rocketmq.source.enumerator.allocate; 18 | 19 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit; 20 | 21 | import java.util.Collection; 22 | import java.util.HashMap; 23 | import java.util.HashSet; 24 | import java.util.Map; 25 | import java.util.Set; 26 | 27 | public class ConsistentHashAllocateStrategy implements AllocateStrategy { 28 | 29 | @Override 30 | public String getStrategyName() { 31 | return AllocateStrategyFactory.STRATEGY_NAME_CONSISTENT_HASH; 32 | } 33 | 34 | /** Returns the index of the target subtask that a specific queue should be assigned to. */ 35 | private int getSplitOwner(RocketMQSourceSplit sourceSplit, int parallelism) { 36 | int startIndex = 37 | ((sourceSplit.getMessageQueue().hashCode() * 31) & Integer.MAX_VALUE) % parallelism; 38 | return startIndex % parallelism; 39 | } 40 | 41 | @Override 42 | public Map> allocate( 43 | final Collection mqAll, final int parallelism) { 44 | Map> result = new HashMap<>(); 45 | for (RocketMQSourceSplit mq : mqAll) { 46 | int readerIndex = this.getSplitOwner(mq, parallelism); 47 | result.computeIfAbsent(readerIndex, k -> new HashSet<>()).add(mq); 48 | } 49 | return result; 50 | } 51 | 52 | @Override 53 | public Map> allocate( 54 | Collection mqAll, int parallelism, int globalAssignedNumber) { 55 | return allocate(mqAll, parallelism); 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/PunctuatedAssigner.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.rocketmq.legacy.common.watermark; 19 | 20 | import org.apache.flink.connector.rocketmq.legacy.RocketMQConfig; 21 | import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; 22 | import org.apache.flink.streaming.api.watermark.Watermark; 23 | 24 | import org.apache.rocketmq.common.message.MessageExt; 25 | 26 | /** 27 | * With Punctuated Watermarks To generate watermarks whenever a certain event indicates that a new 28 | * watermark might be generated, use AssignerWithPunctuatedWatermarks. For this class Flink will 29 | * first call the extractTimestamp(...) method to assign the element a timestamp, and then 30 | * immediately call the checkAndGetNextWatermark(...) method on that element. 31 | * 32 | *

The checkAndGetNextWatermark(...) method is passed the timestamp that was assigned in the 33 | * extractTimestamp(...) method, and can decide whether it wants to generate a watermark. Whenever 34 | * the checkAndGetNextWatermark(...) method returns a non-null watermark, and that watermark is 35 | * larger than the latest previous watermark, that new watermark will be emitted. 36 | */ 37 | public class PunctuatedAssigner implements AssignerWithPunctuatedWatermarks { 38 | @Override 39 | public long extractTimestamp(MessageExt element, long previousElementTimestamp) { 40 | return element.getBornTimestamp(); 41 | } 42 | 43 | @Override 44 | public Watermark checkAndGetNextWatermark(MessageExt lastElement, long extractedTimestamp) { 45 | String lastValue = lastElement.getProperty(RocketMQConfig.WATERMARK); 46 | return lastValue != null ? new Watermark(extractedTimestamp) : null; 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/WaterMarkPerQueue.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.rocketmq.legacy.common.watermark; 19 | 20 | import org.apache.flink.streaming.api.watermark.Watermark; 21 | 22 | import org.apache.rocketmq.common.message.MessageQueue; 23 | 24 | import java.util.Map; 25 | import java.util.concurrent.ConcurrentHashMap; 26 | import java.util.concurrent.ConcurrentMap; 27 | 28 | public class WaterMarkPerQueue { 29 | 30 | private ConcurrentMap maxEventTimeTable; 31 | 32 | private long maxOutOfOrderness = 5000L; // 5 seconds 33 | 34 | public WaterMarkPerQueue() {} 35 | 36 | public WaterMarkPerQueue(long maxOutOfOrderness) { 37 | this.maxOutOfOrderness = maxOutOfOrderness; 38 | maxEventTimeTable = new ConcurrentHashMap<>(); 39 | } 40 | 41 | public void extractTimestamp(MessageQueue mq, long timestamp) { 42 | long maxEventTime = maxEventTimeTable.getOrDefault(mq, maxOutOfOrderness); 43 | maxEventTimeTable.put(mq, Math.max(maxEventTime, timestamp)); 44 | } 45 | 46 | public Watermark getCurrentWatermark() { 47 | // return the watermark as current highest timestamp minus the out-of-orderness bound 48 | long minTimestamp = maxOutOfOrderness; 49 | for (Map.Entry entry : maxEventTimeTable.entrySet()) { 50 | minTimestamp = Math.min(minTimestamp, entry.getValue()); 51 | } 52 | return new Watermark(minTimestamp - maxOutOfOrderness); 53 | } 54 | 55 | @Override 56 | public String toString() { 57 | return "WaterMarkPerQueue{" 58 | + "maxEventTimeTable=" 59 | + maxEventTimeTable 60 | + ", maxOutOfOrderness=" 61 | + maxOutOfOrderness 62 | + '}'; 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogFactoryTest.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.rocketmq.catalog; 20 | 21 | import org.apache.flink.configuration.ConfigOption; 22 | import org.apache.flink.table.catalog.Catalog; 23 | import org.apache.flink.table.factories.FactoryUtil; 24 | 25 | import org.junit.Test; 26 | 27 | import java.util.HashMap; 28 | import java.util.Set; 29 | 30 | import static org.junit.Assert.assertEquals; 31 | import static org.junit.Assert.assertNotNull; 32 | 33 | public class RocketMQCatalogFactoryTest { 34 | 35 | @Test 36 | public void testCreateCatalog() { 37 | RocketMQCatalogFactory factory = new RocketMQCatalogFactory(); 38 | FactoryUtil.DefaultCatalogContext context = 39 | new FactoryUtil.DefaultCatalogContext( 40 | "rocketmq-catalog", 41 | new HashMap<>(), 42 | null, 43 | this.getClass().getClassLoader()); 44 | Catalog catalog = factory.createCatalog(context); 45 | assertNotNull(catalog); 46 | } 47 | 48 | @Test 49 | public void testFactoryIdentifier() { 50 | RocketMQCatalogFactory factory = new RocketMQCatalogFactory(); 51 | assertEquals(factory.factoryIdentifier(), "rocketmq_catalog"); 52 | } 53 | 54 | @Test 55 | public void testRequiredOptions() { 56 | RocketMQCatalogFactory factory = new RocketMQCatalogFactory(); 57 | Set> options = factory.requiredOptions(); 58 | assertNotNull(options); 59 | } 60 | 61 | @Test 62 | public void testOptionalOptions() { 63 | RocketMQCatalogFactory factory = new RocketMQCatalogFactory(); 64 | Set> options = factory.optionalOptions(); 65 | assertEquals(options.size(), 3); 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/QueryableSchema.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.rocketmq.source.reader.deserializer; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.api.common.serialization.DeserializationSchema.InitializationContext; 23 | import org.apache.flink.api.java.typeutils.ResultTypeQueryable; 24 | import org.apache.flink.util.Collector; 25 | 26 | import java.io.IOException; 27 | import java.io.Serializable; 28 | 29 | /** An interface for the deserialization of records. */ 30 | public interface QueryableSchema extends Serializable, ResultTypeQueryable { 31 | 32 | /** 33 | * Initialization method for the schema. It is called before the actual working methods {@link 34 | * #deserialize} and thus suitable for one time setup work. 35 | * 36 | *

The provided {@link InitializationContext} can be used to access additional features such 37 | * as e.g. registering user metrics. 38 | * 39 | * @param context Contextual information that can be used during initialization. 40 | */ 41 | @PublicEvolving 42 | default void open(InitializationContext context) throws Exception {} 43 | 44 | /** 45 | * Deserializes the byte message. 46 | * 47 | *

Can output multiple records through the {@link Collector}. Note that number and size of 48 | * the produced records should be relatively small. Depending on the source implementation 49 | * records can be buffered in memory or collecting records might delay emitting checkpoint 50 | * barrier. 51 | * 52 | * @param record The record to deserialize. 53 | * @param out The collector to put the resulting messages. 54 | */ 55 | @PublicEvolving 56 | void deserialize(IN record, Collector out) throws IOException; 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/AllocateStrategy.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.rocketmq.source.enumerator.allocate; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit; 22 | 23 | import java.util.Collection; 24 | import java.util.Map; 25 | import java.util.Set; 26 | 27 | /** This interface defines a strategy for allocating RocketMQ source splits to Flink tasks. */ 28 | @PublicEvolving 29 | public interface AllocateStrategy { 30 | 31 | /** 32 | * Allocate strategy name 33 | * 34 | * @return Current strategy name 35 | */ 36 | String getStrategyName(); 37 | 38 | /** 39 | * Allocates RocketMQ source splits to Flink tasks based on the selected allocation strategy. 40 | * 41 | * @param mqAll a collection of all available RocketMQ source splits 42 | * @param parallelism the desired parallelism for the Flink tasks 43 | * @return a map of task indices to sets of corresponding RocketMQ source splits 44 | */ 45 | Map> allocate( 46 | final Collection mqAll, final int parallelism); 47 | 48 | /** 49 | * Allocates RocketMQ source splits to Flink tasks based on the selected allocation strategy. 50 | * 51 | * @param mqAll a collection of all available RocketMQ source splits 52 | * @param parallelism the desired parallelism for the Flink tasks 53 | * @param globalAssignedNumber number of allocated queues 54 | * @return a map of task indices to sets of corresponding RocketMQ source splits 55 | */ 56 | Map> allocate( 57 | final Collection mqAll, 58 | final int parallelism, 59 | int globalAssignedNumber); 60 | } 61 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/legacy/common/util/RetryUtilTest.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.rocketmq.legacy.common.util; 20 | 21 | import org.apache.flink.connector.rocketmq.legacy.RunningChecker; 22 | 23 | import junit.framework.TestCase; 24 | import lombok.Data; 25 | import lombok.extern.slf4j.Slf4j; 26 | 27 | import java.util.concurrent.ExecutorService; 28 | import java.util.concurrent.Executors; 29 | 30 | /** Tests for {@link RetryUtil}. */ 31 | @Slf4j 32 | public class RetryUtilTest extends TestCase { 33 | 34 | public void testCall() { 35 | try { 36 | User user = new User(); 37 | RunningChecker runningChecker = new RunningChecker(); 38 | runningChecker.setRunning(true); 39 | ExecutorService executorService = Executors.newCachedThreadPool(); 40 | executorService.execute( 41 | () -> 42 | RetryUtil.call( 43 | () -> { 44 | user.setName("test"); 45 | user.setAge(Integer.parseInt("12e")); 46 | return true; 47 | }, 48 | "Something is error", 49 | runningChecker)); 50 | Thread.sleep(10000); 51 | executorService.shutdown(); 52 | log.info("Thread has finished"); 53 | assertEquals(0, user.age); 54 | assertEquals("test", user.name); 55 | assertEquals(false, runningChecker.isRunning()); 56 | } catch (Exception e) { 57 | log.warn("Exception has been caught"); 58 | } 59 | } 60 | 61 | @Data 62 | public class User { 63 | String name; 64 | int age; 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/allocate/AllocateStrategyFactory.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.rocketmq.source.enumerator.allocate; 19 | 20 | import org.apache.flink.api.connector.source.SplitEnumeratorContext; 21 | import org.apache.flink.configuration.Configuration; 22 | import org.apache.flink.connector.rocketmq.source.RocketMQSourceOptions; 23 | import org.apache.flink.connector.rocketmq.source.enumerator.RocketMQSourceEnumState; 24 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit; 25 | 26 | public class AllocateStrategyFactory { 27 | 28 | public static final String STRATEGY_NAME_BROADCAST = "broadcast"; 29 | public static final String STRATEGY_NAME_CONSISTENT_HASH = "hash"; 30 | public static final String STRATEGY_NAME_AVERAGE = "average"; 31 | 32 | private AllocateStrategyFactory() { 33 | // No public constructor. 34 | } 35 | 36 | public static AllocateStrategy getStrategy( 37 | Configuration rocketmqSourceOptions, 38 | SplitEnumeratorContext context, 39 | RocketMQSourceEnumState enumState) { 40 | 41 | String allocateStrategyName = 42 | rocketmqSourceOptions.getString( 43 | RocketMQSourceOptions.ALLOCATE_MESSAGE_QUEUE_STRATEGY); 44 | 45 | switch (allocateStrategyName) { 46 | case STRATEGY_NAME_CONSISTENT_HASH: 47 | return new ConsistentHashAllocateStrategy(); 48 | case STRATEGY_NAME_BROADCAST: 49 | return new BroadcastAllocateStrategy(); 50 | case STRATEGY_NAME_AVERAGE: 51 | return new AverageAllocateStrategy(); 52 | default: 53 | throw new IllegalArgumentException( 54 | "We don't support this allocate strategy: " + allocateStrategyName); 55 | } 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorByStrategy.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.rocketmq.source.enumerator.offset; 19 | 20 | import org.apache.flink.connector.rocketmq.legacy.common.config.OffsetResetStrategy; 21 | 22 | import org.apache.rocketmq.common.consumer.ConsumeFromWhere; 23 | import org.apache.rocketmq.common.message.MessageQueue; 24 | 25 | import java.util.Collection; 26 | import java.util.Map; 27 | import java.util.Properties; 28 | 29 | public class OffsetsSelectorByStrategy implements OffsetsSelector, OffsetsValidator { 30 | 31 | private final ConsumeFromWhere consumeFromWhere; 32 | private final OffsetResetStrategy offsetResetStrategy; 33 | 34 | OffsetsSelectorByStrategy( 35 | ConsumeFromWhere consumeFromWhere, OffsetResetStrategy offsetResetStrategy) { 36 | this.consumeFromWhere = consumeFromWhere; 37 | this.offsetResetStrategy = offsetResetStrategy; 38 | } 39 | 40 | @Override 41 | @SuppressWarnings("deprecation") 42 | public Map getMessageQueueOffsets( 43 | Collection messageQueues, MessageQueueOffsetsRetriever offsetsRetriever) { 44 | 45 | switch (consumeFromWhere) { 46 | case CONSUME_FROM_FIRST_OFFSET: 47 | return offsetsRetriever.minOffsets(messageQueues); 48 | case CONSUME_FROM_MAX_OFFSET: 49 | return offsetsRetriever.maxOffsets(messageQueues); 50 | case CONSUME_FROM_LAST_OFFSET: 51 | default: 52 | return offsetsRetriever.committedOffsets(messageQueues); 53 | } 54 | } 55 | 56 | @Override 57 | public OffsetResetStrategy getAutoOffsetResetStrategy() { 58 | return offsetResetStrategy; 59 | } 60 | 61 | @Override 62 | public void validate(Properties rocketmqSourceProperties) throws IllegalStateException {} 63 | } 64 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/sink/writer/context/RocketMQSinkContextImpl.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.rocketmq.sink.writer.context; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | import org.apache.flink.api.common.operators.MailboxExecutor; 22 | import org.apache.flink.api.common.operators.ProcessingTimeService; 23 | import org.apache.flink.api.connector.sink2.Sink.InitContext; 24 | import org.apache.flink.configuration.Configuration; 25 | 26 | @PublicEvolving 27 | public class RocketMQSinkContextImpl implements RocketMQSinkContext { 28 | 29 | private final int numberOfParallelSubtasks; 30 | private final int parallelInstanceId; 31 | private final ProcessingTimeService processingTimeService; 32 | private final MailboxExecutor mailboxExecutor; 33 | private final boolean enableSchemaEvolution; 34 | 35 | public RocketMQSinkContextImpl(InitContext initContext, Configuration configuration) { 36 | this.parallelInstanceId = initContext.getSubtaskId(); 37 | this.numberOfParallelSubtasks = initContext.getNumberOfParallelSubtasks(); 38 | this.processingTimeService = initContext.getProcessingTimeService(); 39 | this.mailboxExecutor = initContext.getMailboxExecutor(); 40 | this.enableSchemaEvolution = false; 41 | } 42 | 43 | @Override 44 | public int getParallelInstanceId() { 45 | return parallelInstanceId; 46 | } 47 | 48 | @Override 49 | public int getNumberOfParallelInstances() { 50 | return numberOfParallelSubtasks; 51 | } 52 | 53 | @Override 54 | public boolean isEnableSchemaEvolution() { 55 | return enableSchemaEvolution; 56 | } 57 | 58 | @Override 59 | public long processTime() { 60 | return processingTimeService.getCurrentProcessingTime(); 61 | } 62 | 63 | @Override 64 | public MailboxExecutor getMailboxExecutor() { 65 | return mailboxExecutor; 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/serialization/SimpleKeyValueDeserializationSchema.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 | package org.apache.flink.connector.rocketmq.legacy.common.serialization; 18 | 19 | import org.apache.flink.api.common.typeinfo.TypeInformation; 20 | import org.apache.flink.api.java.typeutils.MapTypeInfo; 21 | 22 | import java.nio.charset.StandardCharsets; 23 | import java.util.HashMap; 24 | import java.util.Map; 25 | 26 | public class SimpleKeyValueDeserializationSchema 27 | implements KeyValueDeserializationSchema> { 28 | public static final String DEFAULT_KEY_FIELD = "key"; 29 | public static final String DEFAULT_VALUE_FIELD = "value"; 30 | 31 | private final String keyField; 32 | private final String valueField; 33 | 34 | public SimpleKeyValueDeserializationSchema() { 35 | this(DEFAULT_KEY_FIELD, DEFAULT_VALUE_FIELD); 36 | } 37 | 38 | /** 39 | * SimpleKeyValueDeserializationSchema Constructor. 40 | * 41 | * @param keyField tuple field for selecting the key 42 | * @param valueField tuple field for selecting the value 43 | */ 44 | public SimpleKeyValueDeserializationSchema(String keyField, String valueField) { 45 | this.keyField = keyField; 46 | this.valueField = valueField; 47 | } 48 | 49 | @Override 50 | public Map deserializeKeyAndValue(byte[] key, byte[] value) { 51 | HashMap map = new HashMap<>(2); 52 | if (keyField != null) { 53 | String k = key != null ? new String(key, StandardCharsets.UTF_8) : null; 54 | map.put(keyField, k); 55 | } 56 | if (valueField != null) { 57 | String v = value != null ? new String(value, StandardCharsets.UTF_8) : null; 58 | map.put(valueField, v); 59 | } 60 | return map; 61 | } 62 | 63 | @Override 64 | public TypeInformation> getProducedType() { 65 | return new MapTypeInfo<>(String.class, String.class); 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/catalog/RocketMQCatalogFactory.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.rocketmq.catalog; 20 | 21 | import org.apache.flink.configuration.ConfigOption; 22 | import org.apache.flink.table.catalog.Catalog; 23 | import org.apache.flink.table.factories.CatalogFactory; 24 | import org.apache.flink.table.factories.FactoryUtil; 25 | 26 | import java.util.Collections; 27 | import java.util.HashSet; 28 | import java.util.Set; 29 | 30 | import static org.apache.flink.connector.rocketmq.catalog.RocketMQCatalogFactoryOptions.DEFAULT_DATABASE; 31 | import static org.apache.flink.connector.rocketmq.catalog.RocketMQCatalogFactoryOptions.IDENTIFIER; 32 | import static org.apache.flink.connector.rocketmq.catalog.RocketMQCatalogFactoryOptions.NAME_SERVER_ADDR; 33 | import static org.apache.flink.connector.rocketmq.catalog.RocketMQCatalogFactoryOptions.SCHEMA_REGISTRY_BASE_URL; 34 | 35 | /** The {@CatalogFactory} implementation of RocketMQ. */ 36 | public class RocketMQCatalogFactory implements CatalogFactory { 37 | 38 | @Override 39 | public Catalog createCatalog(Context context) { 40 | final FactoryUtil.CatalogFactoryHelper helper = 41 | FactoryUtil.createCatalogFactoryHelper(this, context); 42 | helper.validate(); 43 | return new RocketMQCatalog( 44 | context.getName(), 45 | helper.getOptions().get(DEFAULT_DATABASE), 46 | helper.getOptions().get(NAME_SERVER_ADDR), 47 | helper.getOptions().get(SCHEMA_REGISTRY_BASE_URL)); 48 | } 49 | 50 | @Override 51 | public String factoryIdentifier() { 52 | return IDENTIFIER; 53 | } 54 | 55 | @Override 56 | public Set> requiredOptions() { 57 | return Collections.emptySet(); 58 | } 59 | 60 | @Override 61 | public Set> optionalOptions() { 62 | Set> options = new HashSet<>(); 63 | options.add(DEFAULT_DATABASE); 64 | options.add(NAME_SERVER_ADDR); 65 | options.add(SCHEMA_REGISTRY_BASE_URL); 66 | return options; 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /style/copyright/profiles_settings.xml: -------------------------------------------------------------------------------- 1 | 17 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 27 | 28 | 31 | 32 | 35 | 36 | 39 | 40 | 43 | 44 | 47 | 48 | 51 | 52 | 55 | 56 | 59 | 60 | 63 | 64 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/RetryUtil.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 | package org.apache.flink.connector.rocketmq.legacy.common.util; 18 | 19 | import org.apache.flink.connector.rocketmq.legacy.RunningChecker; 20 | 21 | import org.slf4j.Logger; 22 | import org.slf4j.LoggerFactory; 23 | 24 | import java.util.concurrent.Callable; 25 | 26 | public class RetryUtil { 27 | 28 | private static final Logger log = LoggerFactory.getLogger(RetryUtil.class); 29 | 30 | private static final long INITIAL_BACKOFF = 200; 31 | private static final long MAX_BACKOFF = 5000; 32 | private static final int MAX_ATTEMPTS = 5; 33 | 34 | public static final boolean DEBUG = false; 35 | 36 | private RetryUtil() {} 37 | 38 | public static void waitForMs(long sleepMs) { 39 | try { 40 | Thread.sleep(sleepMs); 41 | } catch (InterruptedException e) { 42 | Thread.currentThread().interrupt(); 43 | } 44 | } 45 | 46 | public static T call(Callable callable, String errorMsg) throws RuntimeException { 47 | return call(callable, errorMsg, null); 48 | } 49 | 50 | public static T call(Callable callable, String errorMsg, RunningChecker runningChecker) 51 | throws RuntimeException { 52 | long backoff = INITIAL_BACKOFF; 53 | int retries = 0; 54 | do { 55 | try { 56 | return callable.call(); 57 | } catch (Exception ex) { 58 | if (retries >= MAX_ATTEMPTS) { 59 | if (null != runningChecker) { 60 | runningChecker.setRunning(false); 61 | } 62 | throw new RuntimeException(ex); 63 | } 64 | if (DEBUG) { 65 | log.debug("{}, retry {}/{}", errorMsg, retries, MAX_ATTEMPTS, ex); 66 | } else { 67 | log.error("{}, retry {}/{}", errorMsg, retries, MAX_ATTEMPTS, ex); 68 | } 69 | retries++; 70 | } 71 | waitForMs(backoff); 72 | backoff = Math.min(backoff * 2, MAX_BACKOFF); 73 | } while (true); 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/watermark/BoundedOutOfOrdernessGeneratorPerQueue.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.rocketmq.legacy.common.watermark; 19 | 20 | import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; 21 | import org.apache.flink.streaming.api.watermark.Watermark; 22 | 23 | import org.apache.rocketmq.common.message.MessageExt; 24 | 25 | import java.util.Map; 26 | import java.util.concurrent.ConcurrentHashMap; 27 | 28 | public class BoundedOutOfOrdernessGeneratorPerQueue 29 | implements AssignerWithPeriodicWatermarks { 30 | 31 | private Map maxEventTimeTable; 32 | private long maxOutOfOrderness = 5000L; // 5 seconds 33 | 34 | public BoundedOutOfOrdernessGeneratorPerQueue() {} 35 | 36 | public BoundedOutOfOrdernessGeneratorPerQueue(long maxOutOfOrderness) { 37 | this.maxOutOfOrderness = maxOutOfOrderness; 38 | maxEventTimeTable = new ConcurrentHashMap<>(); 39 | } 40 | 41 | @Override 42 | public long extractTimestamp(MessageExt element, long previousElementTimestamp) { 43 | String key = element.getBrokerName() + "_" + element.getQueueId(); 44 | Long maxEventTime = maxEventTimeTable.getOrDefault(key, maxOutOfOrderness); 45 | long timestamp = element.getBornTimestamp(); 46 | maxEventTimeTable.put(key, Math.max(maxEventTime, timestamp)); 47 | return timestamp; 48 | } 49 | 50 | @Override 51 | public Watermark getCurrentWatermark() { 52 | // return the watermark as current highest timestamp minus the out-of-orderness bound 53 | long minTimestamp = 0L; 54 | for (Map.Entry entry : maxEventTimeTable.entrySet()) { 55 | minTimestamp = Math.min(minTimestamp, entry.getValue()); 56 | } 57 | return new Watermark(minTimestamp - maxOutOfOrderness); 58 | } 59 | 60 | @Override 61 | public String toString() { 62 | return "BoundedOutOfOrdernessGeneratorPerQueue{" 63 | + "maxEventTimeTable=" 64 | + maxEventTimeTable 65 | + ", maxOutOfOrderness=" 66 | + maxOutOfOrderness 67 | + '}'; 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/legacy/RocketMQSinkTest.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.rocketmq.legacy; 20 | 21 | import org.apache.flink.api.java.tuple.Tuple2; 22 | import org.apache.flink.connector.rocketmq.legacy.common.selector.DefaultTopicSelector; 23 | import org.apache.flink.connector.rocketmq.legacy.common.selector.TopicSelector; 24 | import org.apache.flink.connector.rocketmq.legacy.common.serialization.KeyValueSerializationSchema; 25 | import org.apache.flink.connector.rocketmq.legacy.common.serialization.SimpleKeyValueSerializationSchema; 26 | import org.apache.flink.connector.rocketmq.legacy.common.util.TestUtils; 27 | 28 | import org.apache.rocketmq.client.producer.DefaultMQProducer; 29 | import org.apache.rocketmq.common.message.Message; 30 | import org.junit.Before; 31 | import org.junit.Ignore; 32 | import org.junit.Test; 33 | 34 | import java.util.Properties; 35 | 36 | import static org.mockito.Mockito.mock; 37 | import static org.mockito.Mockito.verify; 38 | 39 | @Ignore 40 | public class RocketMQSinkTest { 41 | 42 | private RocketMQSink rocketMQSink; 43 | private DefaultMQProducer producer; 44 | 45 | @Before 46 | public void setUp() throws Exception { 47 | KeyValueSerializationSchema serializationSchema = 48 | new SimpleKeyValueSerializationSchema("id", "name"); 49 | TopicSelector topicSelector = new DefaultTopicSelector("tpc"); 50 | Properties props = new Properties(); 51 | props.setProperty( 52 | RocketMQConfig.MSG_DELAY_LEVEL, String.valueOf(RocketMQConfig.MSG_DELAY_LEVEL04)); 53 | rocketMQSink = new RocketMQSink(props); 54 | 55 | producer = mock(DefaultMQProducer.class); 56 | TestUtils.setFieldValue(rocketMQSink, "producer", producer); 57 | } 58 | 59 | @Test 60 | public void testSink() throws Exception { 61 | Tuple2 tuple = new Tuple2<>("id", "province"); 62 | String topic = "testTopic"; 63 | String tag = "testTag"; 64 | Message message = new Message(topic, tag, tuple.f0, tuple.f1.getBytes()); 65 | } 66 | 67 | @Test 68 | public void close() throws Exception { 69 | rocketMQSink.close(); 70 | 71 | verify(producer).shutdown(); 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/sink/committer/SendCommittableSerializer.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.rocketmq.sink.committer; 19 | 20 | import org.apache.flink.core.io.SimpleVersionedSerializer; 21 | 22 | import java.io.ByteArrayInputStream; 23 | import java.io.ByteArrayOutputStream; 24 | import java.io.DataInputStream; 25 | import java.io.DataOutputStream; 26 | import java.io.IOException; 27 | 28 | /** A serializer used to serialize {@link SendCommittable}. */ 29 | public class SendCommittableSerializer implements SimpleVersionedSerializer { 30 | 31 | private static final int CURRENT_VERSION = 1; 32 | 33 | @Override 34 | public int getVersion() { 35 | return CURRENT_VERSION; 36 | } 37 | 38 | @Override 39 | public byte[] serialize(SendCommittable obj) throws IOException { 40 | try (final ByteArrayOutputStream stream = new ByteArrayOutputStream(); 41 | final DataOutputStream out = new DataOutputStream(stream)) { 42 | out.writeUTF(obj.getTopic()); 43 | out.writeUTF(obj.getBrokerName()); 44 | out.writeInt(obj.getQueueId()); 45 | out.writeLong(obj.getQueueOffset()); 46 | out.writeUTF(obj.getMsgId()); 47 | out.writeUTF(obj.getOffsetMsgId()); 48 | out.writeUTF(obj.getTransactionId()); 49 | out.flush(); 50 | return stream.toByteArray(); 51 | } 52 | } 53 | 54 | @Override 55 | public SendCommittable deserialize(int version, byte[] serialized) throws IOException { 56 | try (final ByteArrayInputStream bis = new ByteArrayInputStream(serialized); 57 | final DataInputStream in = new DataInputStream(bis)) { 58 | SendCommittable sendCommittable = new SendCommittable(); 59 | sendCommittable.setTopic(in.readUTF()); 60 | sendCommittable.setBrokerName(in.readUTF()); 61 | sendCommittable.setQueueId(in.readInt()); 62 | sendCommittable.setQueueOffset(in.readLong()); 63 | sendCommittable.setMsgId(in.readUTF()); 64 | sendCommittable.setOffsetMsgId(in.readUTF()); 65 | sendCommittable.setTransactionId(in.readUTF()); 66 | return sendCommittable; 67 | } 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/selector/SimpleTopicSelector.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 | package org.apache.flink.connector.rocketmq.legacy.common.selector; 18 | 19 | import org.slf4j.Logger; 20 | import org.slf4j.LoggerFactory; 21 | 22 | import java.util.Map; 23 | 24 | /** Uses field name to select topic and tag name from tuple. */ 25 | public class SimpleTopicSelector implements TopicSelector { 26 | private static final Logger LOG = LoggerFactory.getLogger(SimpleTopicSelector.class); 27 | 28 | private final String topicFieldName; 29 | private final String defaultTopicName; 30 | 31 | private final String tagFieldName; 32 | private final String defaultTagName; 33 | 34 | /** 35 | * SimpleTopicSelector Constructor. 36 | * 37 | * @param topicFieldName field name used for selecting topic 38 | * @param defaultTopicName default field name used for selecting topic 39 | * @param tagFieldName field name used for selecting tag 40 | * @param defaultTagName default field name used for selecting tag 41 | */ 42 | public SimpleTopicSelector( 43 | String topicFieldName, 44 | String defaultTopicName, 45 | String tagFieldName, 46 | String defaultTagName) { 47 | this.topicFieldName = topicFieldName; 48 | this.defaultTopicName = defaultTopicName; 49 | this.tagFieldName = tagFieldName; 50 | this.defaultTagName = defaultTagName; 51 | } 52 | 53 | @Override 54 | public String getTopic(Map tuple) { 55 | if (tuple.containsKey(topicFieldName)) { 56 | Object topic = tuple.get(topicFieldName); 57 | return topic != null ? topic.toString() : defaultTopicName; 58 | } else { 59 | LOG.warn( 60 | "Field {} Not Found. Returning default topic {}", 61 | topicFieldName, 62 | defaultTopicName); 63 | return defaultTopicName; 64 | } 65 | } 66 | 67 | @Override 68 | public String getTag(Map tuple) { 69 | if (tuple.containsKey(tagFieldName)) { 70 | Object tag = tuple.get(tagFieldName); 71 | return tag != null ? tag.toString() : defaultTagName; 72 | } else { 73 | LOG.warn("Field {} Not Found. Returning default tag {}", tagFieldName, defaultTagName); 74 | return defaultTagName; 75 | } 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQRecordEmitter.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.rocketmq.source.reader; 20 | 21 | import org.apache.flink.api.connector.source.SourceOutput; 22 | import org.apache.flink.connector.base.source.reader.RecordEmitter; 23 | import org.apache.flink.connector.rocketmq.source.reader.deserializer.RocketMQDeserializationSchema; 24 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplitState; 25 | import org.apache.flink.util.Collector; 26 | 27 | import java.io.IOException; 28 | 29 | /** The {@link RecordEmitter} implementation for {@link RocketMQSourceReader}. */ 30 | public class RocketMQRecordEmitter 31 | implements RecordEmitter { 32 | 33 | private final RocketMQDeserializationSchema deserializationSchema; 34 | private final SourceOutputWrapper sourceOutputWrapper = new SourceOutputWrapper<>(); 35 | 36 | public RocketMQRecordEmitter(RocketMQDeserializationSchema deserializationSchema) { 37 | this.deserializationSchema = deserializationSchema; 38 | } 39 | 40 | @Override 41 | public void emitRecord( 42 | MessageView element, SourceOutput output, RocketMQSourceSplitState splitState) 43 | throws IOException { 44 | 45 | try { 46 | sourceOutputWrapper.setSourceOutput(output); 47 | sourceOutputWrapper.setTimestamp(element.getEventTime()); 48 | deserializationSchema.deserialize(element, sourceOutputWrapper); 49 | splitState.setCurrentOffset(element.getQueueOffset() + 1); 50 | } catch (Exception e) { 51 | throw new IOException("Failed to deserialize message due to", e); 52 | } 53 | } 54 | 55 | private static class SourceOutputWrapper implements Collector { 56 | 57 | private SourceOutput sourceOutput; 58 | 59 | private long timestamp; 60 | 61 | @Override 62 | public void collect(T record) { 63 | sourceOutput.collect(record, timestamp); 64 | } 65 | 66 | @Override 67 | public void close() {} 68 | 69 | private void setSourceOutput(SourceOutput sourceOutput) { 70 | this.sourceOutput = sourceOutput; 71 | } 72 | 73 | private void setTimestamp(long timestamp) { 74 | this.timestamp = timestamp; 75 | } 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorByTimestamp.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.rocketmq.source.enumerator.offset; 19 | 20 | import org.apache.flink.connector.rocketmq.legacy.common.config.OffsetResetStrategy; 21 | 22 | import org.apache.rocketmq.common.message.MessageQueue; 23 | 24 | import java.util.Collection; 25 | import java.util.HashMap; 26 | import java.util.Map; 27 | 28 | class OffsetsSelectorByTimestamp implements OffsetsSelector { 29 | private static final long serialVersionUID = 2932230571773627233L; 30 | private final long startingTimestamp; 31 | 32 | OffsetsSelectorByTimestamp(long startingTimestamp) { 33 | this.startingTimestamp = startingTimestamp; 34 | } 35 | 36 | @Override 37 | public Map getMessageQueueOffsets( 38 | Collection messageQueues, MessageQueueOffsetsRetriever offsetsRetriever) { 39 | Map startingTimestamps = new HashMap<>(); 40 | Map initialOffsets = new HashMap<>(); 41 | 42 | // First get the current end offsets of the partitions. This is going to be used 43 | // in case we cannot find a suitable offsets based on the timestamp, i.e. the message 44 | // meeting the requirement of the timestamp have not been produced to RocketMQ yet, 45 | // in this case, we just use the latest offset. 46 | // We need to get the latest offsets before querying offsets by time to ensure that 47 | // no message is going to be missed. 48 | Map endOffsets = offsetsRetriever.maxOffsets(messageQueues); 49 | messageQueues.forEach(tp -> startingTimestamps.put(tp, startingTimestamp)); 50 | offsetsRetriever 51 | .offsetsForTimes(startingTimestamps) 52 | .forEach( 53 | (mq, offsetByTimestamp) -> { 54 | long offset = 55 | offsetByTimestamp != null 56 | ? offsetByTimestamp 57 | : endOffsets.get(mq); 58 | initialOffsets.put(mq, offset); 59 | }); 60 | return initialOffsets; 61 | } 62 | 63 | @Override 64 | public OffsetResetStrategy getAutoOffsetResetStrategy() { 65 | return OffsetResetStrategy.LATEST; 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/split/RocketMQPartitionSplitSerializer.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.rocketmq.source.split; 20 | 21 | import org.apache.flink.core.io.SimpleVersionedSerializer; 22 | 23 | import java.io.ByteArrayInputStream; 24 | import java.io.ByteArrayOutputStream; 25 | import java.io.DataInputStream; 26 | import java.io.DataOutputStream; 27 | import java.io.IOException; 28 | 29 | /** The {@link SimpleVersionedSerializer serializer} for {@link RocketMQSourceSplit}. */ 30 | public class RocketMQPartitionSplitSerializer 31 | implements SimpleVersionedSerializer { 32 | 33 | private static final int SNAPSHOT_VERSION = 0; 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(RocketMQSourceSplit split) throws IOException { 43 | try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); 44 | DataOutputStream out = new DataOutputStream(byteArrayOutputStream)) { 45 | out.writeUTF(split.getTopic()); 46 | out.writeUTF(split.getBrokerName()); 47 | out.writeInt(split.getQueueId()); 48 | out.writeLong(split.getStartingOffset()); 49 | out.writeLong(split.getStoppingOffset()); 50 | out.writeBoolean(split.getIsIncrease()); 51 | out.flush(); 52 | return byteArrayOutputStream.toByteArray(); 53 | } 54 | } 55 | 56 | @Override 57 | public RocketMQSourceSplit deserialize(int version, byte[] serialized) throws IOException { 58 | try (ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(serialized); 59 | DataInputStream in = new DataInputStream(byteArrayInputStream)) { 60 | String topic = in.readUTF(); 61 | String broker = in.readUTF(); 62 | int partition = in.readInt(); 63 | long startingOffset = in.readLong(); 64 | long stoppingOffset = in.readLong(); 65 | if (version == SNAPSHOT_VERSION) { 66 | return new RocketMQSourceSplit( 67 | topic, broker, partition, startingOffset, stoppingOffset); 68 | } 69 | boolean isIncrease = in.readBoolean(); 70 | return new RocketMQSourceSplit( 71 | topic, broker, partition, startingOffset, stoppingOffset, isIncrease); 72 | } 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/sink/RocketMQSink.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.rocketmq.sink; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | import org.apache.flink.api.connector.sink2.Committer; 22 | import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; 23 | import org.apache.flink.configuration.Configuration; 24 | import org.apache.flink.connector.rocketmq.legacy.common.selector.MessageQueueSelector; 25 | import org.apache.flink.connector.rocketmq.sink.committer.RocketMQCommitter; 26 | import org.apache.flink.connector.rocketmq.sink.committer.SendCommittable; 27 | import org.apache.flink.connector.rocketmq.sink.committer.SendCommittableSerializer; 28 | import org.apache.flink.connector.rocketmq.sink.writer.RocketMQWriter; 29 | import org.apache.flink.connector.rocketmq.sink.writer.serializer.RocketMQSerializationSchema; 30 | import org.apache.flink.core.io.SimpleVersionedSerializer; 31 | 32 | @PublicEvolving 33 | public class RocketMQSink implements TwoPhaseCommittingSink { 34 | 35 | private final Configuration configuration; 36 | private final MessageQueueSelector messageQueueSelector; 37 | private final RocketMQSerializationSchema serializationSchema; 38 | 39 | RocketMQSink( 40 | Configuration configuration, 41 | MessageQueueSelector messageQueueSelector, 42 | RocketMQSerializationSchema serializationSchema) { 43 | this.configuration = configuration; 44 | this.messageQueueSelector = messageQueueSelector; 45 | this.serializationSchema = serializationSchema; 46 | } 47 | 48 | /** 49 | * Create a {@link RocketMQSinkBuilder} to construct a new {@link RocketMQSink}. 50 | * 51 | * @param type of incoming records 52 | * @return {@link RocketMQSinkBuilder} 53 | */ 54 | public static RocketMQSinkBuilder builder() { 55 | return new RocketMQSinkBuilder<>(); 56 | } 57 | 58 | @Override 59 | public PrecommittingSinkWriter createWriter(InitContext context) { 60 | return new RocketMQWriter<>( 61 | configuration, messageQueueSelector, serializationSchema, context); 62 | } 63 | 64 | @Override 65 | public Committer createCommitter() { 66 | return new RocketMQCommitter(configuration); 67 | } 68 | 69 | @Override 70 | public SimpleVersionedSerializer getCommittableSerializer() { 71 | return new SendCommittableSerializer(); 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/sink/InnerProducer.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.rocketmq.sink; 19 | 20 | import org.apache.flink.connector.rocketmq.sink.committer.SendCommittable; 21 | 22 | import org.apache.rocketmq.client.producer.SendResult; 23 | import org.apache.rocketmq.common.message.Message; 24 | 25 | import java.util.concurrent.CompletableFuture; 26 | 27 | /** 28 | * InnerProducer is an interface that represents a message producer used for sending messages to a 29 | * messaging system. 30 | * 31 | * @see AutoCloseable 32 | */ 33 | public interface InnerProducer extends AutoCloseable { 34 | 35 | /** Starts the inner consumer. */ 36 | void start(); 37 | 38 | /** 39 | * Gets the consumer group of the consumer. 40 | * 41 | * @return the consumer group of the consumer 42 | */ 43 | String getProducerGroup(); 44 | 45 | /** 46 | * Sends the message to the messaging system and returns a Future for the send operation. 47 | * 48 | * @param message the message to be sent 49 | * @return a Future for the send operation that returns a SendResult object 50 | * @see CompletableFuture 51 | * @see SendResult 52 | */ 53 | CompletableFuture send(Message message); 54 | 55 | /** 56 | * Sends the message to the messaging system and returns a Future for the send operation. 57 | * 58 | * @param message the message to be sent 59 | * @return a Future for the send operation that returns a SendResult object 60 | * @see CompletableFuture 61 | * @see SendResult 62 | */ 63 | CompletableFuture sendMessageInTransaction(Message message); 64 | 65 | /** 66 | * Commits the send operation identified by the specified SendCommittable object. 67 | * 68 | * @param sendCommittable the SendCommittable object identifying the send operation 69 | * @return a Future that indicates whether the commit operation was successful 70 | * @see CompletableFuture 71 | * @see SendCommittable 72 | */ 73 | CompletableFuture commit(SendCommittable sendCommittable); 74 | 75 | /** 76 | * Rolls back the send operation identified by the specified SendCommittable object. 77 | * 78 | * @param sendCommittable the SendCommittable object identifying the send operation 79 | * @return a Future that indicates whether the rollback operation was successful 80 | * @see CompletableFuture 81 | * @see SendCommittable 82 | */ 83 | CompletableFuture rollback(SendCommittable sendCommittable); 84 | } 85 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/example/SimpleConsumer.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.rocketmq.example; 19 | 20 | import org.apache.rocketmq.client.AccessChannel; 21 | import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; 22 | import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; 23 | import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; 24 | import org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely; 25 | import org.apache.rocketmq.client.exception.MQClientException; 26 | import org.apache.rocketmq.common.consumer.ConsumeFromWhere; 27 | import org.apache.rocketmq.common.message.MessageExt; 28 | 29 | import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.ENDPOINTS; 30 | import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.SOURCE_TOPIC_1; 31 | import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.SOURCE_TOPIC_2; 32 | import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.TAGS; 33 | import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.getAclRpcHook; 34 | 35 | public class SimpleConsumer { 36 | 37 | public static void main(String[] args) throws MQClientException { 38 | 39 | DefaultMQPushConsumer consumer = 40 | new DefaultMQPushConsumer( 41 | ConnectorConfig.CONSUMER_GROUP, 42 | getAclRpcHook(), 43 | new AllocateMessageQueueAveragely()); 44 | 45 | consumer.setNamesrvAddr(ENDPOINTS); 46 | consumer.setAccessChannel(AccessChannel.CLOUD); 47 | consumer.subscribe(SOURCE_TOPIC_1, TAGS); 48 | consumer.subscribe(SOURCE_TOPIC_2, TAGS); 49 | consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); 50 | consumer.registerMessageListener( 51 | (MessageListenerConcurrently) 52 | (msgList, context) -> { 53 | for (MessageExt msg : msgList) { 54 | System.out.printf( 55 | "Receive: topic=%s, msgId=%s, brokerName=%s, queueId=%s, payload=%s\n", 56 | msg.getTopic(), 57 | msg.getMsgId(), 58 | msg.getBrokerName(), 59 | msg.getQueueId(), 60 | new String(msg.getBody())); 61 | } 62 | return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; 63 | }); 64 | consumer.start(); 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/example/SimpleProducer.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.rocketmq.example; 19 | 20 | import org.apache.rocketmq.client.AccessChannel; 21 | import org.apache.rocketmq.client.exception.MQClientException; 22 | import org.apache.rocketmq.client.producer.DefaultMQProducer; 23 | import org.apache.rocketmq.client.producer.MQProducer; 24 | import org.apache.rocketmq.client.producer.SendResult; 25 | import org.apache.rocketmq.common.message.Message; 26 | import org.slf4j.Logger; 27 | import org.slf4j.LoggerFactory; 28 | 29 | import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.ENDPOINTS; 30 | import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.KEY_PREFIX; 31 | import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.SOURCE_TOPIC_1; 32 | import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.SOURCE_TOPIC_2; 33 | import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.TAGS; 34 | import static org.apache.flink.connector.rocketmq.example.ConnectorConfig.getAclRpcHook; 35 | 36 | public class SimpleProducer { 37 | 38 | private static final Logger LOGGER = LoggerFactory.getLogger(SimpleProducer.class); 39 | 40 | public static void main(String[] args) { 41 | 42 | DefaultMQProducer producer = 43 | new DefaultMQProducer(ConnectorConfig.PRODUCER_GROUP, getAclRpcHook(), true, null); 44 | producer.setNamesrvAddr(ENDPOINTS); 45 | producer.setAccessChannel(AccessChannel.CLOUD); 46 | 47 | try { 48 | producer.start(); 49 | } catch (MQClientException e) { 50 | e.printStackTrace(); 51 | } 52 | 53 | send(producer, SOURCE_TOPIC_1); 54 | send(producer, SOURCE_TOPIC_2); 55 | 56 | producer.shutdown(); 57 | } 58 | 59 | private static void send(MQProducer producer, String topic) { 60 | for (int i = 0; i < ConnectorConfig.MESSAGE_NUM; i++) { 61 | String content = "Test Message " + i; 62 | Message msg = new Message(topic, TAGS, KEY_PREFIX + i, content.getBytes()); 63 | try { 64 | SendResult sendResult = producer.send(msg); 65 | assert sendResult != null; 66 | System.out.printf( 67 | "Send result: topic=%s, msgId=%s, brokerName=%s, queueId=%s\n", 68 | topic, 69 | sendResult.getMsgId(), 70 | sendResult.getMessageQueue().getBrokerName(), 71 | sendResult.getMessageQueue().getQueueId()); 72 | } catch (Exception e) { 73 | LOGGER.info("Send message failed. {}", e.toString()); 74 | } 75 | } 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/reader/MessageView.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.rocketmq.source.reader; 19 | 20 | import java.util.Collection; 21 | import java.util.Map; 22 | 23 | /** This interface defines the methods for obtaining information about a message in RocketMQ. */ 24 | public interface MessageView { 25 | 26 | /** 27 | * Get the unique message ID. 28 | * 29 | * @return the message ID 30 | */ 31 | String getMessageId(); 32 | 33 | /** 34 | * Get the topic that the message belongs to. 35 | * 36 | * @return the topic 37 | */ 38 | String getTopic(); 39 | 40 | /** 41 | * Get the name of the broker that handles the message. 42 | * 43 | * @return the broker name 44 | */ 45 | String getBrokerName(); 46 | 47 | /** 48 | * Get the ID of the queue that the message is stored in. 49 | * 50 | * @return the queue ID 51 | */ 52 | int getQueueId(); 53 | 54 | /** 55 | * Get the offset of the message within the queue. 56 | * 57 | * @return the queue offset 58 | */ 59 | long getQueueOffset(); 60 | 61 | /** 62 | * Get the tag of the message, which is used for filtering. 63 | * 64 | * @return the message tag 65 | */ 66 | String getTag(); 67 | 68 | /** 69 | * Get the keys of the message, which are used for partitioning and indexing. 70 | * 71 | * @return the message keys 72 | */ 73 | Collection getKeys(); 74 | 75 | /** 76 | * Get the size of the message in bytes. 77 | * 78 | * @return the message size 79 | */ 80 | int getStoreSize(); 81 | 82 | /** 83 | * Get the body of the message. 84 | * 85 | * @return the message body 86 | */ 87 | byte[] getBody(); 88 | 89 | /** 90 | * Get the number of times that the message has been attempted to be delivered. 91 | * 92 | * @return the delivery attempt count 93 | */ 94 | int getDeliveryAttempt(); 95 | 96 | /** 97 | * Get the event time of the message, which is used for filtering and sorting. 98 | * 99 | * @return the event time 100 | */ 101 | long getEventTime(); 102 | 103 | /** 104 | * Get the ingestion time of the message, which is the time that the message was received by the 105 | * broker. 106 | * 107 | * @return the ingestion time 108 | */ 109 | long getIngestionTime(); 110 | 111 | /** 112 | * Get the properties of the message, which are set by the producer. 113 | * 114 | * @return the message properties 115 | */ 116 | Map getProperties(); 117 | } 118 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/offset/OffsetsSelectorBySpecified.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.rocketmq.source.enumerator.offset; 19 | 20 | import org.apache.flink.connector.rocketmq.legacy.common.config.OffsetResetStrategy; 21 | 22 | import org.apache.rocketmq.common.message.MessageQueue; 23 | 24 | import java.util.ArrayList; 25 | import java.util.Collection; 26 | import java.util.Collections; 27 | import java.util.HashMap; 28 | import java.util.List; 29 | import java.util.Map; 30 | import java.util.Properties; 31 | 32 | public class OffsetsSelectorBySpecified implements OffsetsSelector, OffsetsValidator { 33 | 34 | private final Map initialOffsets; 35 | private final OffsetResetStrategy offsetResetStrategy; 36 | 37 | OffsetsSelectorBySpecified( 38 | Map initialOffsets, OffsetResetStrategy offsetResetStrategy) { 39 | this.initialOffsets = Collections.unmodifiableMap(initialOffsets); 40 | this.offsetResetStrategy = offsetResetStrategy; 41 | } 42 | 43 | @Override 44 | public Map getMessageQueueOffsets( 45 | Collection messageQueues, MessageQueueOffsetsRetriever offsetsRetriever) { 46 | Map offsets = new HashMap<>(); 47 | List toLookup = new ArrayList<>(); 48 | for (MessageQueue tp : messageQueues) { 49 | Long offset = initialOffsets.get(tp); 50 | if (offset == null) { 51 | toLookup.add(tp); 52 | } else { 53 | offsets.put(tp, offset); 54 | } 55 | } 56 | if (!toLookup.isEmpty()) { 57 | // First check the committed offsets. 58 | Map committedOffsets = offsetsRetriever.committedOffsets(toLookup); 59 | offsets.putAll(committedOffsets); 60 | toLookup.removeAll(committedOffsets.keySet()); 61 | 62 | switch (offsetResetStrategy) { 63 | case EARLIEST: 64 | offsets.putAll(offsetsRetriever.minOffsets(toLookup)); 65 | break; 66 | case LATEST: 67 | offsets.putAll(offsetsRetriever.maxOffsets(toLookup)); 68 | break; 69 | default: 70 | throw new IllegalStateException( 71 | "Cannot find initial offsets for partitions: " + toLookup); 72 | } 73 | } 74 | return offsets; 75 | } 76 | 77 | @Override 78 | public OffsetResetStrategy getAutoOffsetResetStrategy() { 79 | return offsetResetStrategy; 80 | } 81 | 82 | @Override 83 | public void validate(Properties rocketmqSourceProperties) {} 84 | } 85 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/legacy/common/util/RocketMQUtils.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 | package org.apache.flink.connector.rocketmq.legacy.common.util; 18 | 19 | import org.apache.rocketmq.client.AccessChannel; 20 | import org.apache.rocketmq.common.message.MessageQueue; 21 | 22 | import java.lang.management.ManagementFactory; 23 | import java.util.ArrayList; 24 | import java.util.Collection; 25 | import java.util.Collections; 26 | import java.util.List; 27 | import java.util.Properties; 28 | 29 | public final class RocketMQUtils { 30 | 31 | public static int getInteger(Properties props, String key, int defaultValue) { 32 | return Integer.parseInt(props.getProperty(key, String.valueOf(defaultValue))); 33 | } 34 | 35 | public static long getLong(Properties props, String key, long defaultValue) { 36 | return Long.parseLong(props.getProperty(key, String.valueOf(defaultValue))); 37 | } 38 | 39 | public static boolean getBoolean(Properties props, String key, boolean defaultValue) { 40 | return Boolean.parseBoolean(props.getProperty(key, String.valueOf(defaultValue))); 41 | } 42 | 43 | public static AccessChannel getAccessChannel( 44 | Properties props, String key, AccessChannel defaultValue) { 45 | return AccessChannel.valueOf(props.getProperty(key, String.valueOf(defaultValue))); 46 | } 47 | 48 | public static String getInstanceName(String... args) { 49 | if (null != args && args.length > 0) { 50 | return String.join("_", args); 51 | } 52 | return ManagementFactory.getRuntimeMXBean().getName() + "_" + System.nanoTime(); 53 | } 54 | 55 | /** 56 | * Average Hashing queue algorithm Refer: 57 | * org.apache.rocketmq.client.consumer.rebalance.AllocateStrategyByAveragely 58 | */ 59 | public static List allocate( 60 | Collection mqSet, int numberOfParallelTasks, int indexOfThisTask) { 61 | ArrayList mqAll = new ArrayList<>(mqSet); 62 | Collections.sort(mqAll); 63 | List result = new ArrayList<>(); 64 | int mod = mqAll.size() % numberOfParallelTasks; 65 | int averageSize = 66 | mqAll.size() <= numberOfParallelTasks 67 | ? 1 68 | : (mod > 0 && indexOfThisTask < mod 69 | ? mqAll.size() / numberOfParallelTasks + 1 70 | : mqAll.size() / numberOfParallelTasks); 71 | int startIndex = 72 | (mod > 0 && indexOfThisTask < mod) 73 | ? indexOfThisTask * averageSize 74 | : indexOfThisTask * averageSize + mod; 75 | int range = Math.min(averageSize, mqAll.size() - startIndex); 76 | for (int i = 0; i < range; i++) { 77 | result.add(mqAll.get((startIndex + i) % mqAll.size())); 78 | } 79 | return result; 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/source/reader/RocketMQRecordEmitterTest.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.rocketmq.source.reader; 20 | 21 | import org.apache.flink.api.common.eventtime.Watermark; 22 | import org.apache.flink.api.connector.source.ReaderOutput; 23 | import org.apache.flink.api.connector.source.SourceOutput; 24 | import org.apache.flink.connector.rocketmq.source.reader.deserializer.RocketMQSchemaWrapper; 25 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit; 26 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplitState; 27 | import org.apache.flink.table.data.GenericRowData; 28 | import org.apache.flink.table.data.RowData; 29 | 30 | import org.apache.rocketmq.common.message.MessageExt; 31 | import org.junit.Test; 32 | 33 | import java.io.IOException; 34 | import java.net.InetSocketAddress; 35 | 36 | /** Test for {@link RocketMQRecordEmitter}. */ 37 | public class RocketMQRecordEmitterTest { 38 | 39 | @Test 40 | public void testEmitRecord() throws IOException { 41 | RocketMQRecordEmitter recordEmitter = 42 | new RocketMQRecordEmitter<>(new RocketMQSchemaWrapper<>()); 43 | MessageExt message = 44 | new MessageExt( 45 | 1, 46 | System.currentTimeMillis(), 47 | InetSocketAddress.createUnresolved("localhost", 8080), 48 | System.currentTimeMillis(), 49 | InetSocketAddress.createUnresolved("localhost", 8088), 50 | "184019387"); 51 | message.setBody("test_emit_record_message".getBytes()); 52 | GenericRowData rowData = new GenericRowData(1); 53 | rowData.setField(0, message.getBody()); 54 | String topic = "test-record-emitter"; 55 | String broker = "taobaodaily"; 56 | int partition = 256; 57 | long startingOffset = 100; 58 | long stoppingTimestamp = System.currentTimeMillis(); 59 | 60 | MessageView messageView = new MessageViewExt(message); 61 | RocketMQSourceSplitState partitionSplitState = 62 | new RocketMQSourceSplitState( 63 | new RocketMQSourceSplit(topic, broker, partition, startingOffset, -1L)); 64 | 65 | recordEmitter.emitRecord(messageView, new TestingEmitterOutput<>(), partitionSplitState); 66 | } 67 | 68 | private static final class TestingEmitterOutput implements ReaderOutput { 69 | 70 | private TestingEmitterOutput() {} 71 | 72 | public void collect(E record) {} 73 | 74 | public void collect(E record, long timestamp) { 75 | this.collect(record); 76 | } 77 | 78 | public void emitWatermark(Watermark watermark) { 79 | throw new UnsupportedOperationException(); 80 | } 81 | 82 | public void markIdle() { 83 | throw new UnsupportedOperationException(); 84 | } 85 | 86 | @Override 87 | public void markActive() {} 88 | 89 | public SourceOutput createOutputForSplit(String splitId) { 90 | return this; 91 | } 92 | 93 | public void releaseOutputForSplit(String splitId) {} 94 | } 95 | } 96 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/reader/deserializer/RocketMQDeserializationSchema.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.rocketmq.source.reader.deserializer; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.api.common.serialization.DeserializationSchema; 23 | import org.apache.flink.api.common.serialization.DeserializationSchema.InitializationContext; 24 | import org.apache.flink.api.java.typeutils.ResultTypeQueryable; 25 | import org.apache.flink.connector.rocketmq.source.reader.MessageView; 26 | import org.apache.flink.util.Collector; 27 | 28 | import java.io.IOException; 29 | import java.io.Serializable; 30 | 31 | /** An interface for the deserialization of RocketMQ records. */ 32 | @PublicEvolving 33 | public interface RocketMQDeserializationSchema extends Serializable, ResultTypeQueryable { 34 | 35 | /** 36 | * Initialization method for the schema. It is called before the actual working methods {@link 37 | * #deserialize} and thus suitable for one time setup work. 38 | * 39 | *

The provided {@link InitializationContext} can be used to access additional features such 40 | * as e.g. registering user metrics. 41 | * 42 | * @param context Contextual information that can be used during initialization. 43 | */ 44 | default void open(DeserializationSchema.InitializationContext context) throws Exception { 45 | // Nothing to do here for the default implementation. 46 | } 47 | 48 | /** 49 | * Deserializes the byte message. 50 | * 51 | *

Can output multiple records through the {@link Collector}. Note that number and size of 52 | * the produced records should be relatively small. Depending on the source implementation 53 | * records can be buffered in memory or collecting records might delay emitting checkpoint 54 | * barrier. 55 | * 56 | * @param messageView The MessageView to deserialize. 57 | * @param out The collector to put the resulting messages. 58 | */ 59 | void deserialize(MessageView messageView, Collector out) throws IOException; 60 | 61 | /** 62 | * Create a RocketMQDeserializationSchema by using the flink's {@link DeserializationSchema}. It 63 | * would consume the rocketmq message as byte array and decode the message by using flink's 64 | * logic. 65 | */ 66 | static RocketMQDeserializationSchema flinkSchema( 67 | DeserializationSchema deserializationSchema) { 68 | return new RocketMQDeserializationSchemaWrapper<>(deserializationSchema); 69 | } 70 | 71 | /** 72 | * Wraps a {@link DeserializationSchema} as the value deserialization schema. The other fields 73 | * such as key, headers, timestamp are ignored. 74 | * 75 | * @param deserializationSchema the {@link DeserializationSchema} used to deserialize the value 76 | * of a {@link RocketMQDeserializationSchemaWrapper}. 77 | * @param the type of the deserialized record. 78 | */ 79 | static RocketMQDeserializationSchema flinkBodyOnlySchema( 80 | DeserializationSchema deserializationSchema) { 81 | return new RocketMQDeserializationSchemaWrapper<>(deserializationSchema); 82 | } 83 | 84 | static RocketMQDeserializationSchema rocketMQSchema( 85 | DeserializationSchema valueDeserializationSchema) { 86 | return new RocketMQDeserializationSchemaWrapper<>(valueDeserializationSchema); 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /src/test/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumStateSerializerTest.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.rocketmq.source.enumerator; 20 | 21 | import org.apache.flink.api.connector.source.SplitsAssignment; 22 | import org.apache.flink.connector.rocketmq.source.split.RocketMQSourceSplit; 23 | 24 | import org.junit.Assert; 25 | import org.junit.Test; 26 | 27 | import java.io.IOException; 28 | import java.util.Arrays; 29 | import java.util.HashMap; 30 | import java.util.HashSet; 31 | 32 | /** Test for {@link RocketMQSourceEnumStateSerializer}. */ 33 | public class RocketMQSourceEnumStateSerializerTest { 34 | 35 | @Test 36 | public void testSerializeDeserializeSourceEnumState() throws IOException { 37 | RocketMQSourceEnumStateSerializer serializer = new RocketMQSourceEnumStateSerializer(); 38 | RocketMQSourceEnumState expected = prepareSourceEnumeratorState(); 39 | assert expected != null; 40 | RocketMQSourceEnumState actual = serializer.deserialize(0, serializer.serialize(expected)); 41 | Assert.assertEquals( 42 | expected.getCurrentSplitAssignment(), actual.getCurrentSplitAssignment()); 43 | } 44 | 45 | private RocketMQSourceEnumState prepareSourceEnumeratorState() { 46 | SplitsAssignment pendingAssignment = 47 | new SplitsAssignment<>(new HashMap<>()); 48 | pendingAssignment 49 | .assignment() 50 | .put( 51 | 0, 52 | Arrays.asList( 53 | new RocketMQSourceSplit( 54 | "0", "taobaodaily-01", 1, 0, System.currentTimeMillis()), 55 | new RocketMQSourceSplit( 56 | "3", "taobaodaily-01", 2, 0, System.currentTimeMillis()), 57 | new RocketMQSourceSplit( 58 | "6", "taobaodaily-01", 3, 0, System.currentTimeMillis()), 59 | new RocketMQSourceSplit( 60 | "9", "taobaodaily-01", 4, 0, System.currentTimeMillis()))); 61 | pendingAssignment 62 | .assignment() 63 | .put( 64 | 1, 65 | Arrays.asList( 66 | new RocketMQSourceSplit( 67 | "1", "taobaodaily-02", 5, 0, System.currentTimeMillis()), 68 | new RocketMQSourceSplit( 69 | "4", "taobaodaily-02", 6, 0, System.currentTimeMillis()), 70 | new RocketMQSourceSplit( 71 | "7", "taobaodaily-02", 7, 0, System.currentTimeMillis()))); 72 | pendingAssignment 73 | .assignment() 74 | .put( 75 | 2, 76 | Arrays.asList( 77 | new RocketMQSourceSplit( 78 | "2", "taobaodaily-03", 8, 0, System.currentTimeMillis()), 79 | new RocketMQSourceSplit( 80 | "5", "taobaodaily-03", 9, 0, System.currentTimeMillis()), 81 | new RocketMQSourceSplit( 82 | "8", "taobaodaily-03", 10, 0, System.currentTimeMillis()))); 83 | 84 | return new RocketMQSourceEnumState(new HashSet<>()); 85 | } 86 | } 87 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/common/config/RocketMQConfiguration.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.rocketmq.common.config; 19 | 20 | import org.apache.flink.configuration.ConfigOption; 21 | import org.apache.flink.configuration.ConfigOptions; 22 | import org.apache.flink.configuration.Configuration; 23 | import org.apache.flink.configuration.UnmodifiableConfiguration; 24 | 25 | import java.util.HashMap; 26 | import java.util.List; 27 | import java.util.Map; 28 | import java.util.function.Consumer; 29 | import java.util.function.Function; 30 | 31 | import static java.util.function.Function.identity; 32 | import static java.util.stream.Collectors.toList; 33 | 34 | /** 35 | * An unmodifiable {@link Configuration} for RocketMQ. We provide extra methods for building the 36 | * different RocketMQ client instance. 37 | */ 38 | public class RocketMQConfiguration extends UnmodifiableConfiguration { 39 | 40 | private static final long serialVersionUID = 3050894147145572345L; 41 | 42 | /** 43 | * Creates a new RocketMQConfiguration, which holds a copy of the given configuration that can't 44 | * be altered. 45 | * 46 | * @param config The configuration with the original contents. 47 | */ 48 | protected RocketMQConfiguration(Configuration config) { 49 | super(config); 50 | } 51 | 52 | /** 53 | * Get the option value by a prefix. We would return an empty map if the option doesn't exist. 54 | */ 55 | public Map getProperties(ConfigOption> option) { 56 | Map properties = new HashMap<>(); 57 | if (contains(option)) { 58 | Map map = get(option); 59 | properties.putAll(map); 60 | } 61 | 62 | // Filter the sub config option. These options could be provided by SQL. 63 | String prefix = option.key() + "."; 64 | List keys = 65 | keySet().stream() 66 | .filter(key -> key.startsWith(prefix) && key.length() > prefix.length()) 67 | .collect(toList()); 68 | 69 | // Put these config options' value into return result. 70 | for (String key : keys) { 71 | ConfigOption o = ConfigOptions.key(key).stringType().noDefaultValue(); 72 | String value = get(o); 73 | properties.put(key.substring(prefix.length()), value); 74 | } 75 | 76 | return properties; 77 | } 78 | 79 | /** Get an option value from the given config, convert it into a new value instance. */ 80 | public T get(ConfigOption option, Function convertor) { 81 | F value = get(option); 82 | if (value != null) { 83 | return convertor.apply(value); 84 | } else { 85 | return null; 86 | } 87 | } 88 | 89 | /** Set the config option's value to a given builder. */ 90 | public void useOption(ConfigOption option, Consumer setter) { 91 | useOption(option, identity(), setter); 92 | } 93 | 94 | /** 95 | * Query the config option's value, convert it into a required type, set it to a given builder. 96 | */ 97 | public void useOption( 98 | ConfigOption option, Function convertor, Consumer setter) { 99 | if (contains(option) || option.hasDefaultValue()) { 100 | V value = get(option, convertor); 101 | setter.accept(value); 102 | } 103 | } 104 | } 105 | -------------------------------------------------------------------------------- /src/main/java/org/apache/flink/connector/rocketmq/source/enumerator/RocketMQSourceEnumStateSerializer.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.rocketmq.source.enumerator; 20 | 21 | import org.apache.flink.core.io.SimpleVersionedSerializer; 22 | 23 | import com.alibaba.fastjson.JSON; 24 | import org.apache.rocketmq.common.message.MessageQueue; 25 | import org.slf4j.Logger; 26 | import org.slf4j.LoggerFactory; 27 | 28 | import java.io.ByteArrayInputStream; 29 | import java.io.ByteArrayOutputStream; 30 | import java.io.DataInputStream; 31 | import java.io.DataOutputStream; 32 | import java.io.IOException; 33 | import java.util.HashSet; 34 | import java.util.Set; 35 | 36 | /** The {@link SimpleVersionedSerializer Serializer} for the enumerator state of RocketMQ source. */ 37 | public class RocketMQSourceEnumStateSerializer 38 | implements SimpleVersionedSerializer { 39 | 40 | private static final Logger LOG = 41 | LoggerFactory.getLogger(RocketMQSourceEnumStateSerializer.class); 42 | 43 | private static final int CURRENT_VERSION = 0; 44 | 45 | @Override 46 | public int getVersion() { 47 | return CURRENT_VERSION; 48 | } 49 | 50 | @Override 51 | public byte[] serialize(RocketMQSourceEnumState enumState) throws IOException { 52 | Set assignments = enumState.getCurrentSplitAssignment(); 53 | 54 | LOG.info("Serializer enumerator state, {}", JSON.toJSONString(enumState)); 55 | 56 | try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); 57 | DataOutputStream out = new DataOutputStream(baos)) { 58 | 59 | out.writeInt(assignments.size()); 60 | for (MessageQueue assignment : assignments) { 61 | out.writeUTF(assignment.getBrokerName()); 62 | out.writeUTF(assignment.getTopic()); 63 | out.writeInt(assignment.getQueueId()); 64 | } 65 | out.flush(); 66 | 67 | return baos.toByteArray(); 68 | } 69 | } 70 | 71 | @Override 72 | public RocketMQSourceEnumState deserialize(int version, byte[] serialized) throws IOException { 73 | // Check whether the version of serialized bytes is supported. 74 | if (version == CURRENT_VERSION) { 75 | Set queues = deserializeMessageQueue(serialized); 76 | return new RocketMQSourceEnumState(queues); 77 | } 78 | throw new IOException( 79 | String.format( 80 | "The bytes are serialized with version %d, " 81 | + "while this deserializer only supports version up to %d", 82 | version, getVersion())); 83 | } 84 | 85 | private Set deserializeMessageQueue(byte[] serialized) throws IOException { 86 | try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); 87 | DataInputStream in = new DataInputStream(bais)) { 88 | 89 | int size = in.readInt(); 90 | HashSet result = new HashSet<>(); 91 | for (int i = 0; i < size; i++) { 92 | String brokerName = in.readUTF(); 93 | String topic = in.readUTF(); 94 | int queueId = in.readInt(); 95 | 96 | MessageQueue queue = new MessageQueue(topic, brokerName, queueId); 97 | result.add(queue); 98 | } 99 | 100 | return result; 101 | } 102 | } 103 | } 104 | --------------------------------------------------------------------------------