diff --git a/flink-connector-rabbitmq/README.md b/flink-connector-rabbitmq/README.md new file mode 100644 index 00000000000..f66e374a75f --- /dev/null +++ b/flink-connector-rabbitmq/README.md @@ -0,0 +1,161 @@ +# License of the RabbitMQ Connector + +Flink's RabbitMQ connector defines a Maven dependency on the +"RabbitMQ AMQP Java Client", is triple-licensed under the Mozilla Public License 1.1 ("MPL"), +the GNU General Public License version 2 ("GPL") and the Apache License version 2 ("ASL"). + +Flink itself neither reuses source code from the "RabbitMQ AMQP Java Client" +nor packages binaries from the "RabbitMQ AMQP Java Client". + +Users that create and publish derivative work based on Flink's +RabbitMQ connector (thereby re-distributing the "RabbitMQ AMQP Java Client") +must be aware that this may be subject to conditions declared in the +Mozilla Public License 1.1 ("MPL"), the GNU General Public License version 2 ("GPL") +and the Apache License version 2 ("ASL"). + +This connector allows consuming messages from and publishing to RabbitMQ. It implements the +Source API specified in [FLIP-27](https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface) +and the Sink API specified in [FLIP-143](https://cwiki.apache.org/confluence/display/FLINK/FLIP-143%3A+Unified+Sink+API). + +For more information about RabbitMQ visit https://www.rabbitmq.com/. + +# RabbitMQ Source + +Flink's RabbitMQ connector provides a streaming-only source which enables you to receive messages +from a RabbitMQ queue in three different consistency modes: at-most-once, at-least-once, +and exactly-once. + +## Consistency Modes + +With **at-most-once**, the source will receive each message and automatically acknowledges it to +RabbitMQ. The message content is then polled by the output. If the system crashes in the meantime, +the messages that the source buffers are lost. + +By contrast, the messages in the **at-least-once** mode are not automatically acknowledged but +instead the delivery tag is stored in order to acknowledge it later to RabbitMQ. Messages are polled +by the output and when the notification for a completed checkpoint is received the messages that were +polled are acknowledged to RabbitMQ. Therefore, the mode requires _checkpointing enabled_. This way, +it is assured that the messages are correctly processed by the system. If the system crashes in the +meantime, the unacknowledged messages will be resend by RabbitMQ to assure at-least-once behavior. + +The **exactly-once-mode** mode uses _correlation ids_ to deduplicate messages. Correlation ids are +properties of the messages and need to be set by the message publisher (who publishes the messages +to RabbitMQ) in order for the mode to function. The user has the obligation to ensure that the set +correlation id for a message is unique, otherwise no exactly-once can be guaranteed here since +RabbitMQ itself has no support for automatic exactly-once ids or the required behavior. In addition, +it requires _checkpointing enabled_and only \_parallelism 1_ is allowed. Similar to at-least-once, +the messages are received from RabbitMQ,buffered, and passed to the output when polled. A set of +seen correlation ids is maintained to apply the deduplication. During a checkpoint, the seen +correlation ids are stored so that in case of failure they can be recovered and used for +deduplication. When the notification for a completed checkpoint is received, all polled messages are +acknowledged as one transaction to ensure the reception by RabbitMQ. Afterwards, the set of +correlation ids is updated as RabbitMQ will not send the acknowledged messages again. This behavior +assures exactly-once processing but also has a performance drawback. Committing many messages will +take time and will thus increase the overall time it takes to do a checkpoint. This can result in +checkpoint delays and in peaks where checkpoint have either many or just a few messages. + +## How to use it + +```java +public class Main { + public static void main(String[]args) { + + RabbitMQSource source = + RabbitMQSource.builder() + .setConnectionConfig(RMQ_CONNECTION_CONFIG) + .setQueueName(RABBITMQ_QUEUE_NAME) + .setDeserializationSchema(DESERIALIZATION_SCHEMA) + .setConsistencyMode(CONSISTENCY_MODE) + .build(); + + // ******************* An example usage looks like this ******************* + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + RMQConnectionConfig rmqConnectionConfig = + new RMQConnectionConfig.Builder() + .setHost("localhost") + .setVirtualHost("/") + .setUserName("guest") + .setPassword("guest") + .setPort(5672) + .build(); + + RabbitMQSource rmqSource = + RabbitMQSource.builder() + .setConnectionConfig(rmqConnectionConfig) + .setQueueName("consume-queue") + .setDeserializationSchema(new SimpleStringSchema()) + .setConsistencyMode(ConsistencyMode.AT_MOST_ONCE) + .build(); + + DataStream stream = env.fromSource(rmqSource, WatermarkStrategy.noWatermarks(), "RMQSource"); + } +} +``` + +# RabbitMQ Sink + +Flink's RabbitMQ connector provides a sink which enables you to publish your stream directly +to a RabbitMQ exchange in three different consistency modes: at-most-once, at-least-once, +and exactly-once. Furthermore, user defined publish options can be used to customize each message +options in regard to exchange and publish settings in the RabbitMQ context. + +## Consistency Mode + +With **at-most-once**, the sink will simply take each message and publish the serialization of it +(with publish options if given) to RabbitMQ. At this point the sink gives up the ownership of the message. + +For **at-least-once** the same process as for at-most-once is executed except that the ownership of +the message does not end immediately with publishing it. The sink will keep the individual publishing +id for each message as well as the message itself and buffer it as long as it takes to receive the +message received acknowledgment from RabbitMQ. Since we are in the need of buffering messages and waiting +for their asynchronous acknowledgment, this requires _checkpointing enabled_. On each checkpoint, +all to this point buffered (and thus send) but unacknowledged messages will be stored. Simultaneously, +on each checkpoint a resend will be triggered to send all unacknowledged messages once again since +we have to assume that something went wrong for it during the publishing process. Since it can take a +moment until messages get acknowledged from RabbitMQ this can and probably will result in a message +duplication and therefore this logic becomes at-least-once. + +By contrast, the **exactly-once-mode** mode will not send messages on receive. All incoming messages +will be buffered until a checkpoint is triggered. On each checkpoint all messages will be +published/committed as one transaction to ensure the reception acknowledge by RabbitMQ. +If successful, all messages which were committed will be given up, otherwise they will be stored +and tried to commit again in the next transaction during the next checkpoint. +This consistency mode ensures that each message will be stored in RabbitMQ exactly once but also has +a performance drawback. Committing many messages will take time and will thus increase the overall +time it takes to do a checkpoint. This can result in checkpoint delays and in peaks where +checkpoint have either many or just a few messages. This also correlates to the latency of each message. + +## How to use it + +```java +RabbitMQSink sink = + RabbitMQSink.builder() + .setConnectionConfig() + .setQueueName() + .setSerializationSchema() + .setConsistencyMode() + .build(); + +// ******************* An example usage looks like this ******************* + +RMQConnectionConfig rmqConnectionConfig = + new RMQConnectionConfig.Builder() + .setHost("localhost") + .setVirtualHost("/") + .setUserName("guest") + .setPassword("guest") + .setPort(5672) + .build(); + +RabbitMQSink rmqSink = + RabbitMQSink.builder() + .setConnectionConfig(rmqConnectionConfig) + .setQueueName("publish-queue") + .setSerializationSchema(new SimpleStringSchema()) + .setConsistencyMode(ConsistencyMode.AT_MOST_ONCE) + .build(); + +(DataStream).sinkTo(rmqSink) +``` diff --git a/flink-connector-rabbitmq/pom.xml b/flink-connector-rabbitmq/pom.xml new file mode 100644 index 00000000000..e4d1b075b4b --- /dev/null +++ b/flink-connector-rabbitmq/pom.xml @@ -0,0 +1,119 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.16-SNAPSHOT + + + flink-connector-rabbitmq + Flink : Connectors : RabbitMQ + + jar + + + + 5.9.0 + + + + + + + + org.apache.flink + flink-connector-base + ${flink.version} + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + + + + com.rabbitmq + amqp-client + ${rabbitmq.version} + + + + + + org.testcontainers + rabbitmq + 1.15.1 + test + + + + org.apache.flink + flink-test-utils + test + + + + org.apache.flink + flink-connector-test-utils + ${flink.version} + test + + + + org.apache.flink + flink-runtime + ${flink.version} + test-jar + test + + + + org.apache.flink + flink-streaming-java + ${flink.version} + test + test-jar + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + + diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/common/ConsistencyMode.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/common/ConsistencyMode.java new file mode 100644 index 00000000000..2575cc5e46a --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/common/ConsistencyMode.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.common; + +/** + * The different consistency modes that can be defined for the sink and source individually. + * + *

The available consistency modes are as follows. + * + *

    + *
  • AT_MOST_ONCE Messages are consumed by the output once or never. + *
  • AT_LEAST_ONCE Messages are consumed by the output at least once. + *
  • EXACTLY_ONCE Messages are consumed by the output exactly once. + *
+ * + *

Note that the higher the consistency guarantee gets, fewer messages can be processed by the + * system. At-least-once and exactly-once should only be used if necessary. + */ +public enum ConsistencyMode { + AT_MOST_ONCE, + AT_LEAST_ONCE, + EXACTLY_ONCE, +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/common/RabbitMQConnectionConfig.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/common/RabbitMQConnectionConfig.java new file mode 100644 index 00000000000..e1b237f9319 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/common/RabbitMQConnectionConfig.java @@ -0,0 +1,551 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.common; + +import org.apache.flink.connector.rabbitmq.source.RabbitMQSource; +import org.apache.flink.util.Preconditions; + +import com.rabbitmq.client.ConnectionFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.net.URISyntaxException; +import java.security.KeyManagementException; +import java.security.NoSuchAlgorithmException; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +/** + * This class is copied from the previous RabbitMQ connector. Connection Configuration for RMQ. If + * {@link Builder#setUri(String)} has been set then {@link + * RabbitMQConnectionConfig#RabbitMQConnectionConfig(String, Integer, Boolean, Boolean, Integer, + * Integer, Integer, Integer, Integer)} will be used to initialize the RMQ connection or {@link + * RabbitMQConnectionConfig#RabbitMQConnectionConfig(String, Integer, String, String, String, + * Integer, Boolean, Boolean, Integer, Integer, Integer, Integer, Integer)} will be used to + * initialize the RMQ connection. + */ +public class RabbitMQConnectionConfig implements Serializable { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(RabbitMQConnectionConfig.class); + + private String host; + private Integer port; + private String virtualHost; + private String username; + private String password; + private String uri; + + private Integer networkRecoveryInterval; + private Boolean automaticRecovery; + private Boolean topologyRecovery; + + private Integer connectionTimeout; + private Integer requestedChannelMax; + private Integer requestedFrameMax; + private Integer requestedHeartbeat; + + private Integer prefetchCount; + + /** + * @param host host name + * @param port port + * @param virtualHost virtual host + * @param username username + * @param password password + * @param networkRecoveryInterval connection recovery interval in milliseconds + * @param automaticRecovery if automatic connection recovery + * @param topologyRecovery if topology recovery + * @param connectionTimeout connection timeout + * @param requestedChannelMax requested maximum channel number + * @param requestedFrameMax requested maximum frame size + * @param requestedHeartbeat requested heartbeat interval + * @throws NullPointerException if host or virtual host or username or password is null + */ + private RabbitMQConnectionConfig( + String host, + Integer port, + String virtualHost, + String username, + String password, + Integer networkRecoveryInterval, + Boolean automaticRecovery, + Boolean topologyRecovery, + Integer connectionTimeout, + Integer requestedChannelMax, + Integer requestedFrameMax, + Integer requestedHeartbeat, + Integer prefetchCount) { + this.host = requireNonNull(host); + this.port = requireNonNull(port); + this.virtualHost = requireNonNull(virtualHost); + this.username = requireNonNull(username); + this.password = requireNonNull(password); + + this.networkRecoveryInterval = networkRecoveryInterval; + this.automaticRecovery = automaticRecovery; + this.topologyRecovery = topologyRecovery; + this.connectionTimeout = connectionTimeout; + this.requestedChannelMax = requestedChannelMax; + this.requestedFrameMax = requestedFrameMax; + this.requestedHeartbeat = requestedHeartbeat; + this.prefetchCount = prefetchCount; + } + + /** + * @param uri the connection URI + * @param networkRecoveryInterval connection recovery interval in milliseconds + * @param automaticRecovery if automatic connection recovery + * @param topologyRecovery if topology recovery + * @param connectionTimeout connection timeout + * @param requestedChannelMax requested maximum channel number + * @param requestedFrameMax requested maximum frame size + * @param requestedHeartbeat requested heartbeat interval + * @throws NullPointerException if URI is null + */ + private RabbitMQConnectionConfig( + String uri, + Integer networkRecoveryInterval, + Boolean automaticRecovery, + Boolean topologyRecovery, + Integer connectionTimeout, + Integer requestedChannelMax, + Integer requestedFrameMax, + Integer requestedHeartbeat, + Integer prefetchCount) { + Preconditions.checkNotNull(uri, "Uri can not be null"); + this.uri = uri; + + this.networkRecoveryInterval = networkRecoveryInterval; + this.automaticRecovery = automaticRecovery; + this.topologyRecovery = topologyRecovery; + this.connectionTimeout = connectionTimeout; + this.requestedChannelMax = requestedChannelMax; + this.requestedFrameMax = requestedFrameMax; + this.requestedHeartbeat = requestedHeartbeat; + this.prefetchCount = prefetchCount; + } + + /** @return the host to use for connections */ + public String getHost() { + return host; + } + + /** @return the port to use for connections */ + public int getPort() { + return port; + } + + /** + * Retrieve the virtual host. + * + * @return the virtual host to use when connecting to the broker + */ + public String getVirtualHost() { + return virtualHost; + } + + /** + * Retrieve the user name. + * + * @return the AMQP user name to use when connecting to the broker + */ + public String getUsername() { + return username; + } + + /** + * Retrieve the password. + * + * @return the password to use when connecting to the broker + */ + public String getPassword() { + return password; + } + + /** + * Retrieve the URI. + * + * @return the connection URI when connecting to the broker + */ + public String getUri() { + return uri; + } + + /** + * Returns automatic connection recovery interval in milliseconds. + * + * @return how long will automatic recovery wait before attempting to reconnect, in ms; default + * is 5000 + */ + public Integer getNetworkRecoveryInterval() { + return networkRecoveryInterval; + } + + /** + * Returns true if automatic connection recovery is enabled, false otherwise. + * + * @return true if automatic connection recovery is enabled, false otherwise + */ + public Boolean isAutomaticRecovery() { + return automaticRecovery; + } + + /** + * Returns true if topology recovery is enabled, false otherwise. + * + * @return true if topology recovery is enabled, false otherwise + */ + public Boolean isTopologyRecovery() { + return topologyRecovery; + } + + /** + * Retrieve the connection timeout. + * + * @return the connection timeout, in milliseconds; zero for infinite + */ + public Integer getConnectionTimeout() { + return connectionTimeout; + } + + /** + * Retrieve the requested maximum channel number. + * + * @return the initially requested maximum channel number; zero for unlimited + */ + public Integer getRequestedChannelMax() { + return requestedChannelMax; + } + + /** + * Retrieve the requested maximum frame size. + * + * @return the initially requested maximum frame size, in octets; zero for unlimited + */ + public Integer getRequestedFrameMax() { + return requestedFrameMax; + } + + /** + * Retrieve the requested heartbeat interval. + * + * @return the initially requested heartbeat interval, in seconds; zero for none + */ + public Integer getRequestedHeartbeat() { + return requestedHeartbeat; + } + + /** + * Retrieve the the channel prefetch count. + * + * @return an Optional of the prefetch count, if set, for the consumer channel + */ + public Optional getPrefetchCount() { + return Optional.ofNullable(prefetchCount); + } + + /** + * @return Connection Factory for RMQ + * @throws URISyntaxException if Malformed URI has been passed + * @throws NoSuchAlgorithmException if the ssl factory could not be created + * @throws KeyManagementException if the ssl context could not be initialized + */ + public ConnectionFactory getConnectionFactory() + throws URISyntaxException, NoSuchAlgorithmException, KeyManagementException { + ConnectionFactory factory = new ConnectionFactory(); + if (this.uri != null && !this.uri.isEmpty()) { + try { + factory.setUri(this.uri); + } catch (URISyntaxException e) { + LOG.error("Failed to parse uri", e); + throw e; + } catch (KeyManagementException e) { + // this should never happen + LOG.error("Failed to initialize ssl context.", e); + throw e; + } catch (NoSuchAlgorithmException e) { + // this should never happen + LOG.error("Failed to setup ssl factory.", e); + throw e; + } + } else { + factory.setHost(this.host); + factory.setPort(this.port); + factory.setVirtualHost(this.virtualHost); + factory.setUsername(this.username); + factory.setPassword(this.password); + } + + if (this.automaticRecovery != null) { + factory.setAutomaticRecoveryEnabled(this.automaticRecovery); + } + if (this.connectionTimeout != null) { + factory.setConnectionTimeout(this.connectionTimeout); + } + if (this.networkRecoveryInterval != null) { + factory.setNetworkRecoveryInterval(this.networkRecoveryInterval); + } + if (this.requestedHeartbeat != null) { + factory.setRequestedHeartbeat(this.requestedHeartbeat); + } + if (this.topologyRecovery != null) { + factory.setTopologyRecoveryEnabled(this.topologyRecovery); + } + if (this.requestedChannelMax != null) { + factory.setRequestedChannelMax(this.requestedChannelMax); + } + if (this.requestedFrameMax != null) { + factory.setRequestedFrameMax(this.requestedFrameMax); + } + + return factory; + } + + /** The Builder Class for {@link RabbitMQConnectionConfig}. */ + public static class Builder { + + private String host; + private Integer port; + private String virtualHost; + private String username; + private String password; + + private Integer networkRecoveryInterval; + private Boolean automaticRecovery; + private Boolean topologyRecovery; + + private Integer connectionTimeout; + private Integer requestedChannelMax; + private Integer requestedFrameMax; + private Integer requestedHeartbeat; + + // basicQos options for consumers + private Integer prefetchCount; + + private String uri; + + /** + * Set the target port. + * + * @param port the default port to use for connections + * @return the Builder + */ + public Builder setPort(int port) { + this.port = port; + return this; + } + + /** + * @param host the default host to use for connections + * @return the Builder + */ + public Builder setHost(String host) { + this.host = host; + return this; + } + + /** + * Set the virtual host. + * + * @param virtualHost the virtual host to use when connecting to the broker + * @return the Builder + */ + public Builder setVirtualHost(String virtualHost) { + this.virtualHost = virtualHost; + return this; + } + + /** + * Set the user name. + * + * @param username the AMQP user name to use when connecting to the broker + * @return the Builder + */ + public Builder setUserName(String username) { + this.username = username; + return this; + } + + /** + * Set the password. + * + * @param password the password to use when connecting to the broker + * @return the Builder + */ + public Builder setPassword(String password) { + this.password = password; + return this; + } + + /** + * Convenience method for setting the fields in an AMQP URI: host, port, username, password + * and virtual host. If any part of the URI is omitted, the ConnectionFactory's + * corresponding variable is left unchanged. + * + * @param uri is the AMQP URI containing the data + * @return the Builder + */ + public Builder setUri(String uri) { + this.uri = uri; + return this; + } + + /** + * Enables or disables topology recovery. + * + * @param topologyRecovery if true, enables topology recovery + * @return the Builder + */ + public Builder setTopologyRecoveryEnabled(boolean topologyRecovery) { + this.topologyRecovery = topologyRecovery; + return this; + } + + /** + * Set the requested heartbeat. + * + * @param requestedHeartbeat the initially requested heartbeat interval, in seconds; zero + * for none + * @return the Builder + */ + public Builder setRequestedHeartbeat(int requestedHeartbeat) { + this.requestedHeartbeat = requestedHeartbeat; + return this; + } + + /** + * Set the requested maximum frame size. + * + * @param requestedFrameMax initially requested maximum frame size, in octets; zero for + * unlimited + * @return the Builder + */ + public Builder setRequestedFrameMax(int requestedFrameMax) { + this.requestedFrameMax = requestedFrameMax; + return this; + } + + /** + * Set the requested maximum channel number. + * + * @param requestedChannelMax initially requested maximum channel number; zero for unlimited + */ + public Builder setRequestedChannelMax(int requestedChannelMax) { + this.requestedChannelMax = requestedChannelMax; + return this; + } + + /** + * Sets connection recovery interval. Default is 5000. + * + * @param networkRecoveryInterval how long will automatic recovery wait before attempting to + * reconnect, in ms + * @return the Builder + */ + public Builder setNetworkRecoveryInterval(int networkRecoveryInterval) { + this.networkRecoveryInterval = networkRecoveryInterval; + return this; + } + + /** + * Set the connection timeout. + * + * @param connectionTimeout connection establishment timeout in milliseconds; zero for + * infinite + * @return the Builder + */ + public Builder setConnectionTimeout(int connectionTimeout) { + this.connectionTimeout = connectionTimeout; + return this; + } + + /** + * Enables or disables automatic connection recovery. + * + * @param automaticRecovery if true, enables connection recovery + * @return the Builder + */ + public Builder setAutomaticRecovery(boolean automaticRecovery) { + this.automaticRecovery = automaticRecovery; + return this; + } + + /** + * Enables setting basicQos for the consumer channel. Only applicable to the {@link + * RabbitMQSource}. Set to 0 for unlimited, which is the default. + * + * @see Consumer Prefetch + * @see Channel + * Prefetch (QoS) + * @param prefetchCount the max number of messages to receive without acknowledgement. + * @return the Builder + */ + public Builder setPrefetchCount(int prefetchCount) { + this.prefetchCount = prefetchCount; + return this; + } + + /** + * The Builder method. + * + *

If URI is NULL we use host, port, vHost, username, password combination to initialize + * connection. using {@link RabbitMQConnectionConfig#RabbitMQConnectionConfig(String, + * Integer, String, String, String, Integer, Boolean, Boolean, Integer, Integer, Integer, + * Integer, Integer)}. + * + *

Otherwise the URI will be used to initialize the client connection {@link + * RabbitMQConnectionConfig#RabbitMQConnectionConfig(String, Integer, Boolean, Boolean, + * Integer, Integer, Integer, Integer, Integer)} + * + * @return RMQConnectionConfig + */ + public RabbitMQConnectionConfig build() { + if (this.uri != null) { + return new RabbitMQConnectionConfig( + this.uri, + this.networkRecoveryInterval, + this.automaticRecovery, + this.topologyRecovery, + this.connectionTimeout, + this.requestedChannelMax, + this.requestedFrameMax, + this.requestedHeartbeat, + this.prefetchCount); + } else { + return new RabbitMQConnectionConfig( + this.host, + this.port, + this.virtualHost, + this.username, + this.password, + this.networkRecoveryInterval, + this.automaticRecovery, + this.topologyRecovery, + this.connectionTimeout, + this.requestedChannelMax, + this.requestedFrameMax, + this.requestedHeartbeat, + this.prefetchCount); + } + } + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/README.md b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/README.md new file mode 100644 index 00000000000..eaee8d0ffea --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/README.md @@ -0,0 +1,14 @@ +# License of the RabbitMQ Connector + +Flink's RabbitMQ connector defines a Maven dependency on the +"RabbitMQ AMQP Java Client", is triple-licensed under the Mozilla Public License 1.1 ("MPL"), +the GNU General Public License version 2 ("GPL") and the Apache License version 2 ("ASL"). + +Flink itself neither reuses source code from the "RabbitMQ AMQP Java Client" +nor packages binaries from the "RabbitMQ AMQP Java Client". + +Users that create and publish derivative work based on Flink's +RabbitMQ connector (thereby re-distributing the "RabbitMQ AMQP Java Client") +must be aware that this may be subject to conditions declared in the +Mozilla Public License 1.1 ("MPL"), the GNU General Public License version 2 ("GPL") +and the Apache License version 2 ("ASL"). diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/RabbitMQSink.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/RabbitMQSink.java new file mode 100644 index 00000000000..ef0f313a140 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/RabbitMQSink.java @@ -0,0 +1,351 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink.Committer; +import org.apache.flink.api.connector.sink.GlobalCommitter; +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.rabbitmq.common.ConsistencyMode; +import org.apache.flink.connector.rabbitmq.common.RabbitMQConnectionConfig; +import org.apache.flink.connector.rabbitmq.sink.common.RabbitMQSinkPublishOptions; +import org.apache.flink.connector.rabbitmq.sink.common.SerializableReturnListener; +import org.apache.flink.connector.rabbitmq.sink.state.RabbitMQSinkWriterState; +import org.apache.flink.connector.rabbitmq.sink.state.RabbitMQSinkWriterStateSerializer; +import org.apache.flink.connector.rabbitmq.sink.writer.RabbitMQSinkWriterBase; +import org.apache.flink.connector.rabbitmq.sink.writer.specialized.RabbitMQSinkWriterAtLeastOnce; +import org.apache.flink.connector.rabbitmq.sink.writer.specialized.RabbitMQSinkWriterAtMostOnce; +import org.apache.flink.connector.rabbitmq.sink.writer.specialized.RabbitMQSinkWriterExactlyOnce; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +/** + * RabbitMQ sink that publishes messages to a RabbitMQ queue. It provides at-most-once, + * at-least-once or exactly-once processing semantics. For at-least-once and exactly-once, + * checkpointing needs to be enabled. + * + *

{@code
+ * RabbitMQSink
+ *     .builder()
+ *     .setConnectionConfig(connectionConfig)
+ *     .setQueueName("queue")
+ *     .setSerializationSchema(new SimpleStringSchema())
+ *     .setConsistencyMode(ConsistencyMode.AT_LEAST_ONCE)
+ *     .build();
+ * }
+ * + *

When creating the sink a {@code connectionConfig} must be specified via {@link + * RabbitMQConnectionConfig}. It contains required information for the RabbitMQ java client to + * connect to the RabbitMQ server. A minimum configuration contains a (virtual) host, a username, a + * password and a port. Besides that, the {@code queueName} to publish to and a {@link + * SerializationSchema} for the sink input type is required. {@code publishOptions} can be added + * optionally to route messages in RabbitMQ. + * + *

If at-least-once is required messages are buffered until an acknowledgement arrives because + * delivery needs to be guaranteed. On each checkpoint, all unacknowledged messages will be resent + * to RabbitMQ. In case of a failure, all unacknowledged messages can be restored and resend. + * + *

In the case of exactly-once a transactional RabbitMQ channel is used to achieve that all + * messages within a checkpoint are delivered once and only once. All messages that arrive in a + * checkpoint interval are buffered and sent to RabbitMQ in a single transaction when the checkpoint + * is triggered. If the transaction fails, all messages that were a part of the transaction are put + * back into the buffer and a resend is issued in the next checkpoint. + * + *

Keep in mind that the transactional channels are heavyweight and the performance will drop. + * Under heavy load, checkpoints can be delayed if a transaction takes longer than the specified + * checkpointing interval. + * + *

If publish options are used and the checkpointing mode is at-least-once or exactly-once, they + * require a {@link DeserializationSchema} to be provided because messages that were persisted as + * part of an earlier checkpoint are needed to recompute routing/exchange. + */ +public class RabbitMQSink implements Sink, Void> { + + private final RabbitMQConnectionConfig connectionConfig; + private final String queueName; + private final SerializationSchema serializationSchema; + private final RabbitMQSinkPublishOptions publishOptions; + private final ConsistencyMode consistencyMode; + private final SerializableReturnListener returnListener; + + private static final ConsistencyMode DEFAULT_CONSISTENCY_MODE = ConsistencyMode.AT_MOST_ONCE; + + private RabbitMQSink( + RabbitMQConnectionConfig connectionConfig, + String queueName, + SerializationSchema serializationSchema, + ConsistencyMode consistencyMode, + @Nullable SerializableReturnListener returnListener, + @Nullable RabbitMQSinkPublishOptions publishOptions) { + this.connectionConfig = requireNonNull(connectionConfig); + this.queueName = requireNonNull(queueName); + this.serializationSchema = requireNonNull(serializationSchema); + this.consistencyMode = requireNonNull(consistencyMode); + + this.returnListener = returnListener; + + Preconditions.checkState( + verifyPublishOptions(), + "If consistency mode is stronger than at-most-once and publish options are defined " + + "then publish options need a deserialization schema"); + this.publishOptions = publishOptions; + } + + private boolean verifyPublishOptions() { + // If at-most-once, doesn't matter if publish options are provided (no state in writer). + if (consistencyMode == ConsistencyMode.AT_MOST_ONCE || publishOptions == null) { + return true; + } + + // If we are at-least or exactly-once and publish options are set, we need a deserialization + // schema to recover the original messages from the state to recompute publish options. + return publishOptions.getDeserializationSchema().isPresent(); + } + + /** + * Get a {@link RabbitMQSinkBuilder} for the sink. + * + * @param type of the sink + * @return a sink builder + * @see RabbitMQSinkBuilder + */ + public static RabbitMQSinkBuilder builder() { + return new RabbitMQSinkBuilder<>(); + } + + /** + * Create and return an extension of {@link RabbitMQSinkWriterBase} based on the selected {@link + * ConsistencyMode}. + * + * @param context The initialization context of the Sink + * @param states A list of states to initialize the writer with + * @return The SinkWriter implementation depending on the consistency mode set by the user + */ + @Override + public SinkWriter> createWriter( + InitContext context, List> states) { + try { + RabbitMQSinkWriterBase sinkWriter = createSpecializedWriter(); + // Setup RabbitMQ needs to be called before recover from states as the writer might send + // messages to RabbitMQ on recover. + sinkWriter.setupRabbitMQ(); + sinkWriter.recoverFromStates(states); + return sinkWriter; + } catch (Exception e) { + throw new RuntimeException(e.getMessage()); + } + } + + private RabbitMQSinkWriterBase createSpecializedWriter() throws IllegalStateException { + switch (consistencyMode) { + case AT_MOST_ONCE: + return new RabbitMQSinkWriterAtMostOnce<>( + connectionConfig, + queueName, + serializationSchema, + publishOptions, + returnListener); + case AT_LEAST_ONCE: + return new RabbitMQSinkWriterAtLeastOnce<>( + connectionConfig, + queueName, + serializationSchema, + publishOptions, + returnListener); + case EXACTLY_ONCE: + return new RabbitMQSinkWriterExactlyOnce<>( + connectionConfig, + queueName, + serializationSchema, + publishOptions, + returnListener); + default: + throw new IllegalStateException( + "Error in creating a SinkWriter: No valid consistency mode (" + + consistencyMode + + ") was specified."); + } + } + + @Override + public Optional> createCommitter() { + return Optional.empty(); + } + + @Override + public Optional> createGlobalCommitter() { + return Optional.empty(); + } + + @Override + public Optional> getCommittableSerializer() { + return Optional.empty(); + } + + @Override + public Optional> getGlobalCommittableSerializer() { + return Optional.empty(); + } + + /** + * If publish options are specified and the sink writer has state (at-least-once or + * exactly-once) the deserialization schema for the messages need to be provided for the state + * serializer. + * + * @return The serializer for the state of the SinkWriter + * @see RabbitMQSinkWriterStateSerializer + */ + @Override + public Optional>> + getWriterStateSerializer() { + if (publishOptions != null && publishOptions.getDeserializationSchema().isPresent()) { + return Optional.of( + new RabbitMQSinkWriterStateSerializer<>( + publishOptions.getDeserializationSchema().get())); + } else { + return Optional.of(new RabbitMQSinkWriterStateSerializer<>()); + } + } + + /** + * A Builder for the {@link RabbitMQSink}. Available consistency modes are contained in {@link + * ConsistencyMode} Required parameters are a {@code queueName}, a {@code connectionConfig} and + * a {@code serializationSchema}. Optional parameters include {@code publishOptions}, a {@code + * minimalResendIntervalMilliseconds} (for at-least-once) and a {@code returnListener}. + * + *

{@code
+     * RabbitMQSink
+     *   .builder()
+     *   .setConnectionConfig(connectionConfig)
+     *   .setQueueName("queue")
+     *   .setSerializationSchema(new SimpleStringSchema())
+     *   .setConsistencyMode(ConsistencyMode.AT_LEAST_ONCE)
+     *   .build();
+     * }
+ */ + public static class RabbitMQSinkBuilder { + + private String queueName; + private RabbitMQConnectionConfig connectionConfig; + private SerializationSchema serializationSchema; + private ConsistencyMode consistencyMode; + private RabbitMQSinkPublishOptions publishOptions; + private SerializableReturnListener returnListener; + + public RabbitMQSinkBuilder() { + this.consistencyMode = RabbitMQSink.DEFAULT_CONSISTENCY_MODE; + } + + /** + * Builds the sink instance. + * + * @return new Sink instance that has the specified configuration + */ + public RabbitMQSink build() { + return new RabbitMQSink<>( + connectionConfig, + queueName, + serializationSchema, + consistencyMode, + returnListener, + publishOptions); + } + + /** + * Sets the RMQConnectionConfig for this sink. + * + * @param connectionConfig configuration required to connect to RabbitMQ + * @return this builder + */ + public RabbitMQSinkBuilder setConnectionConfig( + RabbitMQConnectionConfig connectionConfig) { + this.connectionConfig = connectionConfig; + return this; + } + + /** + * Sets the name of the queue to publish to. + * + * @param queueName name of an existing queue in RabbitMQ + * @return this builder + */ + public RabbitMQSinkBuilder setQueueName(String queueName) { + this.queueName = queueName; + return this; + } + + /** + * Sets the SerializationSchema used to serialize incoming objects. + * + * @param serializationSchema the serialization schema to use + * @return this builder + */ + public RabbitMQSinkBuilder setSerializationSchema( + SerializationSchema serializationSchema) { + this.serializationSchema = serializationSchema; + return this; + } + + /** + * Sets the RabbitMQSinkPublishOptions for this sink. Publish options can be used for + * routing in an exchange in RabbitMQ. + * + * @param publishOptions the publish options to be used + * @return this builder + */ + public RabbitMQSinkBuilder setPublishOptions( + RabbitMQSinkPublishOptions publishOptions) { + this.publishOptions = publishOptions; + return this; + } + + /** + * Set the ConsistencyMode for this sink to operate in. Available modes are AT_MOST_ONCE, + * AT_LEAST_ONCE and EXACTLY_ONCE + * + * @param consistencyMode set the consistency mode + * @return this builder + */ + public RabbitMQSinkBuilder setConsistencyMode(ConsistencyMode consistencyMode) { + this.consistencyMode = consistencyMode; + return this; + } + + /** + * Set the {@link SerializableReturnListener} for this sink. If no ReturnListener is set, + * unrouted messages, which are returned by RabbitMQ, will be dropped silently. + * + * @param returnListener the return listener to use + * @return this builder + */ + public RabbitMQSinkBuilder setReturnListener(SerializableReturnListener returnListener) { + this.returnListener = returnListener; + return this; + } + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/common/RabbitMQSinkConnection.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/common/RabbitMQSinkConnection.java new file mode 100644 index 00000000000..0e7aa543c79 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/common/RabbitMQSinkConnection.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink.common; + +import org.apache.flink.connector.rabbitmq.common.RabbitMQConnectionConfig; +import org.apache.flink.util.Preconditions; + +import com.rabbitmq.client.Channel; +import com.rabbitmq.client.Connection; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; + +import static java.util.Objects.requireNonNull; + +/** + * This class provides basic RabbitMQ functionality and common behaviour such as establishing and + * closing a connection via the {@code connectionConfig}. In addition, it provides methods for + * serializing and sending messages to RabbitMQ (with or without publish options). + * + * @param The type of the messages that are published + */ +public class RabbitMQSinkConnection { + protected static final Logger LOG = LoggerFactory.getLogger(RabbitMQSinkConnection.class); + + private final RabbitMQConnectionConfig connectionConfig; + private final String queueName; + private Connection rmqConnection; + private Channel rmqChannel; + + @Nullable private final RabbitMQSinkPublishOptions publishOptions; + + @Nullable private final SerializableReturnListener returnListener; + + public RabbitMQSinkConnection( + RabbitMQConnectionConfig connectionConfig, + String queueName, + @Nullable RabbitMQSinkPublishOptions publishOptions, + @Nullable SerializableReturnListener returnListener) { + this.connectionConfig = requireNonNull(connectionConfig); + this.queueName = requireNonNull(queueName); + this.publishOptions = publishOptions; + this.returnListener = returnListener; + } + + /** + * Setup the RabbitMQ connection and a channel to send messages to. + * + * @throws Exception that might occur when setting up the connection and channel. + */ + public void setupRabbitMQ() throws Exception { + LOG.info("Setup RabbitMQ"); + this.rmqConnection = setupConnection(connectionConfig); + this.rmqChannel = setupChannel(rmqConnection, queueName, returnListener); + } + + private Connection setupConnection(RabbitMQConnectionConfig connectionConfig) throws Exception { + return connectionConfig.getConnectionFactory().newConnection(); + } + + private Channel setupChannel( + Connection rmqConnection, String queueName, SerializableReturnListener returnListener) + throws IOException { + final Channel rmqChannel = rmqConnection.createChannel(); + rmqChannel.queueDeclare(queueName, true, false, false, null); + if (returnListener != null) { + rmqChannel.addReturnListener(returnListener); + } + return rmqChannel; + } + + /** + * Only used by at-least-once and exactly-once for resending messages that could not be + * delivered. + * + * @param message sink message wrapping the atomic message object + */ + public void send(RabbitMQSinkMessageWrapper message) throws IOException { + send(message.getMessage(), message.getBytes()); + } + + /** + * Publish a message to a queue in RabbitMQ. With publish options enabled, first compute the + * necessary publishing information. + * + * @param message original message, only required for publishing with publish options present + * @param serializedMessage serialized message to send to RabbitMQ + */ + public void send(T message, byte[] serializedMessage) throws IOException { + if (publishOptions == null) { + rmqChannel.basicPublish("", queueName, null, serializedMessage); + } else { + publishWithOptions(message, serializedMessage); + } + } + + private void publishWithOptions(T message, byte[] serializedMessage) throws IOException { + if (publishOptions == null) { + throw new RuntimeException("Try to publish with options without publishOptions."); + } + + boolean mandatory = publishOptions.computeMandatory(message); + boolean immediate = publishOptions.computeImmediate(message); + + Preconditions.checkState( + !(returnListener == null && (mandatory || immediate)), + "Setting mandatory and/or immediate flags to true requires a ReturnListener."); + + String rk = publishOptions.computeRoutingKey(message); + String exchange = publishOptions.computeExchange(message); + + rmqChannel.basicPublish( + exchange, + rk, + mandatory, + immediate, + publishOptions.computeProperties(message), + serializedMessage); + } + + /** + * Close the channel and connection to RabbitMQ. + * + * @throws Exception channel or connection closing failed + */ + public void close() throws Exception { + // close the channel + if (rmqChannel != null) { + rmqChannel.close(); + rmqChannel = null; + } + + // close the connection + if (rmqConnection != null) { + rmqConnection.close(); + rmqConnection = null; + } + } + + /** + * Get the internally used RabbitMQ channel. + * + * @return RabbitMQ channel object + */ + public Channel getRmqChannel() { + return rmqChannel; + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/common/RabbitMQSinkMessageWrapper.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/common/RabbitMQSinkMessageWrapper.java new file mode 100644 index 00000000000..7ce40c8a51b --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/common/RabbitMQSinkMessageWrapper.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink.common; + +import org.apache.flink.connector.rabbitmq.sink.writer.specialized.RabbitMQSinkWriterAtLeastOnce; +import org.apache.flink.connector.rabbitmq.sink.writer.specialized.RabbitMQSinkWriterExactlyOnce; + +import static java.util.Objects.requireNonNull; + +/** + * A wrapper class for messages that need to be persisted in the state of a {@link + * RabbitMQSinkWriterAtLeastOnce} or {@link RabbitMQSinkWriterExactlyOnce}. + * + *

It holds the message in its serialized format which gets sent to RabbitMQ. In the case of + * publish options being present and checkpointing modes of at-least-once or exactly-once the + * original message needs to be stored as well because it is needed for recomputing the + * exchange/routing key from the message content. + */ +public class RabbitMQSinkMessageWrapper { + private T message; + private final byte[] bytes; + + public RabbitMQSinkMessageWrapper(byte[] bytes) { + this.bytes = requireNonNull(bytes); + } + + public RabbitMQSinkMessageWrapper(T message, byte[] bytes) { + this(bytes); + this.message = requireNonNull(message); + } + + public byte[] getBytes() { + return bytes; + } + + public T getMessage() { + return message; + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/common/RabbitMQSinkPublishOptions.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/common/RabbitMQSinkPublishOptions.java new file mode 100644 index 00000000000..e6613ead400 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/common/RabbitMQSinkPublishOptions.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink.common; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.DeserializationSchema; + +import com.rabbitmq.client.AMQP.BasicProperties; + +import java.util.Optional; + +/** + * This class was copied from the old RabbitMQ connector and got extended by the serialization + * schema which is required for at-least-once and exactly-once. + * + *

The message computation provides methods to compute the message routing key and/or the + * properties. + * + * @param The type of the data used by the sink. + */ +@PublicEvolving +public interface RabbitMQSinkPublishOptions extends java.io.Serializable { + + /** + * Compute the message's routing key from the data. + * + * @param a The data used by the sink + * @return The routing key of the message null will raise a NullPointerException + */ + String computeRoutingKey(IN a); + + /** + * Compute the message's properties from the data. + * + * @param a The data used by the sink + * @return The message's properties (can be null) + */ + BasicProperties computeProperties(IN a); + + /** + * Compute the exchange from the data. + * + * @param a The data used by the sink + * @return The exchange to publish the message to null will raise a NullPointerException + */ + String computeExchange(IN a); + + /** + * Compute the mandatory flag passed to method {@link + * com.rabbitmq.client.Channel#basicPublish(String, String, boolean, boolean, BasicProperties, + * byte[])}. A {@link SerializableReturnListener} is mandatory if this flag can be true. + * + * @param a The data used by the sink + * @return The mandatory flag + */ + default boolean computeMandatory(IN a) { + return false; + } + + /** + * Compute the immediate flag passed to method {@link + * com.rabbitmq.client.Channel#basicPublish(String, String, boolean, boolean, BasicProperties, + * byte[])}. A {@link SerializableReturnListener} is mandatory if this flag can be true. + * + * @param a The data used by the sink + * @return The mandatory flag + */ + default boolean computeImmediate(IN a) { + return false; + } + + /** + * Get the deserialization schema for the serialized messages send to RabbitMQ by the + * SinkWriter. This is necessary if at-least or exactly-once is required. In these cases, + * messages need to be stored serialized in checkpoints.On initialization of a SinkWriter after + * a failure, checkpointed message need to be retrieved, deserialize and resend. The + * deserialization step is important to support the other publish options. + * + * @return a optional deserialization schema + */ + default Optional> getDeserializationSchema() { + return Optional.empty(); + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/common/SerializableReturnListener.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/common/SerializableReturnListener.java new file mode 100644 index 00000000000..42ab127309d --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/common/SerializableReturnListener.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink.common; + +import com.rabbitmq.client.ReturnListener; + +import java.io.Serializable; + +/** + * This class was copied from the old RabbitMQ connector. A serializable {@link ReturnListener} to + * handle unroutable but "mandatory" messages. + * + *

If a message has the "mandatory" flag set, but cannot be routed, RabbitMQ's broker will return + * the message to the publishing client (via an AMQP.Basic.Return command). This ReturnListener + * implements a callback handler to get notified in such returns and act on these messages as + * wanted. + */ +public interface SerializableReturnListener extends Serializable, ReturnListener {} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/state/RabbitMQSinkWriterState.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/state/RabbitMQSinkWriterState.java new file mode 100644 index 00000000000..01aa4034cf7 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/state/RabbitMQSinkWriterState.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink.state; + +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.rabbitmq.sink.common.RabbitMQSinkMessageWrapper; +import org.apache.flink.connector.rabbitmq.sink.writer.specialized.RabbitMQSinkWriterAtLeastOnce; +import org.apache.flink.connector.rabbitmq.sink.writer.specialized.RabbitMQSinkWriterExactlyOnce; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +/** + * The state of a {@link SinkWriter} implementation. Contains {@code outstandingMessages} that could + * not be delivered in a checkpoint. Used in the {@link RabbitMQSinkWriterAtLeastOnce} and {@link + * RabbitMQSinkWriterExactlyOnce} implementations. + */ +public class RabbitMQSinkWriterState { + private final List> outstandingMessages; + + public RabbitMQSinkWriterState(List> outstandingMessages) { + this.outstandingMessages = requireNonNull(outstandingMessages); + } + + public List> getOutstandingMessages() { + return outstandingMessages; + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/state/RabbitMQSinkWriterStateSerializer.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/state/RabbitMQSinkWriterStateSerializer.java new file mode 100644 index 00000000000..b9edcc25fb3 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/state/RabbitMQSinkWriterStateSerializer.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink.state; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.connector.rabbitmq.sink.common.RabbitMQSinkMessageWrapper; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Serializer for a {@link RabbitMQSinkWriterState} used for at-least and exactly-once consistency + * of the sink. + */ +public class RabbitMQSinkWriterStateSerializer + implements SimpleVersionedSerializer> { + private final DeserializationSchema deserializationSchema; + + public RabbitMQSinkWriterStateSerializer( + @Nullable DeserializationSchema deserializationSchema) { + this.deserializationSchema = deserializationSchema; + } + + public RabbitMQSinkWriterStateSerializer() { + this(null); + } + + @Override + public int getVersion() { + return 1; + } + + /** + * Serializes all {@code outstandingMessages} of a state of a single sink writer. + * + * @param rabbitMQSinkWriterState A state containing a list of {@code outstandingMessages} + * @throws IOException If output stream cant write the required data + */ + @Override + public byte[] serialize(RabbitMQSinkWriterState rabbitMQSinkWriterState) throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos); + serializeV1(out, rabbitMQSinkWriterState.getOutstandingMessages()); + return baos.toByteArray(); + } + + private void serializeV1(DataOutputStream out, List> messages) + throws IOException { + out.writeInt(messages.size()); + for (RabbitMQSinkMessageWrapper message : messages) { + out.writeInt(message.getBytes().length); + out.write(message.getBytes()); + } + out.flush(); + } + + /** + * Deserializes {@link RabbitMQSinkMessageWrapper} objects that wrap the byte representation of + * a message that needs to be delivered to RabbitMQ as well as the original object + * representation if a deserialization schema is provided. + * + * @param version which deserialization version should be used + * @param bytes Serialized outstanding sink messages + * @return A list of messages that need to be redelivered to RabbitMQ + * @throws IOException If input stream cant read the required data + */ + @Override + public RabbitMQSinkWriterState deserialize(int version, byte[] bytes) throws IOException { + switch (version) { + case 1: + return deserializeV1(bytes); + default: + throw new IOException("Unrecognized version or corrupt state: " + version); + } + } + + private RabbitMQSinkWriterState deserializeV1(byte[] bytes) throws IOException { + ByteArrayInputStream bais = new ByteArrayInputStream(bytes); + DataInputStream in = new DataInputStream(bais); + return new RabbitMQSinkWriterState<>(readSinkMessages(in)); + } + + private List> readSinkMessages(DataInputStream in) + throws IOException { + final int numberOfMessages = in.readInt(); + List> messages = new ArrayList<>(); + for (int i = 0; i < numberOfMessages; i++) { + byte[] bytes = new byte[in.readInt()]; + in.read(bytes); + + // In this case, the messages need to be deserialized again, so we can recompute publish + // options + if (deserializationSchema != null) { + messages.add( + new RabbitMQSinkMessageWrapper<>( + deserializationSchema.deserialize(bytes), bytes)); + } else { + messages.add(new RabbitMQSinkMessageWrapper<>(bytes)); + } + } + return messages; + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/writer/RabbitMQSinkWriterBase.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/writer/RabbitMQSinkWriterBase.java new file mode 100644 index 00000000000..567101a8ff9 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/writer/RabbitMQSinkWriterBase.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink.writer; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.rabbitmq.common.RabbitMQConnectionConfig; +import org.apache.flink.connector.rabbitmq.sink.common.RabbitMQSinkConnection; +import org.apache.flink.connector.rabbitmq.sink.common.RabbitMQSinkMessageWrapper; +import org.apache.flink.connector.rabbitmq.sink.common.RabbitMQSinkPublishOptions; +import org.apache.flink.connector.rabbitmq.sink.common.SerializableReturnListener; +import org.apache.flink.connector.rabbitmq.sink.state.RabbitMQSinkWriterState; +import org.apache.flink.connector.rabbitmq.sink.writer.specialized.RabbitMQSinkWriterAtLeastOnce; +import org.apache.flink.connector.rabbitmq.sink.writer.specialized.RabbitMQSinkWriterAtMostOnce; +import org.apache.flink.connector.rabbitmq.sink.writer.specialized.RabbitMQSinkWriterExactlyOnce; + +import com.rabbitmq.client.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import static java.util.Objects.requireNonNull; + +/** + * RabbitMQSinkWriterBase is the common abstract class of {@link RabbitMQSinkWriterAtMostOnce}, + * {@link RabbitMQSinkWriterAtLeastOnce} and {@link RabbitMQSinkWriterExactlyOnce}. + * + * @param Type of the elements in this sink + */ +public abstract class RabbitMQSinkWriterBase + implements SinkWriter> { + private static final Logger LOG = LoggerFactory.getLogger(RabbitMQSinkWriterBase.class); + + private final RabbitMQSinkConnection rmqSinkConnection; + private final SerializationSchema serializationSchema; + + public RabbitMQSinkWriterBase( + RabbitMQConnectionConfig connectionConfig, + String queueName, + SerializationSchema serializationSchema, + RabbitMQSinkPublishOptions publishOptions, + SerializableReturnListener returnListener) { + this.rmqSinkConnection = + new RabbitMQSinkConnection<>( + connectionConfig, queueName, publishOptions, returnListener); + this.serializationSchema = requireNonNull(serializationSchema); + } + + /** + * Receive the next stream element and publish it to RabbitMQ. + * + * @param element element from upstream flink task + * @param context context of this sink writer + */ + @Override + public void write(T element, Context context) throws IOException { + getRmqSinkConnection() + .send( + new RabbitMQSinkMessageWrapper<>( + element, serializationSchema.serialize(element))); + } + + /** + * Recover the writer with a specific state. + * + * @param states a list of states to recover the reader with + * @throws IOException that can be thrown as specialized writers might want to send messages. + */ + public void recoverFromStates(List> states) throws IOException {} + + /** + * Setup the RabbitMQ connection and a channel to send messages to. In the end specialized + * writers can configure the channel through {@link #configureChannel()}. + * + * @throws Exception that might occur when setting up the connection and channel. + */ + public void setupRabbitMQ() throws Exception { + this.rmqSinkConnection.setupRabbitMQ(); + configureChannel(); + } + + /** + * This method provides a hook to apply additional configuration to the channel. + * + * @throws IOException possible IOException that might be thrown when configuring the channel + */ + protected void configureChannel() throws IOException {} + + @Override + public List prepareCommit(boolean flush) { + return Collections.emptyList(); + } + + @Override + public List> snapshotState() throws IOException { + return Collections.emptyList(); + } + + @Override + public void close() throws Exception { + LOG.info("Close Sink Writer"); + rmqSinkConnection.close(); + } + + protected RabbitMQSinkConnection getRmqSinkConnection() { + return rmqSinkConnection; + } + + protected Channel getRmqChannel() { + return rmqSinkConnection.getRmqChannel(); + } + + protected SerializationSchema getSerializationSchema() { + return serializationSchema; + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/writer/specialized/RabbitMQSinkWriterAtLeastOnce.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/writer/specialized/RabbitMQSinkWriterAtLeastOnce.java new file mode 100644 index 00000000000..40cefddb5cd --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/writer/specialized/RabbitMQSinkWriterAtLeastOnce.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink.writer.specialized; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.rabbitmq.common.RabbitMQConnectionConfig; +import org.apache.flink.connector.rabbitmq.sink.RabbitMQSink; +import org.apache.flink.connector.rabbitmq.sink.common.RabbitMQSinkMessageWrapper; +import org.apache.flink.connector.rabbitmq.sink.common.RabbitMQSinkPublishOptions; +import org.apache.flink.connector.rabbitmq.sink.common.SerializableReturnListener; +import org.apache.flink.connector.rabbitmq.sink.state.RabbitMQSinkWriterState; +import org.apache.flink.connector.rabbitmq.sink.writer.RabbitMQSinkWriterBase; + +import com.rabbitmq.client.ConfirmCallback; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentNavigableMap; +import java.util.concurrent.ConcurrentSkipListMap; + +/** + * A {@link SinkWriter} implementation for {@link RabbitMQSink} that has at-least-once semantics, + * meaning it guarantees that outgoing message arrive at RabbitMQ at least once. + * + *

At-least-once consistency is implemented by assigning sequence numbers to arriving messages + * and buffering them together in the state of the writer until an ack arrives. + * + *

Checkpointing is required for at-least-once to work because messages are resend only when a + * checkpoint is triggered (to avoid complex time tracking mechanisms for each individual message). + * Thus on each checkpoint, all messages which were sent at least once before to RabbitMQ but are + * still unacknowledged will be send once again - duplications are possible by this behavior. + * + *

After a failure, a new writer gets initialized with one or more states that contain + * unacknowledged messages. These messages get resend immediately while buffering them in the new + * state of the writer. + * + * @param Type of the elements in this sink + */ +public class RabbitMQSinkWriterAtLeastOnce extends RabbitMQSinkWriterBase { + protected final ConcurrentNavigableMap> outstandingConfirms; + private static final Logger LOG = LoggerFactory.getLogger(RabbitMQSinkWriterAtLeastOnce.class); + + private Set lastSeenMessageIds; + + /** + * Create a new RabbitMQSinkWriterAtLeastOnce. + * + * @param connectionConfig configuration parameters used to connect to RabbitMQ + * @param queueName name of the queue to publish to + * @param serializationSchema serialization schema to turn elements into byte representation + * @param publishOptions optionally used to compute routing/exchange for messages + * @param returnListener returnListener + */ + public RabbitMQSinkWriterAtLeastOnce( + RabbitMQConnectionConfig connectionConfig, + String queueName, + SerializationSchema serializationSchema, + RabbitMQSinkPublishOptions publishOptions, + SerializableReturnListener returnListener) { + super(connectionConfig, queueName, serializationSchema, publishOptions, returnListener); + this.outstandingConfirms = new ConcurrentSkipListMap<>(); + this.lastSeenMessageIds = new HashSet<>(); + } + + /** + * On recover all stored messages in the states get resend. + * + * @param states a list of states to recover the reader with + * @throws IOException as messages are send to RabbitMQ + */ + @Override + public void recoverFromStates(List> states) throws IOException { + for (RabbitMQSinkWriterState state : states) { + for (RabbitMQSinkMessageWrapper message : state.getOutstandingMessages()) { + send(message); + } + } + } + + private void send(RabbitMQSinkMessageWrapper msg) throws IOException { + long sequenceNumber = getRmqChannel().getNextPublishSeqNo(); + getRmqSinkConnection().send(msg); + outstandingConfirms.put(sequenceNumber, msg); + } + + private void resendMessages() throws IOException { + Set temp = outstandingConfirms.keySet(); + Set messagesToResend = new HashSet<>(temp); + messagesToResend.retainAll(lastSeenMessageIds); + for (Long id : messagesToResend) { + // remove the old message from the map, since the message was added a second time + // under a new id or is put into the list of messages to resend + RabbitMQSinkMessageWrapper msg = outstandingConfirms.remove(id); + if (msg != null) { + send(msg); + } + } + lastSeenMessageIds = temp; + } + + private ConfirmCallback handleAcknowledgements() { + return (sequenceNumber, multiple) -> { + // multiple flag indicates that all messages < sequenceNumber can be safely acknowledged + if (multiple) { + // create a view of the portion of the map that contains keys < sequenceNumber + ConcurrentNavigableMap> confirmed = + outstandingConfirms.headMap(sequenceNumber, true); + // changes to the view are reflected in the original map + confirmed.clear(); + } else { + outstandingConfirms.remove(sequenceNumber); + } + }; + } + + private ConfirmCallback handleNegativeAcknowledgements() { + return (sequenceNumber, multiple) -> { + RabbitMQSinkMessageWrapper message = outstandingConfirms.get(sequenceNumber); + LOG.error( + "Message with body {} has been nack-ed. Sequence number: {}, multiple: {}", + message.getMessage(), + sequenceNumber, + multiple); + }; + } + + @Override + protected void configureChannel() throws IOException { + ConfirmCallback ackCallback = handleAcknowledgements(); + ConfirmCallback nackCallback = handleNegativeAcknowledgements(); + // register callbacks for cases of ack and negative ack of messages (seq numbers) + getRmqChannel().addConfirmListener(ackCallback, nackCallback); + getRmqChannel().confirmSelect(); + } + + /** + * All messages that are sent to RabbitMQ and not acknowledged yet will be resend. A single + * state is returned that contains just the messages that could not be acknowledged within the + * last checkpoint. + * + * @return A singleton list of RabbitMQSinkWriterState with outstanding confirms + * @throws IOException if resend of messages fails + */ + @Override + public List> snapshotState() throws IOException { + resendMessages(); + return Collections.singletonList( + new RabbitMQSinkWriterState<>(new ArrayList<>(outstandingConfirms.values()))); + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/writer/specialized/RabbitMQSinkWriterAtMostOnce.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/writer/specialized/RabbitMQSinkWriterAtMostOnce.java new file mode 100644 index 00000000000..30b2539858d --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/writer/specialized/RabbitMQSinkWriterAtMostOnce.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink.writer.specialized; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.rabbitmq.common.RabbitMQConnectionConfig; +import org.apache.flink.connector.rabbitmq.sink.RabbitMQSink; +import org.apache.flink.connector.rabbitmq.sink.common.RabbitMQSinkPublishOptions; +import org.apache.flink.connector.rabbitmq.sink.common.SerializableReturnListener; +import org.apache.flink.connector.rabbitmq.sink.writer.RabbitMQSinkWriterBase; + +import java.io.IOException; + +/** + * A {@link SinkWriter} implementation for {@link RabbitMQSink}. + * + *

It uses exclusively the basic functionalities provided by {@link RabbitMQSinkWriterBase} for + * publishing messages to RabbitMQ (serializing a stream element and publishing it to RabbitMQ in a + * fire-and-forget fashion). + */ +public class RabbitMQSinkWriterAtMostOnce extends RabbitMQSinkWriterBase { + + /** + * Create a new RabbitMQSinkWriterExactlyOnce. + * + * @param connectionConfig configuration parameters used to connect to RabbitMQ + * @param queueName name of the queue to publish to + * @param serializationSchema serialization schema to turn elements into byte representation + * @param publishOptions optionally used to compute routing/exchange for messages + * @param returnListener returnListener + */ + public RabbitMQSinkWriterAtMostOnce( + RabbitMQConnectionConfig connectionConfig, + String queueName, + SerializationSchema serializationSchema, + RabbitMQSinkPublishOptions publishOptions, + SerializableReturnListener returnListener) { + super(connectionConfig, queueName, serializationSchema, publishOptions, returnListener); + } + + @Override + public void write(T element, Context context) throws IOException { + getRmqSinkConnection().send(element, getSerializationSchema().serialize(element)); + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/writer/specialized/RabbitMQSinkWriterExactlyOnce.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/writer/specialized/RabbitMQSinkWriterExactlyOnce.java new file mode 100644 index 00000000000..ef25d6f99e3 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/sink/writer/specialized/RabbitMQSinkWriterExactlyOnce.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink.writer.specialized; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.rabbitmq.common.RabbitMQConnectionConfig; +import org.apache.flink.connector.rabbitmq.sink.RabbitMQSink; +import org.apache.flink.connector.rabbitmq.sink.common.RabbitMQSinkMessageWrapper; +import org.apache.flink.connector.rabbitmq.sink.common.RabbitMQSinkPublishOptions; +import org.apache.flink.connector.rabbitmq.sink.common.SerializableReturnListener; +import org.apache.flink.connector.rabbitmq.sink.state.RabbitMQSinkWriterState; +import org.apache.flink.connector.rabbitmq.sink.writer.RabbitMQSinkWriterBase; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * A {@link SinkWriter} implementation for {@link RabbitMQSink} that provides exactly-once delivery + * guarantees, which means incoming stream elements will be delivered to RabbitMQ exactly once. For + * this, checkpointing needs to be enabled. + * + *

Exactly-once consistency is implemented using a transactional RabbitMQ channel. All incoming + * stream elements are buffered in the state of this writer until the next checkpoint is triggered. + * All buffered {@code messages} are then send to RabbitMQ in a single transaction. When successful, + * all messages committed get removed from the state. If the transaction is aborted, all messages + * are put back into the state and send on the next checkpoint. + * + *

The transactional channel is heavyweight and will decrease throughput. If the system is under + * heavy load, consecutive checkpoints can be delayed if commits take longer than the checkpointing + * interval specified. Only use exactly-once if necessary (no duplicated messages in RabbitMQ + * allowed), otherwise consider using at-least-once. + * + * @param Type of the elements in this sink + */ +public class RabbitMQSinkWriterExactlyOnce extends RabbitMQSinkWriterBase { + private static final Logger LOG = LoggerFactory.getLogger(RabbitMQSinkWriterExactlyOnce.class); + + // All messages that arrived and could not be committed this far. + private List> messages; + + /** + * Create a new RabbitMQSinkWriterExactlyOnce. + * + * @param connectionConfig configuration parameters used to connect to RabbitMQ + * @param queueName name of the queue to publish to + * @param serializationSchema serialization schema to turn elements into byte representation + * @param publishOptions optionally used to compute routing/exchange for messages + * @param returnListener return listener + */ + public RabbitMQSinkWriterExactlyOnce( + RabbitMQConnectionConfig connectionConfig, + String queueName, + SerializationSchema serializationSchema, + RabbitMQSinkPublishOptions publishOptions, + SerializableReturnListener returnListener) { + super(connectionConfig, queueName, serializationSchema, publishOptions, returnListener); + this.messages = Collections.synchronizedList(new ArrayList<>()); + } + + /** + * On recover the messages are set to the outstanding messages from the states. + * + * @param states a list of states to recover the reader with + */ + @Override + public void recoverFromStates(List> states) { + for (RabbitMQSinkWriterState state : states) { + this.messages.addAll(state.getOutstandingMessages()); + } + } + + @Override + protected void configureChannel() throws IOException { + // puts channel in commit mode + getRmqChannel().txSelect(); + } + + @Override + public void write(T element, Context context) { + messages.add( + new RabbitMQSinkMessageWrapper<>( + element, getSerializationSchema().serialize(element))); + } + + @Override + public List> snapshotState() { + commitMessages(); + return Collections.singletonList(new RabbitMQSinkWriterState<>(messages)); + } + + private void commitMessages() { + List> messagesToSend = messages.subList(0, messages.size()); + try { + for (RabbitMQSinkMessageWrapper msg : messagesToSend) { + getRmqSinkConnection().send(msg); + } + getRmqChannel().txCommit(); + LOG.info("Successfully committed {} messages.", messagesToSend.size()); + messagesToSend.clear(); + } catch (IOException commitException) { + try { + getRmqChannel().txRollback(); + } catch (IOException rollbackException) { + throw new RuntimeException( + "Cannot rollback RabbitMQ transaction after commit failure, this might leave the transaction in a pending state. Commit Error: " + + commitException.getMessage() + + " Rollback Error: " + + rollbackException.getMessage()); + } + throw new RuntimeException( + "Error during transactional commit of messages. Rollback was successful. Error: " + + commitException.getMessage()); + } + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/README.md b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/README.md new file mode 100644 index 00000000000..eaee8d0ffea --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/README.md @@ -0,0 +1,14 @@ +# License of the RabbitMQ Connector + +Flink's RabbitMQ connector defines a Maven dependency on the +"RabbitMQ AMQP Java Client", is triple-licensed under the Mozilla Public License 1.1 ("MPL"), +the GNU General Public License version 2 ("GPL") and the Apache License version 2 ("ASL"). + +Flink itself neither reuses source code from the "RabbitMQ AMQP Java Client" +nor packages binaries from the "RabbitMQ AMQP Java Client". + +Users that create and publish derivative work based on Flink's +RabbitMQ connector (thereby re-distributing the "RabbitMQ AMQP Java Client") +must be aware that this may be subject to conditions declared in the +Mozilla Public License 1.1 ("MPL"), the GNU General Public License version 2 ("GPL") +and the Apache License version 2 ("ASL"). diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/RabbitMQSource.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/RabbitMQSource.java new file mode 100644 index 00000000000..31acf2f3778 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/RabbitMQSource.java @@ -0,0 +1,310 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.rabbitmq.common.ConsistencyMode; +import org.apache.flink.connector.rabbitmq.common.RabbitMQConnectionConfig; +import org.apache.flink.connector.rabbitmq.source.enumerator.RabbitMQSourceEnumState; +import org.apache.flink.connector.rabbitmq.source.enumerator.RabbitMQSourceEnumStateSerializer; +import org.apache.flink.connector.rabbitmq.source.enumerator.RabbitMQSourceEnumerator; +import org.apache.flink.connector.rabbitmq.source.reader.RabbitMQSourceReaderBase; +import org.apache.flink.connector.rabbitmq.source.reader.specialized.RabbitMQSourceReaderAtLeastOnce; +import org.apache.flink.connector.rabbitmq.source.reader.specialized.RabbitMQSourceReaderAtMostOnce; +import org.apache.flink.connector.rabbitmq.source.reader.specialized.RabbitMQSourceReaderExactlyOnce; +import org.apache.flink.connector.rabbitmq.source.split.RabbitMQSourceSplit; +import org.apache.flink.connector.rabbitmq.source.split.RabbitMQSourceSplitSerializer; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static java.util.Objects.requireNonNull; + +/** + * RabbitMQ source (consumer) that consumes messages from a RabbitMQ queue. It provides + * at-most-once, at-least-once and exactly-once processing semantics. For at-least-once and + * exactly-once, checkpointing needs to be enabled. The source operates as a StreamingSource and + * thus works in a streaming fashion. Please use a {@link RabbitMQSourceBuilder} to construct the + * source. The following example shows how to create a RabbitMQSource emitting records of + * String type. + * + *

{@code
+ * RabbitMQSource source = RabbitMQSource
+ *     .builder()
+ *     .setConnectionConfig(MY_RMQ_CONNECTION_CONFIG)
+ *     .setQueueName("myQueue")
+ *     .setDeliveryDeserializer(new SimpleStringSchema())
+ *     .setConsistencyMode(MY_CONSISTENCY_MODE)
+ *     .build();
+ * }
+ * + *

When creating the source a {@code connectionConfig} must be specified via {@link + * RabbitMQConnectionConfig}. It contains required information for the RabbitMQ java client to + * connect to the RabbitMQ server. A minimum configuration contains a (virtual) host, a username, a + * password and a port. Besides that, the {@code queueName} to consume from and a {@link + * DeserializationSchema} + * + *

When using at-most-once consistency, messages are automatically acknowledged when received + * from RabbitMQ and later consumed by the output. In case of a failure, messages might be lost. + * More details in {@link RabbitMQSourceReaderAtMostOnce}. + * + *

In case of at-least-once consistency, message are buffered and later consumed by the output. + * Once a checkpoint is finished, the messages that were consumed by the output are acknowledged to + * RabbitMQ. This way, we ensure that the messages are successfully received by the output. In case + * of a system failure, the message that were acknowledged to RabbitMQ will be resend by RabbitMQ. + * More details in {@link RabbitMQSourceReaderAtLeastOnce}. + * + *

To ensure exactly-once consistency, messages are deduplicated through {@code correlationIds}. + * Similar to at-least-once consistency, we store the {@code deliveryTags} of the messages that are + * consumed by the output to acknowledge them later. A transactional RabbitMQ channel is used to + * ensure that all messages are successfully acknowledged to RabbitMQ. More details in {@link + * RabbitMQSourceReaderExactlyOnce}. + * + *

Keep in mind that the transactional channels are heavyweight and performance will drop. Under + * heavy load, checkpoints can be delayed if a transaction takes longer than the specified + * checkpointing interval. + * + * @param the output type of the source. + */ +public class RabbitMQSource + implements Source, ResultTypeQueryable { + private static final Logger LOG = LoggerFactory.getLogger(RabbitMQSource.class); + + private final RabbitMQConnectionConfig connectionConfig; + private final String queueName; + private final DeserializationSchema deserializationSchema; + private final ConsistencyMode consistencyMode; + + private RabbitMQSource( + RabbitMQConnectionConfig connectionConfig, + String queueName, + DeserializationSchema deserializationSchema, + ConsistencyMode consistencyMode) { + this.connectionConfig = requireNonNull(connectionConfig); + this.queueName = requireNonNull(queueName); + this.deserializationSchema = requireNonNull(deserializationSchema); + this.consistencyMode = requireNonNull(consistencyMode); + + LOG.info("Create RabbitMQ source"); + } + + /** + * Get a {@link RabbitMQSourceBuilder} for the source. + * + * @param type of the source. + * @return a source builder + * @see RabbitMQSourceBuilder + */ + public static RabbitMQSourceBuilder builder() { + return new RabbitMQSourceBuilder<>(); + } + + /** + * The boundedness is always continuous unbounded as this is a streaming-only source. + * + * @return Boundedness continuous unbounded. + * @see Boundedness + */ + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + /** + * Returns a new initialized source reader of the source's consistency mode. + * + * @param sourceReaderContext context which the reader will be created in. + * @return RabbitMQSourceReader a source reader of the specified consistency type. + * @see RabbitMQSourceReaderBase + */ + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) { + LOG.info("New Source Reader of type {} requested.", consistencyMode); + switch (consistencyMode) { + case AT_MOST_ONCE: + return new RabbitMQSourceReaderAtMostOnce<>( + sourceReaderContext, deserializationSchema); + case AT_LEAST_ONCE: + return new RabbitMQSourceReaderAtLeastOnce<>( + sourceReaderContext, deserializationSchema); + case EXACTLY_ONCE: + return new RabbitMQSourceReaderExactlyOnce<>( + sourceReaderContext, deserializationSchema); + default: + throw new IllegalStateException( + "Error in creating a SourceReader: No valid consistency mode (" + + consistencyMode + + ") was specified."); + } + } + + /** + * @param splitEnumeratorContext context which the enumerator will be created in + * @return a new split enumerator + * @see SplitEnumerator + */ + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext splitEnumeratorContext) { + return new RabbitMQSourceEnumerator( + splitEnumeratorContext, consistencyMode, connectionConfig, queueName); + } + + /** + * @param splitEnumeratorContext context which the enumerator will be created in + * @param enumState enum state the + * @return a new split enumerator + * @see SplitEnumerator + */ + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext splitEnumeratorContext, + RabbitMQSourceEnumState enumState) { + return new RabbitMQSourceEnumerator( + splitEnumeratorContext, consistencyMode, connectionConfig, queueName, enumState); + } + + /** + * @return a simple serializer for a RabbitMQPartitionSplit + * @see SimpleVersionedSerializer + */ + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new RabbitMQSourceSplitSerializer(); + } + + /** + * @return a simple serializer for a RabbitMQSourceEnumState + * @see SimpleVersionedSerializer + */ + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return new RabbitMQSourceEnumStateSerializer(); + } + + /** + * @return type information + * @see TypeInformation + */ + @Override + public TypeInformation getProducedType() { + return deserializationSchema.getProducedType(); + } + + /** + * A @builder class to simplify the creation of a {@link RabbitMQSource}. + * + *

The following example shows the minimum setup to create a RabbitMQSource that reads String + * messages from a Queue. + * + *

{@code
+     * RabbitMQSource source = RabbitMQSource
+     *     .builder()
+     *     .setConnectionConfig(MY_RMQ_CONNECTION_CONFIG)
+     *     .setQueueName("myQueue")
+     *     .setDeliveryDeserializer(new SimpleStringSchema())
+     *     .setConsistencyMode(MY_CONSISTENCY_MODE)
+     *     .build();
+     * }
+ * + *

For details about the connection config refer to {@link RabbitMQConnectionConfig}. For + * details about the available consistency modes refer to {@link ConsistencyMode}. + * + * @param the output type of the source. + */ + public static class RabbitMQSourceBuilder { + // The configuration for the RabbitMQ connection. + private RabbitMQConnectionConfig connectionConfig; + // Name of the queue to consume from. + private String queueName; + // The deserializer for the messages of RabbitMQ. + private DeserializationSchema deserializationSchema; + // The consistency mode for the source. + private ConsistencyMode consistencyMode; + + /** + * Build the {@link RabbitMQSource}. + * + * @return a RabbitMQSource with the configuration set for this builder. + */ + public RabbitMQSource build() { + return new RabbitMQSource<>( + connectionConfig, queueName, deserializationSchema, consistencyMode); + } + + /** + * Set the connection config for RabbitMQ. + * + * @param connectionConfig the connection configuration for RabbitMQ. + * @return this RabbitMQSourceBuilder + * @see RabbitMQConnectionConfig + */ + public RabbitMQSourceBuilder setConnectionConfig( + RabbitMQConnectionConfig connectionConfig) { + this.connectionConfig = connectionConfig; + return this; + } + + /** + * Set the name of the queue to consume from. + * + * @param queueName the name of the queue to consume from. + * @return this RabbitMQSourceBuilder + */ + public RabbitMQSourceBuilder setQueueName(String queueName) { + this.queueName = queueName; + return this; + } + + /** + * Set the deserializer for the message deliveries from RabbitMQ. + * + * @param deserializationSchema a deserializer for the message deliveries from RabbitMQ. + * @return this RabbitMQSourceBuilder + * @see DeserializationSchema + */ + public RabbitMQSourceBuilder setDeserializationSchema( + DeserializationSchema deserializationSchema) { + this.deserializationSchema = deserializationSchema; + return this; + } + + /** + * Set the consistency mode for the source. + * + * @param consistencyMode the consistency mode for the source. + * @return this RabbitMQSourceBuilder + * @see ConsistencyMode + */ + public RabbitMQSourceBuilder setConsistencyMode(ConsistencyMode consistencyMode) { + this.consistencyMode = consistencyMode; + return this; + } + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/common/RabbitMQSourceMessageWrapper.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/common/RabbitMQSourceMessageWrapper.java new file mode 100644 index 00000000000..0289baa72ac --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/common/RabbitMQSourceMessageWrapper.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source.common; + +import javax.annotation.Nullable; + +/** + * A wrapper class for the message received from RabbitMQ that holds the deserialized message, the + * delivery tag and the correlation id. + * + * @param The type of the message to hold. + */ +public class RabbitMQSourceMessageWrapper { + private final long deliveryTag; + private final String correlationId; + private final T message; + + public RabbitMQSourceMessageWrapper( + long deliveryTag, @Nullable String correlationId, @Nullable T message) { + this.deliveryTag = deliveryTag; + this.correlationId = correlationId; + this.message = message; + } + + public RabbitMQSourceMessageWrapper(long deliveryTag, String correlationId) { + this(deliveryTag, correlationId, null); + } + + public long getDeliveryTag() { + return deliveryTag; + } + + public T getMessage() { + return message; + } + + public String getCorrelationId() { + return correlationId; + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/enumerator/RabbitMQSourceEnumState.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/enumerator/RabbitMQSourceEnumState.java new file mode 100644 index 00000000000..c7c7592d4ca --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/enumerator/RabbitMQSourceEnumState.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source.enumerator; + +/** + * The EnumState is empty because every reader gets assigned the same split. And therefore, no split + * assignment needs to be remembered. + * + * @see RabbitMQSourceEnumerator + */ +public class RabbitMQSourceEnumState {} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/enumerator/RabbitMQSourceEnumStateSerializer.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/enumerator/RabbitMQSourceEnumStateSerializer.java new file mode 100644 index 00000000000..b95e1cf734c --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/enumerator/RabbitMQSourceEnumStateSerializer.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source.enumerator; + +import org.apache.flink.core.io.SimpleVersionedSerializer; + +/** + * The EnumStateSerializer does nothing particular because the EnumState does not contain data. + * + * @see RabbitMQSourceEnumState + */ +public class RabbitMQSourceEnumStateSerializer + implements SimpleVersionedSerializer { + + @Override + public int getVersion() { + return 1; + } + + @Override + public byte[] serialize(RabbitMQSourceEnumState rabbitMQSourceEnumState) { + if (getVersion() == 1) { + return new byte[0]; + } + throw new RuntimeException("Version " + getVersion() + " is not supported"); + } + + @Override + public RabbitMQSourceEnumState deserialize(int i, byte[] bytes) { + if (getVersion() == 1) { + return new RabbitMQSourceEnumState(); + } + throw new RuntimeException("Version " + getVersion() + " is not supported"); + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/enumerator/RabbitMQSourceEnumerator.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/enumerator/RabbitMQSourceEnumerator.java new file mode 100644 index 00000000000..5a708ae607a --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/enumerator/RabbitMQSourceEnumerator.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source.enumerator; + +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.rabbitmq.common.ConsistencyMode; +import org.apache.flink.connector.rabbitmq.common.RabbitMQConnectionConfig; +import org.apache.flink.connector.rabbitmq.source.split.RabbitMQSourceSplit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +/** + * The source enumerator provides the source readers with the split. All source readers receive the + * same split as it only contains information about the connection and in case of exactly-once, the + * seen correlation ids. But in this case, the enumerator makes sure that at maximum one source + * reader receives the split. During exactly-once if multiple reader should be assigned a split a + * {@link RuntimeException} is thrown. + */ +public class RabbitMQSourceEnumerator + implements SplitEnumerator { + private final SplitEnumeratorContext context; + private final ConsistencyMode consistencyMode; + private static final Logger LOG = LoggerFactory.getLogger(RabbitMQSourceEnumerator.class); + private RabbitMQSourceSplit split; + + public RabbitMQSourceEnumerator( + SplitEnumeratorContext context, + ConsistencyMode consistencyMode, + RabbitMQConnectionConfig connectionConfig, + String rmqQueueName, + RabbitMQSourceEnumState enumState) { + // The enumState is not used since the enumerator has no state in this architecture. + this(context, consistencyMode, connectionConfig, rmqQueueName); + } + + public RabbitMQSourceEnumerator( + SplitEnumeratorContext context, + ConsistencyMode consistencyMode, + RabbitMQConnectionConfig connectionConfig, + String rmqQueueName) { + this.context = requireNonNull(context); + this.consistencyMode = requireNonNull(consistencyMode); + this.split = new RabbitMQSourceSplit(connectionConfig, rmqQueueName); + } + + @Override + public void start() { + LOG.info("Start RabbitMQ source enumerator"); + } + + @Override + public void handleSplitRequest(int i, @Nullable String s) { + LOG.info("Split request from reader {}.", i); + assignSplitToReader(i, split); + } + + @Override + public void addSplitsBack(List list, int i) { + if (list == null || list.size() == 0) { + return; + } + + // Every Source Reader will only receive one split, thus we will never get back more. + if (list.size() != 1) { + throw new RuntimeException( + "There should only be one split added back at a time. per reader"); + } + + LOG.info("Split returned from reader {}.", i); + // In case of exactly-once (parallelism 1) the single split gets updated with the + // correlation ids and in case of a recovery we have to store this split until we can + // assign it to the recovered reader. + split = list.get(0); + } + + /** + * In the case of exactly-once multiple readers are not allowed. + * + * @see RabbitMQSourceEnumerator#assignSplitToReader(int, RabbitMQSourceSplit) + * @param i reader id + */ + @Override + public void addReader(int i) {} + + /** @return empty enum state object */ + @Override + public RabbitMQSourceEnumState snapshotState(long checkpointId) throws Exception { + return new RabbitMQSourceEnumState(); + } + + @Override + public void close() {} + + private void assignSplitToReader(int readerId, RabbitMQSourceSplit split) { + if (consistencyMode == ConsistencyMode.EXACTLY_ONCE && context.currentParallelism() > 1) { + throw new RuntimeException( + "The consistency mode is exactly-once and a parallelism higher than one was defined. " + + "For exactly once a parallelism higher than one is forbidden."); + } + + SplitsAssignment assignment = new SplitsAssignment<>(split, readerId); + context.assignSplits(assignment); + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/RabbitMQCollector.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/RabbitMQCollector.java new file mode 100644 index 00000000000..1886dc8f75e --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/RabbitMQCollector.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source.reader; + +import org.apache.flink.connector.rabbitmq.source.common.RabbitMQSourceMessageWrapper; +import org.apache.flink.util.Collector; + +import com.rabbitmq.client.AMQP; +import com.rabbitmq.client.Envelope; + +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +/** + * The collector for the messages received from RabbitMQ. Deserialized receive their identifiers + * through {@link #setMessageIdentifiers(String, long)} before they are collected through {@link + * #collect(Object)}. Messages can be polled in order to be processed by the output. + * + * @param The output type of the source. + * @see RabbitMQSourceMessageWrapper + */ +public class RabbitMQCollector implements Collector { + // Queue that holds the messages. + private final BlockingQueue> unpolledMessageQueue; + // Identifiers of the next message that will be collected. + private long deliveryTag; + private String correlationId; + + public RabbitMQCollector() { + this.unpolledMessageQueue = new LinkedBlockingQueue<>(); + } + + /** @return boolean true if there are messages remaining in the collector. */ + public boolean hasUnpolledMessages() { + return !unpolledMessageQueue.isEmpty(); + } + + /** + * Poll a message from the collector. + * + * @return Message the polled message. + */ + public RabbitMQSourceMessageWrapper pollMessage() { + return unpolledMessageQueue.poll(); + } + + /** + * Sets the correlation id and the delivery tag that corresponds to the records originating from + * the RMQ event. If the correlation id has been processed before, records will not be emitted + * downstream. + * + *

If not set explicitly, the {@link AMQP.BasicProperties#getCorrelationId()} and {@link + * Envelope#getDeliveryTag()} will be used. + */ + public void setMessageIdentifiers(String correlationId, long deliveryTag) { + this.correlationId = correlationId; + this.deliveryTag = deliveryTag; + } + + @Override + public void collect(T record) { + unpolledMessageQueue.add( + new RabbitMQSourceMessageWrapper<>(deliveryTag, correlationId, record)); + } + + @Override + public void close() {} +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/RabbitMQSourceReaderBase.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/RabbitMQSourceReaderBase.java new file mode 100644 index 00000000000..fef9b6a5f35 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/RabbitMQSourceReaderBase.java @@ -0,0 +1,243 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source.reader; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.rabbitmq.source.common.RabbitMQSourceMessageWrapper; +import org.apache.flink.connector.rabbitmq.source.enumerator.RabbitMQSourceEnumerator; +import org.apache.flink.connector.rabbitmq.source.split.RabbitMQSourceSplit; +import org.apache.flink.core.io.InputStatus; + +import com.rabbitmq.client.AMQP; +import com.rabbitmq.client.Channel; +import com.rabbitmq.client.Connection; +import com.rabbitmq.client.ConnectionFactory; +import com.rabbitmq.client.DeliverCallback; +import com.rabbitmq.client.Delivery; +import com.rabbitmq.client.Envelope; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static java.util.Objects.requireNonNull; + +/** + * The source reader for RabbitMQ queues. This is the base class of the different consistency modes. + * + * @param The output type of the source. + */ +public abstract class RabbitMQSourceReaderBase implements SourceReader { + protected static final Logger LOG = LoggerFactory.getLogger(RabbitMQSourceReaderBase.class); + + // The assigned split from the enumerator. + private RabbitMQSourceSplit split; + + private Connection rmqConnection; + private Channel rmqChannel; + + private final SourceReaderContext sourceReaderContext; + // The deserialization schema for the messages of RabbitMQ. + private final DeserializationSchema deliveryDeserializer; + // The collector keeps the messages received from RabbitMQ. + private final RabbitMQCollector collector; + + public RabbitMQSourceReaderBase( + SourceReaderContext sourceReaderContext, + DeserializationSchema deliveryDeserializer) { + this.sourceReaderContext = requireNonNull(sourceReaderContext); + this.deliveryDeserializer = requireNonNull(deliveryDeserializer); + this.collector = new RabbitMQCollector<>(); + } + + @Override + public void start() { + LOG.info("Starting source reader and send split request"); + sourceReaderContext.sendSplitRequest(); + } + + // ------------- start RabbitMQ methods -------------- + + private void setupRabbitMQ() throws Exception { + setupConnection(); + setupChannel(); + LOG.info( + "RabbitMQ Connection was successful: Waiting for messages from the queue. To exit press CTRL+C"); + } + + private ConnectionFactory setupConnectionFactory() throws Exception { + return split.getConnectionConfig().getConnectionFactory(); + } + + private void setupConnection() throws Exception { + rmqConnection = setupConnectionFactory().newConnection(); + } + + /** @return boolean whether messages should be automatically acknowledged to RabbitMQ. */ + protected abstract boolean isAutoAck(); + + /** + * This function will be called when a new message from RabbitMQ gets pushed to the source. The + * message will be deserialized and forwarded to our message collector where it is buffered + * until it can be processed. + * + * @param consumerTag The consumer tag of the message. + * @param delivery The delivery from RabbitMQ. + * @throws IOException if something fails during deserialization. + */ + protected void handleMessageReceivedCallback(String consumerTag, Delivery delivery) + throws IOException { + + AMQP.BasicProperties properties = delivery.getProperties(); + byte[] body = delivery.getBody(); + Envelope envelope = delivery.getEnvelope(); + collector.setMessageIdentifiers(properties.getCorrelationId(), envelope.getDeliveryTag()); + deliveryDeserializer.deserialize(body, collector); + } + + protected void setupChannel() throws IOException { + rmqChannel = rmqConnection.createChannel(); + rmqChannel.queueDeclare(split.getQueueName(), true, false, false, null); + + // Set maximum of unacknowledged messages + if (getSplit().getConnectionConfig().getPrefetchCount().isPresent()) { + // global: false - the prefetch count is set per consumer, not per RabbitMQ channel + rmqChannel.basicQos(getSplit().getConnectionConfig().getPrefetchCount().get(), false); + } + + final DeliverCallback deliverCallback = this::handleMessageReceivedCallback; + rmqChannel.basicConsume( + split.getQueueName(), isAutoAck(), deliverCallback, consumerTag -> {}); + } + + // ------------- end RabbitMQ methods -------------- + + /** + * This method provides a hook that is called when a message gets polled by the output. + * + * @param message the message that was polled by the output. + */ + protected void handleMessagePolled(RabbitMQSourceMessageWrapper message) {} + + @Override + public InputStatus pollNext(ReaderOutput output) { + RabbitMQSourceMessageWrapper message = collector.pollMessage(); + if (message == null) { + return InputStatus.NOTHING_AVAILABLE; + } + + output.collect(message.getMessage()); + handleMessagePolled(message); + + return collector.hasUnpolledMessages() + ? InputStatus.MORE_AVAILABLE + : InputStatus.NOTHING_AVAILABLE; + } + + @Override + public List snapshotState(long checkpointId) { + return split != null ? Collections.singletonList(split.copy()) : new ArrayList<>(); + } + + @Override + public CompletableFuture isAvailable() { + return CompletableFuture.runAsync( + () -> { + while (!collector.hasUnpolledMessages()) { + // supposed to be empty + } + }); + } + + /** + * Assign the split from the enumerator. If the source reader already has a split nothing + * happens. After the split is assigned, the connection to RabbitMQ can be setup. + * + * @param list RabbitMQSourceSplits with only one element. + * @see RabbitMQSourceEnumerator + * @see RabbitMQSourceSplit + */ + @Override + public void addSplits(List list) { + if (split != null) { + return; + } + if (list.size() != 1) { + throw new RuntimeException("The number of added splits should be exaclty one."); + } + split = list.get(0); + try { + setupRabbitMQ(); + } catch (Exception e) { + throw new RuntimeException(e.getMessage()); + } + } + + @Override + public void notifyNoMoreSplits() {} + + @Override + public void notifyCheckpointComplete(long checkpointId) throws IOException {} + + /** + * Acknowledge a list of message ids in the RabbitMQ channel. + * + * @param messageIds ids that will be acknowledged. + * @throws RuntimeException if an error occurs during the acknowledgement. + */ + protected void acknowledgeMessageIds(List messageIds) throws IOException { + for (long id : messageIds) { + rmqChannel.basicAck(id, false); + } + } + + @Override + public void notifyCheckpointAborted(long checkpointId) {} + + @Override + public void close() throws Exception { + LOG.info("Close source reader"); + if (getSplit() == null) { + return; + } + + if (rmqChannel != null) { + rmqChannel.close(); + } + + if (rmqConnection != null) { + rmqConnection.close(); + } + } + + protected Channel getRmqChannel() { + return rmqChannel; + } + + protected RabbitMQSourceSplit getSplit() { + return split; + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/specialized/RabbitMQSourceReaderAtLeastOnce.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/specialized/RabbitMQSourceReaderAtLeastOnce.java new file mode 100644 index 00000000000..966644cc969 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/specialized/RabbitMQSourceReaderAtLeastOnce.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source.reader.specialized; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.rabbitmq.source.common.RabbitMQSourceMessageWrapper; +import org.apache.flink.connector.rabbitmq.source.reader.RabbitMQSourceReaderBase; +import org.apache.flink.connector.rabbitmq.source.split.RabbitMQSourceSplit; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +/** + * The RabbitMQSourceReaderAtLeastOnce provides at-least-once guarantee. The deliveryTag from the + * received messages are used to acknowledge the messages once it is assured that they are safely + * consumed by the output. This means that the deliveryTags of the messages that were polled by the + * output are stored separately. Once a snapshot is executed the deliveryTags get associated with + * the checkpoint id. When the checkpoint is completed successfully, all messages from before are + * acknowledged. In the case of a system failure and a successful restart, the messages that are + * unacknowledged, are resend by RabbitMQ. This way at-least-once is guaranteed. + * + *

In order for the at-least-once source reader to work, checkpointing needs to be enabled. + * + * @param The output type of the source. + * @see RabbitMQSourceReaderBase + */ +public class RabbitMQSourceReaderAtLeastOnce extends RabbitMQSourceReaderBase { + // DeliveryTags which corresponding messages were polled by the output since the last + // checkpoint. + private final List polledAndUnacknowledgedMessageIds; + // List of tuples of checkpoint id and deliveryTags that were polled by the output since the + // last checkpoint. + private final BlockingQueue>> + polledAndUnacknowledgedMessageIdsPerCheckpoint; + + public RabbitMQSourceReaderAtLeastOnce( + SourceReaderContext sourceReaderContext, + DeserializationSchema deliveryDeserializer) { + super(sourceReaderContext, deliveryDeserializer); + this.polledAndUnacknowledgedMessageIds = Collections.synchronizedList(new ArrayList<>()); + this.polledAndUnacknowledgedMessageIdsPerCheckpoint = new LinkedBlockingQueue<>(); + } + + @Override + protected boolean isAutoAck() { + return false; + } + + @Override + protected void handleMessagePolled(RabbitMQSourceMessageWrapper message) { + this.polledAndUnacknowledgedMessageIds.add(message.getDeliveryTag()); + } + + @Override + public List snapshotState(long checkpointId) { + Tuple2> tuple = + new Tuple2<>(checkpointId, polledAndUnacknowledgedMessageIds); + polledAndUnacknowledgedMessageIdsPerCheckpoint.add(tuple); + polledAndUnacknowledgedMessageIds.clear(); + + return super.snapshotState(checkpointId); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws IOException { + Iterator>> checkpointIterator = + polledAndUnacknowledgedMessageIdsPerCheckpoint.iterator(); + while (checkpointIterator.hasNext()) { + final Tuple2> nextCheckpoint = checkpointIterator.next(); + long nextCheckpointId = nextCheckpoint.f0; + if (nextCheckpointId <= checkpointId) { + acknowledgeMessageIds(nextCheckpoint.f1); + checkpointIterator.remove(); + } + } + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/specialized/RabbitMQSourceReaderAtMostOnce.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/specialized/RabbitMQSourceReaderAtMostOnce.java new file mode 100644 index 00000000000..73073a336bb --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/specialized/RabbitMQSourceReaderAtMostOnce.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source.reader.specialized; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.rabbitmq.source.reader.RabbitMQSourceReaderBase; + +/** + * The RabbitMQSourceReaderAtMostOnce provides at-most-once guarantee. Messages are automatically + * acknowledged when received from RabbitMQ and afterwards consumed by the output. In case of a + * failure in Flink messages might be lost. + * + * @param The output type of the source. + * @see RabbitMQSourceReaderBase + */ +public class RabbitMQSourceReaderAtMostOnce extends RabbitMQSourceReaderBase { + + public RabbitMQSourceReaderAtMostOnce( + SourceReaderContext sourceReaderContext, + DeserializationSchema deliveryDeserializer) { + super(sourceReaderContext, deliveryDeserializer); + } + + @Override + protected boolean isAutoAck() { + return true; + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/specialized/RabbitMQSourceReaderExactlyOnce.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/specialized/RabbitMQSourceReaderExactlyOnce.java new file mode 100644 index 00000000000..a990317b4cb --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/reader/specialized/RabbitMQSourceReaderExactlyOnce.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source.reader.specialized; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.rabbitmq.source.common.RabbitMQSourceMessageWrapper; +import org.apache.flink.connector.rabbitmq.source.reader.RabbitMQSourceReaderBase; +import org.apache.flink.connector.rabbitmq.source.split.RabbitMQSourceSplit; +import org.apache.flink.util.Preconditions; + +import com.rabbitmq.client.AMQP; +import com.rabbitmq.client.Delivery; +import com.rabbitmq.client.Envelope; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.stream.Collectors; + +/** + * The RabbitMQSourceReaderExactlyOnce provides exactly-once guarantee. The deliveryTag from the + * received messages are used to acknowledge the messages once it is assured that they are safely + * consumed by the output. In addition, correlation ids are used to deduplicate messages. Messages + * polled by the output are stored so they can be later acknowledged. During a checkpoint the + * messages that were polled since the last checkpoint are associated with the id of the current + * checkpoint. Once the checkpoint is completed, the messages for the checkpoint are acknowledged in + * a transaction to assure that RabbitMQ successfully receives the acknowledgements. + * + *

In order for the exactly-once source reader to work, checkpointing needs to be enabled and the + * message from RabbitMQ need to have a correlation id. + * + * @param The output type of the source. + * @see RabbitMQSourceReaderBase + */ +public class RabbitMQSourceReaderExactlyOnce extends RabbitMQSourceReaderBase { + // Message that were polled by the output since the last checkpoint was created. + // These messages are currently forward but not yet acknowledged to RabbitMQ. + // It needs to be ensured they are persisted before they can be acknowledged and thus be delete + // in RabbitMQ. + private final List> + polledAndUnacknowledgedMessagesSinceLastCheckpoint; + + // All message in polledAndUnacknowledgedMessagesSinceLastCheckpoint will move to hear when + // a new checkpoint is created and therefore the messages can be mapped to it. This mapping is + // necessary to ensure we acknowledge only message which belong to a completed checkpoint. + private final BlockingQueue>>> + polledAndUnacknowledgedMessagesPerCheckpoint; + + // Set of correlation ids that have been seen and are not acknowledged yet. + // The message publisher (who pushes the messages to RabbitMQ) is obligated to set the + // correlation id per message and ensure their uniqueness. + private final ConcurrentHashMap.KeySetView correlationIds; + + public RabbitMQSourceReaderExactlyOnce( + SourceReaderContext sourceReaderContext, + DeserializationSchema deliveryDeserializer) { + super(sourceReaderContext, deliveryDeserializer); + this.polledAndUnacknowledgedMessagesSinceLastCheckpoint = + Collections.synchronizedList(new ArrayList<>()); + this.polledAndUnacknowledgedMessagesPerCheckpoint = new LinkedBlockingQueue<>(); + this.correlationIds = ConcurrentHashMap.newKeySet(); + } + + @Override + protected boolean isAutoAck() { + return false; + } + + @Override + protected void handleMessagePolled(RabbitMQSourceMessageWrapper message) { + this.polledAndUnacknowledgedMessagesSinceLastCheckpoint.add(message); + } + + @Override + protected void handleMessageReceivedCallback(String consumerTag, Delivery delivery) + throws IOException { + AMQP.BasicProperties properties = delivery.getProperties(); + String correlationId = properties.getCorrelationId(); + Preconditions.checkNotNull( + correlationId, + "RabbitMQ source was instantiated " + + "with consistencyMode set EXACTLY_ONCE yet we couldn't extract the correlation id from it !"); + + Envelope envelope = delivery.getEnvelope(); + long deliveryTag = envelope.getDeliveryTag(); + + if (correlationIds.add(correlationId)) { + // Handle the message only if the correlation id hasn't been seen before. + // The message will follow the normal process and be acknowledge when it got polled. + super.handleMessageReceivedCallback(consumerTag, delivery); + } else { + // Otherwise, store the new delivery-tag for later acknowledgments. The correlation id + // was seen before and therefore this is a duplicate received from RabbitMQ. + // Instead of letting the message to be polled, the message will directly be marked + // to be acknowledged in the next wave of acknowledgments under their new deliveryTag. + polledAndUnacknowledgedMessagesSinceLastCheckpoint.add( + new RabbitMQSourceMessageWrapper<>(deliveryTag, correlationId)); + } + } + + @Override + public List snapshotState(long checkpointId) { + Tuple2>> tuple = + new Tuple2<>( + checkpointId, + new ArrayList<>(polledAndUnacknowledgedMessagesSinceLastCheckpoint)); + polledAndUnacknowledgedMessagesPerCheckpoint.add(tuple); + polledAndUnacknowledgedMessagesSinceLastCheckpoint.clear(); + + if (getSplit() != null) { + getSplit().setCorrelationIds(correlationIds); + } + return super.snapshotState(checkpointId); + } + + @Override + public void addSplits(List list) { + super.addSplits(list); + correlationIds.addAll(getSplit().getCorrelationIds()); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws IOException { + Iterator>>> checkpointIterator = + polledAndUnacknowledgedMessagesPerCheckpoint.iterator(); + while (checkpointIterator.hasNext()) { + final Tuple2>> nextCheckpoint = + checkpointIterator.next(); + long nextCheckpointId = nextCheckpoint.f0; + if (nextCheckpointId <= checkpointId) { + acknowledgeMessages(nextCheckpoint.f1); + checkpointIterator.remove(); + } + } + } + + @Override + protected void setupChannel() throws IOException { + super.setupChannel(); + // enable channel transactional mode + getRmqChannel().txSelect(); + } + + private void acknowledgeMessages(List> messages) + throws IOException { + List correlationIds = + messages.stream() + .map(RabbitMQSourceMessageWrapper::getCorrelationId) + .collect(Collectors.toList()); + this.correlationIds.removeAll(correlationIds); + try { + List deliveryTags = + messages.stream() + .map(RabbitMQSourceMessageWrapper::getDeliveryTag) + .collect(Collectors.toList()); + acknowledgeMessageIds(deliveryTags); + getRmqChannel().txCommit(); + LOG.info("Successfully acknowledged " + deliveryTags.size() + " messages."); + } catch (IOException e) { + LOG.error( + "Error during acknowledgement of " + + correlationIds.size() + + " messages. CorrelationIds will be rolled back. Error: " + + e.getMessage()); + this.correlationIds.addAll(correlationIds); + throw e; + } + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/split/RabbitMQSourceSplit.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/split/RabbitMQSourceSplit.java new file mode 100644 index 00000000000..975f978d732 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/split/RabbitMQSourceSplit.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source.split; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.connector.rabbitmq.common.RabbitMQConnectionConfig; +import org.apache.flink.connector.rabbitmq.source.enumerator.RabbitMQSourceEnumerator; + +import java.util.HashSet; +import java.util.Set; + +import static java.util.Objects.requireNonNull; + +/** + * This split is passed by the {@link RabbitMQSourceEnumerator} to the SourceReader. It contains the + * configuration for the connection and the name of the queue to connect to. In case of exactly-once + * the correlation ids for deduplication of messages might contain data. They are fIt might contain + * data ife single reader fails and a new reader needs to be * created. + */ +public class RabbitMQSourceSplit implements SourceSplit { + + private final RabbitMQConnectionConfig connectionConfig; + private final String rmqQueueName; + private Set correlationIds; + + public RabbitMQSourceSplit(RabbitMQConnectionConfig connectionConfig, String rmqQueueName) { + this(connectionConfig, rmqQueueName, new HashSet<>()); + } + + public RabbitMQSourceSplit( + RabbitMQConnectionConfig connectionConfig, + String rmqQueueName, + Set correlationIds) { + this.connectionConfig = requireNonNull(connectionConfig); + this.rmqQueueName = requireNonNull(rmqQueueName); + this.correlationIds = requireNonNull(correlationIds); + } + + /** + * Create a copy of the the split. + * + * @return RabbitMQSourceSplit which is a copy of this split. + */ + public RabbitMQSourceSplit copy() { + return new RabbitMQSourceSplit( + connectionConfig, rmqQueueName, new HashSet<>(correlationIds)); + } + + /** + * Get the correlation ids specified in the split. + * + * @return Set of all correlation ids. + */ + public Set getCorrelationIds() { + return correlationIds; + } + + /** + * Get the name of the queue to consume from defined in the split. + * + * @return String name of the queue + */ + public String getQueueName() { + return rmqQueueName; + } + + /** + * Get the connection configuration of RabbitMQ defined in the split. + * + * @return RMQConnectionConfig connection configuration of RabbitMQ. + * @see RabbitMQConnectionConfig + */ + public RabbitMQConnectionConfig getConnectionConfig() { + return connectionConfig; + } + + /** + * Set the correlation ids specified in this split. + * + * @param newCorrelationIds the correlation ids that will be set. + */ + public void setCorrelationIds(Set newCorrelationIds) { + correlationIds = newCorrelationIds; + } + + @Override + public String splitId() { + // Is fixed as there will be only one split that is relevant for the enumerator. + return "0"; + } +} diff --git a/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/split/RabbitMQSourceSplitSerializer.java b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/split/RabbitMQSourceSplitSerializer.java new file mode 100644 index 00000000000..b129caee962 --- /dev/null +++ b/flink-connector-rabbitmq/src/main/java/org/apache/flink/connector/rabbitmq/source/split/RabbitMQSourceSplitSerializer.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source.split; + +import org.apache.flink.connector.rabbitmq.common.RabbitMQConnectionConfig; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.HashSet; +import java.util.Set; + +/** + * The {@link org.apache.flink.core.io.SimpleVersionedSerializer serializer} for {@link + * RabbitMQSourceSplit}. + * + * @see RabbitMQSourceSplit + */ +public class RabbitMQSourceSplitSerializer + implements SimpleVersionedSerializer { + + @Override + public int getVersion() { + return 1; + } + + @Override + public byte[] serialize(RabbitMQSourceSplit rabbitMQSourceSplit) throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos); + serializeV1(out, rabbitMQSourceSplit); + return baos.toByteArray(); + } + + public void serializeV1(DataOutputStream out, RabbitMQSourceSplit rabbitMQSourceSplit) + throws IOException { + ObjectOutputStream objectOutputStream = new ObjectOutputStream(out); + objectOutputStream.writeObject(rabbitMQSourceSplit.getConnectionConfig()); + out.writeUTF(rabbitMQSourceSplit.getQueueName()); + writeStringSet(out, rabbitMQSourceSplit.getCorrelationIds()); + out.flush(); + } + + @Override + public RabbitMQSourceSplit deserialize(int version, byte[] bytes) throws IOException { + switch (version) { + case 1: + return deserializeV1(bytes); + default: + throw new IOException("Unrecognized version or corrupt state: " + version); + } + } + + public RabbitMQSourceSplit deserializeV1(byte[] bytes) throws IOException { + try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes); + DataInputStream in = new DataInputStream(bais); + ObjectInputStream objectInputStream = new ObjectInputStream(in)) { + RabbitMQConnectionConfig config = + (RabbitMQConnectionConfig) objectInputStream.readObject(); + // Queue names may be up to 255 bytes of UTF-8 characters. + String queueName = in.readUTF(); + Set correlationIds = readStringSet(in); + return new RabbitMQSourceSplit(config, queueName, correlationIds); + } catch (ClassNotFoundException e) { + throw new RuntimeException(e.getException()); + } + } + + private static void writeStringSet(DataOutputStream out, Set strings) + throws IOException { + out.writeInt(strings.size()); + for (String string : strings) { + out.writeUTF(string); + } + } + + private static Set readStringSet(DataInputStream in) throws IOException { + final int len = in.readInt(); + final Set strings = new HashSet<>(); + for (int i = 0; i < len; i++) { + strings.add(in.readUTF()); + } + return strings; + } +} diff --git a/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/common/RabbitMQBaseTest.java b/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/common/RabbitMQBaseTest.java new file mode 100644 index 00000000000..d72627a5f00 --- /dev/null +++ b/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/common/RabbitMQBaseTest.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.common; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.connector.rabbitmq.sink.RabbitMQSink; +import org.apache.flink.connector.rabbitmq.source.RabbitMQSource; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.MiniClusterWithClientResource; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.rules.Timeout; +import org.testcontainers.containers.RabbitMQContainer; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeoutException; + +/** + * The base class for RabbitMQ tests. It sets up a flink cluster and a docker image for RabbitMQ. It + * provides behavior to easily add onto the stream, send message to RabbitMQ and get the messages in + * RabbitMQ. + */ +public abstract class RabbitMQBaseTest { + + private static final int RABBITMQ_PORT = 5672; + private RabbitMQContainerClient client; + protected StreamExecutionEnvironment env; + + @Rule public Timeout globalTimeout = Timeout.seconds(20); + + @Rule + public MiniClusterWithClientResource flinkCluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberSlotsPerTaskManager(1) + .setNumberTaskManagers(1) + .build()); + + @Rule + public RabbitMQContainer rabbitMq = + new RabbitMQContainer( + DockerImageName.parse("rabbitmq").withTag("3.7.25-management-alpine")) + .withExposedPorts(RABBITMQ_PORT); + + @Before + public void setUpContainerClient() { + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(1000); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(10, 1000)); + this.client = new RabbitMQContainerClient<>(rabbitMq); + } + + protected void executeFlinkJob() { + JobGraph job = env.getStreamGraph().getJobGraph(); + flinkCluster.getClusterClient().submitJob(job); + } + + public RabbitMQContainerClient addSinkOn( + DataStream stream, ConsistencyMode consistencyMode, int countDownLatchSize) + throws IOException, TimeoutException { + RabbitMQContainerClient client = + new RabbitMQContainerClient<>( + rabbitMq, new SimpleStringSchema(), countDownLatchSize); + String queueName = client.createQueue(); + final RabbitMQConnectionConfig connectionConfig = + new RabbitMQConnectionConfig.Builder() + .setHost(rabbitMq.getHost()) + .setVirtualHost("/") + .setUserName(rabbitMq.getAdminUsername()) + .setPassword(rabbitMq.getAdminPassword()) + .setPort(rabbitMq.getMappedPort(RABBITMQ_PORT)) + .build(); + + RabbitMQSink sink = + RabbitMQSink.builder() + .setConnectionConfig(connectionConfig) + .setQueueName(queueName) + .setSerializationSchema(new SimpleStringSchema()) + .setConsistencyMode(consistencyMode) + .build(); + stream.sinkTo(sink).setParallelism(1); + return client; + } + + protected DataStream addSourceOn( + StreamExecutionEnvironment env, ConsistencyMode consistencyMode) + throws IOException, TimeoutException { + String queueName = client.createQueue(false); + + final RabbitMQConnectionConfig connectionConfig = + new RabbitMQConnectionConfig.Builder() + .setHost(rabbitMq.getHost()) + .setVirtualHost("/") + .setUserName(rabbitMq.getAdminUsername()) + .setPassword(rabbitMq.getAdminPassword()) + .setPort(rabbitMq.getMappedPort(RABBITMQ_PORT)) + .build(); + + RabbitMQSource rabbitMQSource = + RabbitMQSource.builder() + .setConnectionConfig(connectionConfig) + .setQueueName(queueName) + .setDeserializationSchema(new SimpleStringSchema()) + .setConsistencyMode(consistencyMode) + .build(); + + return env.fromSource(rabbitMQSource, WatermarkStrategy.noWatermarks(), "RabbitMQSource") + .setParallelism(1); + } + + protected void sendToRabbit(List messages) throws IOException { + client.sendMessages(new SimpleStringSchema(), messages); + } + + protected void sendToRabbit(List messages, List correlationIds) + throws IOException { + for (int i = 0; i < messages.size(); i++) { + client.sendMessage(new SimpleStringSchema(), messages.get(i), correlationIds.get(i)); + } + } + + protected List getRandomMessages(int numberOfMessages) { + List messages = new ArrayList<>(); + for (int i = 0; i < numberOfMessages; i++) { + messages.add(UUID.randomUUID().toString()); + } + return messages; + } + + protected List getSequentialMessages(int numberOfMessages) { + List messages = new ArrayList<>(); + for (int i = 0; i < numberOfMessages; i++) { + messages.add("Message " + i); + } + return messages; + } +} diff --git a/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/common/RabbitMQContainerClient.java b/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/common/RabbitMQContainerClient.java new file mode 100644 index 00000000000..66bdb9a098b --- /dev/null +++ b/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/common/RabbitMQContainerClient.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.common; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; + +import com.rabbitmq.client.AMQP; +import com.rabbitmq.client.Channel; +import com.rabbitmq.client.Connection; +import com.rabbitmq.client.ConnectionFactory; +import com.rabbitmq.client.DeliverCallback; +import com.rabbitmq.client.Delivery; +import org.testcontainers.containers.RabbitMQContainer; +import org.testcontainers.containers.wait.strategy.Wait; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeoutException; + +/** + * This class provides a RabbitMQ container client which allows creating queues, sending messages to + * RabbitMQ and get the messages received by RabbitMQ. + */ +public class RabbitMQContainerClient { + + private final RabbitMQContainer container; + private Channel channel; + private final Queue messages; + private String queueName; + private final CountDownLatch latch; + private final DeserializationSchema valueDeserializer; + + public RabbitMQContainerClient( + RabbitMQContainer container, + DeserializationSchema valueDeserializer, + int countDownLatchSize) { + container.withExposedPorts(5762).waitingFor(Wait.forListeningPort()); + this.container = container; + this.messages = new LinkedList<>(); + this.latch = new CountDownLatch(countDownLatchSize); + this.valueDeserializer = valueDeserializer; + } + + public RabbitMQContainerClient(RabbitMQContainer container) { + this(container, null, 0); + } + + public String createQueue(String queueName, boolean withConsumer) + throws IOException, TimeoutException { + this.queueName = queueName; + Connection connection = getRabbitMQConnection(); + this.channel = connection.createChannel(); + channel.queueDeclare(queueName, true, false, false, null); + if (withConsumer) { + final DeliverCallback deliverCallback = this::handleMessageReceivedCallback; + channel.basicConsume(queueName, true, deliverCallback, consumerTag -> {}); + } + return this.queueName; + } + + public String createQueue() throws IOException, TimeoutException { + return createQueue(UUID.randomUUID().toString(), true); + } + + public String createQueue(boolean withConsumer) throws IOException, TimeoutException { + return createQueue(UUID.randomUUID().toString(), withConsumer); + } + + public void sendMessages(SerializationSchema valueSerializer, List messages) + throws IOException { + for (T message : messages) { + channel.basicPublish("", queueName, null, valueSerializer.serialize(message)); + } + } + + public void sendMessage(SerializationSchema valueSerializer, T message, String correlationId) + throws IOException { + AMQP.BasicProperties.Builder builder = new AMQP.BasicProperties.Builder(); + builder.correlationId(correlationId); + AMQP.BasicProperties properties = builder.build(); + channel.basicPublish("", queueName, properties, valueSerializer.serialize(message)); + } + + public List getConsumedMessages() throws IOException { + List deserializedMessages = new ArrayList<>(); + while (!messages.isEmpty()) { + T message = valueDeserializer.deserialize(messages.poll()); + deserializedMessages.add(message); + } + return deserializedMessages; + } + + public void await() throws InterruptedException { + latch.await(); + } + + private void handleMessageReceivedCallback(String consumerTag, Delivery delivery) { + byte[] body = delivery.getBody(); + messages.add(body); + if (latch != null) { + latch.countDown(); + } + } + + private Connection getRabbitMQConnection() throws TimeoutException, IOException { + ConnectionFactory factory = new ConnectionFactory(); + + factory.setUsername(container.getAdminUsername()); + factory.setPassword(container.getAdminPassword()); + factory.setVirtualHost("/"); + factory.setHost(container.getHost()); + factory.setPort(container.getAmqpPort()); + + return factory.newConnection(); + } +} diff --git a/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/sink/RabbitMQSinkITCase.java b/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/sink/RabbitMQSinkITCase.java new file mode 100644 index 00000000000..4753213c03a --- /dev/null +++ b/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/sink/RabbitMQSinkITCase.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink; + +import org.apache.flink.connector.rabbitmq.common.ConsistencyMode; +import org.apache.flink.connector.rabbitmq.common.RabbitMQBaseTest; +import org.apache.flink.connector.rabbitmq.common.RabbitMQContainerClient; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.source.SourceFunction; + +import org.junit.Before; +import org.junit.Test; + +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * The tests for the RabbitMQ sink with different consistency modes. As the tests are working a lot + * with timeouts to uphold stream it is possible that tests might fail. + */ +public class RabbitMQSinkITCase extends RabbitMQBaseTest { + + private static AtomicBoolean shouldFail; + + @Before + public void setup() { + shouldFail = new AtomicBoolean(true); + } + + private static class GeneratorFailureSource implements SourceFunction { + + private final BlockingQueue messagesToSend; + private int failAtNthMessage; + + public GeneratorFailureSource(BlockingQueue messagesToSend, int failAtNthMessage) { + this.messagesToSend = messagesToSend; + this.failAtNthMessage = failAtNthMessage; + shouldFail.set(true); + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + while (true) { + if (failAtNthMessage == 0 && shouldFail.get()) { + shouldFail.set(false); + throw new Exception("Supposed to Fail"); + } + failAtNthMessage -= 1; + String message = messagesToSend.take(); + sourceContext.collect(message); + } + } + + @Override + public void cancel() {} + } + + @Test + public void atMostOnceTest() throws Exception { + List messages = getRandomMessages(100); + + DataStream stream = env.fromCollection(messages); + RabbitMQContainerClient client = + addSinkOn(stream, ConsistencyMode.AT_MOST_ONCE, messages.size()); + executeFlinkJob(); + client.await(); + + List receivedMessages = client.getConsumedMessages(); + assertEquals(messages, receivedMessages); + } + + @Test + public void atLeastOnceTest() throws Exception { + List messages = getRandomMessages(100); + DataStream stream = env.fromCollection(messages); + RabbitMQContainerClient client = + addSinkOn(stream, ConsistencyMode.AT_LEAST_ONCE, messages.size()); + + executeFlinkJob(); + client.await(); + + List receivedMessages = client.getConsumedMessages(); + assertEquals(messages, receivedMessages); + } + + @Test + public void atLeastOnceWithFlinkFailureTest() throws Exception { + LinkedBlockingQueue messages = new LinkedBlockingQueue<>(getRandomMessages(100)); + + GeneratorFailureSource source = new GeneratorFailureSource(messages, 30); + + DataStream stream = env.addSource(source); + RabbitMQContainerClient client = + addSinkOn(stream, ConsistencyMode.AT_LEAST_ONCE, messages.size() + 30); + + executeFlinkJob(); + client.await(); + + List receivedMessages = client.getConsumedMessages(); + assertTrue(receivedMessages.containsAll(messages)); + } + + @Test + public void exactlyOnceTest() throws Exception { + LinkedBlockingQueue messages = new LinkedBlockingQueue<>(getRandomMessages(100)); + env.enableCheckpointing(100); + + GeneratorFailureSource source = new GeneratorFailureSource(messages, -1); + DataStream stream = env.addSource(source); + RabbitMQContainerClient client = + addSinkOn(stream, ConsistencyMode.EXACTLY_ONCE, messages.size()); + + executeFlinkJob(); + client.await(); + + assertArrayEquals(messages.toArray(), client.getConsumedMessages().toArray()); + } + + @Test + public void exactlyOnceWithFlinkFailureTest() throws Exception { + LinkedBlockingQueue messages = new LinkedBlockingQueue<>(getRandomMessages(100)); + env.enableCheckpointing(100); + + GeneratorFailureSource source = new GeneratorFailureSource(messages, 80); + DataStream stream = env.addSource(source); + RabbitMQContainerClient client = + addSinkOn(stream, ConsistencyMode.EXACTLY_ONCE, messages.size()); + + executeFlinkJob(); + client.await(); + + assertArrayEquals(messages.toArray(), client.getConsumedMessages().toArray()); + } +} diff --git a/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/sink/state/RabbitMQSinkWriterStateSerializerTest.java b/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/sink/state/RabbitMQSinkWriterStateSerializerTest.java new file mode 100644 index 00000000000..2566dc5fd4b --- /dev/null +++ b/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/sink/state/RabbitMQSinkWriterStateSerializerTest.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.sink.state; + +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.connector.rabbitmq.sink.common.RabbitMQSinkMessageWrapper; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +/** Test the sink writer state serializer. */ +public class RabbitMQSinkWriterStateSerializerTest { + + private RabbitMQSinkWriterState getSinkWriterState() { + List> outStandingMessages = new ArrayList<>(); + SimpleStringSchema serializer = new SimpleStringSchema(); + for (int i = 0; i < 5; i++) { + String message = "Message " + i; + RabbitMQSinkMessageWrapper messageWrapper = + new RabbitMQSinkMessageWrapper<>(message, serializer.serialize(message)); + outStandingMessages.add(messageWrapper); + } + return new RabbitMQSinkWriterState<>(outStandingMessages); + } + + @Test + public void testWriterStateSerializer() throws IOException { + RabbitMQSinkWriterState writerState = getSinkWriterState(); + RabbitMQSinkWriterStateSerializer serializer = + new RabbitMQSinkWriterStateSerializer<>(); + + byte[] serializedWriterState = serializer.serialize(writerState); + RabbitMQSinkWriterState deserializedWriterState = + serializer.deserialize(serializer.getVersion(), serializedWriterState); + + List expectedBytes = + writerState.getOutstandingMessages().stream() + .map(RabbitMQSinkMessageWrapper::getBytes) + .collect(Collectors.toList()); + List actualBytes = + deserializedWriterState.getOutstandingMessages().stream() + .map(RabbitMQSinkMessageWrapper::getBytes) + .collect(Collectors.toList()); + + assertEquals(expectedBytes.size(), actualBytes.size()); + for (int i = 0; i < expectedBytes.size(); i++) { + Assert.assertArrayEquals(expectedBytes.get(i), actualBytes.get(i)); + } + + List actualMessages = + deserializedWriterState.getOutstandingMessages().stream() + .map(RabbitMQSinkMessageWrapper::getMessage) + .collect(Collectors.toList()); + + for (String message : actualMessages) { + assertNull(message); + } + } + + @Test + public void testWriterStateSerializerWithDeserializationSchema() throws IOException { + RabbitMQSinkWriterState writerState = getSinkWriterState(); + SimpleStringSchema deserializer = new SimpleStringSchema(); + RabbitMQSinkWriterStateSerializer serializer = + new RabbitMQSinkWriterStateSerializer<>(deserializer); + + byte[] serializedWriterState = serializer.serialize(writerState); + RabbitMQSinkWriterState deserializedWriterState = + serializer.deserialize(serializer.getVersion(), serializedWriterState); + + List expectedMessages = + writerState.getOutstandingMessages().stream() + .map(RabbitMQSinkMessageWrapper::getMessage) + .collect(Collectors.toList()); + List expectedBytes = + writerState.getOutstandingMessages().stream() + .map(RabbitMQSinkMessageWrapper::getBytes) + .collect(Collectors.toList()); + + List actualMessages = + deserializedWriterState.getOutstandingMessages().stream() + .map(RabbitMQSinkMessageWrapper::getMessage) + .collect(Collectors.toList()); + List actualBytes = + deserializedWriterState.getOutstandingMessages().stream() + .map(RabbitMQSinkMessageWrapper::getBytes) + .collect(Collectors.toList()); + + assertEquals(expectedMessages, actualMessages); + assertEquals(expectedBytes.size(), actualBytes.size()); + for (int i = 0; i < expectedBytes.size(); i++) { + Assert.assertArrayEquals(expectedBytes.get(i), actualBytes.get(i)); + } + } +} diff --git a/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/source/RabbitMQSourceITCase.java b/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/source/RabbitMQSourceITCase.java new file mode 100644 index 00000000000..717b4928b1c --- /dev/null +++ b/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/source/RabbitMQSourceITCase.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source; + +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.connector.rabbitmq.common.ConsistencyMode; +import org.apache.flink.connector.rabbitmq.common.RabbitMQBaseTest; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; + +import org.apache.commons.collections.CollectionUtils; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * The tests for the RabbitMQ source with different consistency modes. As the tests are working a + * lot with timeouts to uphold stream it is possible that tests might fail. + */ +public class RabbitMQSourceITCase extends RabbitMQBaseTest { + + private static final List collectedMessages = + Collections.synchronizedList(new ArrayList<>()); + private static CountDownLatch messageLatch; + private static CountDownLatch checkpointLatch; + private static int failAtNthMessage; + + @Before + public void setup() { + collectedMessages.clear(); + failAtNthMessage = -1; + messageLatch = null; + } + + /** CollectSink to access the messages from the stream. */ + private static class CollectSink implements SinkFunction, CheckpointListener { + + public static void addOnStream(DataStream stream) { + stream.addSink(new CollectSink()).setParallelism(1); + } + + @Override + public void invoke(String value, Context context) throws Exception { + if (failAtNthMessage > 0) { + failAtNthMessage -= 1; + if (failAtNthMessage == 0) { + throw new Exception("This is supposed to be thrown."); + } + } + collectedMessages.add(value); + messageLatch.countDown(); + } + + @Override + public void notifyCheckpointComplete(long l) { + if (checkpointLatch != null) { + checkpointLatch.countDown(); + } + } + } + + // --------------- at most once --------------- + @Test + public void atMostOnceTest() throws Exception { + List messages = getRandomMessages(100); + messageLatch = new CountDownLatch(messages.size()); + + DataStream stream = addSourceOn(env, ConsistencyMode.AT_MOST_ONCE); + CollectSink.addOnStream(stream); + executeFlinkJob(); + + sendToRabbit(messages); + messageLatch.await(); + + assertEquals( + CollectionUtils.getCardinalityMap(messages), + CollectionUtils.getCardinalityMap(collectedMessages)); + } + + // --------------- at least once --------------- + @Test + public void atLeastOnceTest() throws Exception { + List messages = getRandomMessages(100); + DataStream stream = addSourceOn(env, ConsistencyMode.AT_LEAST_ONCE); + messageLatch = new CountDownLatch(messages.size()); + CollectSink.addOnStream(stream); + executeFlinkJob(); + + sendToRabbit(messages); + messageLatch.await(); + + assertEquals( + CollectionUtils.getCardinalityMap(messages), + CollectionUtils.getCardinalityMap(collectedMessages)); + } + + @Test + public void atLeastOnceFailureTest() throws Exception { + // An exception is thrown in the MapFunction in order to trigger a restart of Flink and it + // is assured that the source receives the messages again. + DataStream stream = addSourceOn(env, ConsistencyMode.AT_LEAST_ONCE); + + List messages = getSequentialMessages(100); + failAtNthMessage = 30; + messageLatch = new CountDownLatch(messages.size() + failAtNthMessage - 1); + CollectSink.addOnStream(stream); + + executeFlinkJob(); + + sendToRabbit(messages); + messageLatch.await(); + + assertTrue(collectedMessages.containsAll(messages)); + } + + // --------------- exactly once --------------- + @Test + public void exactlyOnceTest() throws Exception { + List messages = getRandomMessages(1000); + messageLatch = new CountDownLatch(messages.size()); + + DataStream stream = addSourceOn(env, ConsistencyMode.EXACTLY_ONCE); + CollectSink.addOnStream(stream); + + executeFlinkJob(); + + // use messages as correlation ids here + sendToRabbit(messages, messages); + messageLatch.await(); + + assertEquals(messages, collectedMessages); + } + + @Test + public void exactlyOnceFilterCorrelationIdsTest() throws Exception { + List messages = getRandomMessages(5); + List correlationIds = Arrays.asList("1", "2", "3", "3", "3"); + messageLatch = new CountDownLatch(3); + + env.enableCheckpointing(5000); + DataStream stream = addSourceOn(env, ConsistencyMode.EXACTLY_ONCE); + CollectSink.addOnStream(stream); + executeFlinkJob(); + + sendToRabbit(messages, correlationIds); + + messageLatch.await(); + + List expectedMessages = messages.subList(0, 3); + assertEquals(expectedMessages, collectedMessages); + } + + /** + * This test is supposed to check if we receive all messages once again which were polled after + * the checkpoint and before the exception thrown by the test. Thus, these messages were not yet + * acknowledged to RabbitMQ and therefore will be consumed once again after the recovery. This + * checks that messages will not be lost on failures. + * + *

The CollectSink has no checkpoint logic and will collect message twice. The test expect + * that all messages before the checkpoint are received twice by the CollectSink. + * + * @throws Exception something not supposed failed + */ + @Test + public void exactlyOnceWithFailureAndMessageDuplicationTest() throws Exception { + // An exception is thrown in order to trigger a restart of Flink and it + // is assured that the system receives the messages only once. We disable + // (by setting the interval higher than the test duration) checkpoint to + // expect receiving all pre-exception messages once again. + env.enableCheckpointing(500000); + DataStream stream = addSourceOn(env, ConsistencyMode.EXACTLY_ONCE); + + List messages = getRandomMessages(100); + + int originalFailAthNthMessage = 30; + failAtNthMessage = originalFailAthNthMessage; + messageLatch = new CountDownLatch(messages.size() + failAtNthMessage - 1); + CollectSink.addOnStream(stream); + executeFlinkJob(); + + sendToRabbit(messages, messages); + messageLatch.await(); + + List expectedMessage = + collectedMessages.subList(originalFailAthNthMessage - 1, collectedMessages.size()); + assertEquals(messages, expectedMessage); + } + + /** + * This test checks that messages which were consumed and polled before a successful and + * completed checkpoint will not be consumed from RabbitMQ a second time if a failure happens. + * This mean that these messages will not be polled a second time from Flink (after recovery) as + * well and therefore no duplicated are expected in the CollectSink. + * + * @throws Exception something not supposed failed + */ + @Test + public void exactlyOnceWithFailureWithNoMessageDuplicationTest() throws Exception { + env.enableCheckpointing(1000); + DataStream stream = addSourceOn(env, ConsistencyMode.EXACTLY_ONCE); + + List messages = getSequentialMessages(60); + List messagesA = messages.subList(0, 30); + List messagesB = messages.subList(30, messages.size()); + + failAtNthMessage = messagesA.size() + 1; + messageLatch = new CountDownLatch(messagesA.size() + messagesB.size()); + + CollectSink.addOnStream(stream); + executeFlinkJob(); + + // Send first batch of messages + sendToRabbit(messagesA, messagesA); + + // Wait for successful checkpoints to ensure the previous message are acknowledged and + // thus will not be polled a second . + checkpointLatch = new CountDownLatch(2); + checkpointLatch.await(); + + // Send second batch of messages + sendToRabbit(messagesB, messagesB); + + messageLatch.await(); + + // Expect all message to be received without duplications + assertEquals(messages, collectedMessages); + } +} diff --git a/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/source/split/RabbitMQSourceSplitSerializerTest.java b/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/source/split/RabbitMQSourceSplitSerializerTest.java new file mode 100644 index 00000000000..4b70bc5b425 --- /dev/null +++ b/flink-connector-rabbitmq/src/test/java/org/apache/flink/connector/rabbitmq/source/split/RabbitMQSourceSplitSerializerTest.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.rabbitmq.source.split; + +import org.apache.flink.connector.rabbitmq.common.RabbitMQConnectionConfig; + +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** Test the source split serializer. */ +public class RabbitMQSourceSplitSerializerTest { + + private RabbitMQSourceSplit getSourceSplit() { + + String queueName = "exampleQueueName"; + List ids = new ArrayList<>(); + for (int i = 0; i < 1000; i++) { + ids.add(Integer.toString(i)); + } + Set correlationIds = new HashSet<>(ids); + final RabbitMQConnectionConfig connectionConfig = + new RabbitMQConnectionConfig.Builder() + .setHost("Host") + .setVirtualHost("/") + .setUserName("Username") + .setPassword("Password") + .setPort(3000) + .build(); + return new RabbitMQSourceSplit(connectionConfig, queueName, correlationIds); + } + + @Test + public void testSplitSerializer() throws IOException { + RabbitMQSourceSplit split = getSourceSplit(); + RabbitMQSourceSplitSerializer serializer = new RabbitMQSourceSplitSerializer(); + + byte[] serializedSplit = serializer.serialize(split); + RabbitMQSourceSplit deserializedSplit = serializer.deserialize(1, serializedSplit); + + assertNotNull(deserializedSplit); + assertEquals(split.splitId(), deserializedSplit.splitId()); + assertEquals(split.getCorrelationIds(), deserializedSplit.getCorrelationIds()); + assertEquals(split.getQueueName(), deserializedSplit.getQueueName()); + assertEquals( + split.getConnectionConfig().getHost(), + deserializedSplit.getConnectionConfig().getHost()); + } +} diff --git a/pom.xml b/pom.xml new file mode 100644 index 00000000000..1ecba783374 --- /dev/null +++ b/pom.xml @@ -0,0 +1,1371 @@ + + + + + + org.apache + apache + 20 + + + 4.0.0 + + org.apache.flink + flink-connectors + 1.16-SNAPSHOT + Flink : Connectors : + pom + https://flink.apache.org + 2022 + + + + The Apache Software License, Version 2.0 + https://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + https://github.com/apache/flink-connector-rabbitmq + git@github.com:apache/flink-connector-rabbitmq.git + + scm:git:https://gitbox.apache.org/repos/asf/flink-connector-rabbitmq.git + + + + + flink-connector-rabbitmq + + + + UTF-8 + UTF-8 + + 1.16-SNAPSHOT + 15.0 + 2.12.4 + + + 1.8 + ${target.java.version} + ${target.java.version} + 2.12.7 + 2.12 + + 4.13.2 + 5.8.1 + 3.21.0 + 0.22.0 + 1.16.2 + 2.21.0 + 2.4.2 + + false + 1.14.0 + tools/japicmp-output + + 1.7.36 + 2.17.2 + + validate + + **/*Test.* + + + + + org.apache.flink + flink-shaded-force-shading + ${flink.shaded.version} + + + + + + + org.slf4j + slf4j-api + provided + + + + + com.google.code.findbugs + jsr305 + provided + + + + + org.junit.jupiter + junit-jupiter + test + + + + org.junit.vintage + junit-vintage-engine + test + + + + org.assertj + assertj-core + test + + + + org.mockito + mockito-core + ${mockito.version} + jar + test + + + + org.testcontainers + junit-jupiter + test + + + + + + org.apache.logging.log4j + log4j-slf4j-impl + test + + + + org.apache.logging.log4j + log4j-api + test + + + + org.apache.logging.log4j + log4j-core + test + + + + + org.apache.logging.log4j + log4j-1.2-api + test + + + + org.apache.flink + flink-test-utils-junit + test + + + + + org.apache.flink + flink-architecture-tests-test + test + + + org.apache.flink + flink-architecture-tests-production + test + + + + + + + + + + + + + org.apache.flink + flink-shaded-asm-9 + 9.2-${flink.shaded.version} + + + + org.apache.flink + flink-shaded-guava + 30.1.1-jre-${flink.shaded.version} + + + + org.apache.flink + flink-shaded-jackson + ${flink.shaded.jackson.version}-${flink.shaded.version} + + + + org.apache.flink + flink-shaded-jackson-module-jsonSchema + ${flink.shaded.jackson.version}-${flink.shaded.version} + + + + org.apache.flink + flink-shaded-netty + 4.1.70.Final-${flink.shaded.version} + + + + org.apache.flink + flink-shaded-netty-tcnative-dynamic + 2.0.44.Final-${flink.shaded.version} + test + + + + + + org.apache.flink + flink-test-utils + ${flink.version} + test + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + + + org.apache.flink + flink-test-utils-junit + ${flink.version} + test + + + + + + org.apache.flink + flink-architecture-tests-base + ${flink.version} + test + + + + org.apache.flink + flink-architecture-tests-test + ${flink.version} + test + + + + org.apache.flink + flink-architecture-tests-production + ${flink.version} + test + + + + + com.google.code.findbugs + jsr305 + 1.3.9 + + + + org.slf4j + slf4j-api + ${slf4j.version} + + + + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j.version} + + + + org.apache.logging.log4j + log4j-api + ${log4j.version} + + + + org.apache.logging.log4j + log4j-core + ${log4j.version} + + + + + org.apache.logging.log4j + log4j-1.2-api + ${log4j.version} + + + + org.apache.commons + commons-lang3 + 3.3.2 + + + + + com.fasterxml.jackson + jackson-bom + pom + import + 2.13.0 + + + + + org.junit + junit-bom + ${junit5.version} + pom + import + + + + junit + junit + ${junit4.version} + + + + org.assertj + assertj-core + ${assertj.version} + test + + + + org.xerial.snappy + snappy-java + 1.1.8.3 + + + + + com.esotericsoftware.kryo + kryo + 2.24.0 + + + + + org.objenesis + objenesis + 2.1 + + + + org.scala-lang + scala-compiler + ${scala.version} + + + + org.scala-lang + scala-library + ${scala.version} + + + + org.testcontainers + testcontainers-bom + ${testcontainers.version} + pom + import + + + + com.tngtech.archunit + archunit + ${archunit.version} + test + + + + com.tngtech.archunit + archunit-junit5 + ${archunit.version} + test + + + + + + + + + java11 + + [11,) + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + 1.7 + + + org.apache.maven.plugins + maven-shade-plugin + 3.2.4 + + + com.github.siom79.japicmp + japicmp-maven-plugin + + + javax.xml.bind + jaxb-api + 2.3.0 + + + com.sun.xml.bind + jaxb-impl + 2.3.1 + + + com.sun.xml.bind + jaxb-core + 2.3.0 + + + javax.activation + activation + 1.1.1 + + + + + org.apache.maven.plugins + maven-surefire-plugin + + org.apache.flink.testutils.junit.FailsOnJava11 + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + --add-exports=java.base/sun.net.util=ALL-UNNAMED + + + + + + + + + + java17 + + [17,) + + + + + + + com.diffplug.spotless + spotless-maven-plugin + + + true + + + + + + + + + java11-target + + + + org.apache.maven.plugins + maven-compiler-plugin + + 11 + 11 + + --add-exports=java.base/sun.net.util=ALL-UNNAMED + --add-exports=java.management/sun.management=ALL-UNNAMED + --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED + --add-exports=java.security.jgss/sun.security.krb5=ALL-UNNAMED + + + + + + + + + java17-target + + + + org.apache.maven.plugins + maven-compiler-plugin + + 17 + 17 + + --add-exports=java.base/sun.net.util=ALL-UNNAMED + --add-exports=java.management/sun.management=ALL-UNNAMED + --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED + --add-exports=java.security.jgss/sun.security.krb5=ALL-UNNAMED + + + + + + + + + fast + + + fast + + + + + + + org.apache.rat + apache-rat-plugin + + true + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + true + + + + com.diffplug.spotless + spotless-maven-plugin + + true + + + + org.scalastyle + scalastyle-maven-plugin + + true + + + + org.apache.maven.plugins + maven-enforcer-plugin + + true + + + + org.apache.maven.plugins + maven-javadoc-plugin + + true + + + + com.github.siom79.japicmp + japicmp-maven-plugin + + true + + + + + + + + + check-convergence + + + check-convergence + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + ${flink.convergence.phase} + + + + + + + + + + docs-and-source + + + docs-and-source + + + + + + org.apache.maven.plugins + maven-source-plugin + 2.2.1 + + + attach-sources + + jar + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + true + + + + attach-javadocs + + jar + + + + + + + + + + release + + + release + + + + 1.8 + + + + + org.apache.maven.plugins + maven-gpg-plugin + 1.4 + + + sign-artifacts + verify + + sign + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + enforce-maven + + enforce + + + + + + (,3.3) + + + 1.8.0 + + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + attach-javadocs + + jar + + + + + + + + + org.apache.maven.plugins + maven-release-plugin + 2.1 + + forked-path + false + ${arguments} -Psonatype-oss-release + + + + + + + + + + + + + org.apache.felix + maven-bundle-plugin + 3.0.1 + true + true + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + false + + true + true + + + + + + + org.apache.rat + apache-rat-plugin + 0.13 + false + + + verify + + check + + + + + true + false + 0 + + + + AL2 + Apache License 2.0 + + + Licensed to the Apache Software Foundation (ASF) under one + + + + + + Apache License 2.0 + + + + + **/.*/** + **/*.prefs + **/*.log + + + **/src/test/resources/*-data + out/test/flink-avro/avro/user.avsc + + + **/archunit-violations/** + + + **/README.md + .github/** + + **/*.iml + + out/** + **/target/** + docs/layouts/shortcodes/generated/** + docs/static/generated/** + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + com.diffplug.spotless + spotless-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + + + + org.apache.maven.plugins + maven-surefire-plugin + 3.0.0-M5 + + + + false + + 0${surefire.forkNumber} + true + true + + random + ${project.basedir} + + ${test.randomization.seed} + true + + -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} -XX:+UseG1GC -Duser.country=US -Duser.language=en + + + + + default-test + test + + test + + + + ${test.unit.pattern} + + + + + + integration-tests + integration-test + + test + + + + **/*.* + + + ${test.unit.pattern} + + **/*$* + + false + + + + + + org.apache.maven.plugins + maven-eclipse-plugin + 2.8 + + + + org.eclipse.jdt.launching.JRE_CONTAINER + + + true + true + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + enforce-maven + + enforce + + + + + + [3.1.1,) + + + ${target.java.version} + + + + + + forbid-direct-table-planner-dependencies + + enforce + + + + + + org.apache.flink:flink-table-planner_${scala.binary.version} + + + org.apache.flink:flink-table-planner_${scala.binary.version}:*:*:test + + + Direct dependencies on flink-table-planner are not allowed. + You should depend on either Table API modules or flink-table-planner-loader. + + + + + + + dependency-convergence + + none + + enforce + + + + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + + + * + + + log4j.properties + log4j2.properties + log4j-test.properties + log4j2-test.properties + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + **/META-INF/maven/?*/?*/** + + + + + + + + + Apache Flink + UTF-8 + + + + + + shade-flink + package + + shade + + + false + false + true + ${project.basedir}/target/dependency-reduced-pom.xml + + + + + org.apache.flink:flink-shaded-force-shading + + ** + + + + + io.netty:netty + + META-INF/LICENSE.txt + + + + + + + org.apache.flink:flink-shaded-force-shading + + + + + + + + + + org.commonjava.maven.plugins + directory-maven-plugin + 0.1 + + + directories + + directory-of + + initialize + + rootDir + + org.apache.flink + flink-connectors + + + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.8.0 + + ${target.java.version} + ${target.java.version} + + false + + + -Xpkginfo:always + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.17 + + + com.puppycrawl.tools + checkstyle + + 8.14 + + + + + validate + validate + + check + + + + + /tools/maven/suppressions.xml + true + /tools/maven/checkstyle.xml + true + true + + + + + com.diffplug.spotless + spotless-maven-plugin + ${spotless.version} + + + + 1.7 + + + + + + org.apache.flink,org.apache.flink.shaded,,javax,java,scala,\# + + + + + + + + spotless-check + validate + + check + + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.9.1 + + true + false + + -Xdoclint:none + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 3.0.0-M1 + + + + org.apache.maven.plugins + maven-dependency-plugin + 3.2.0 + + + + org.apache.flink:* + + + + org.apache.flink:force-shading + + com.google.code.findbugs:jsr305 + org.scala-lang:scala-compiler + + org.slf4j:slf4j-api + + log4j:log4j + org.slf4j:slf4j-log4j12 + + org.apache.logging.log4j:log4j-slf4j-impl + org.apache.logging.log4j:log4j-api + org.apache.logging.log4j:log4j-core + org.apache.logging.log4j:log4j-1.2-api + + org.apache.flink:flink-test-utils-junit + junit:junit + org.mockito:mockito-core + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.1 + + + + + org.apache.maven.plugins + maven-site-plugin + + + attach-descriptor + none + + + + + + + com.github.siom79.japicmp + japicmp-maven-plugin + 0.11.0 + + + + org.apache.flink + ${project.artifactId} + ${japicmp.referenceVersion} + ${project.packaging} + + + + + ${project.build.directory}/${project.artifactId}-${project.version}.${project.packaging} + + + + true + + @org.apache.flink.annotation.Public + + + + + @org.apache.flink.annotation.Experimental + @org.apache.flink.annotation.PublicEvolving + @org.apache.flink.annotation.Internal + org.apache.flink.streaming.api.datastream.DataStream#DataStream(org.apache.flink.streaming.api.environment.StreamExecutionEnvironment,org.apache.flink.streaming.api.transformations.StreamTransformation) + org.apache.flink.streaming.api.environment.LegacyLocalStreamEnvironment + org.apache.flink.streaming.api.functions.sink.RichSinkFunction#invoke(java.lang.Object) + org.apache.flink.streaming.api.functions.sink.SinkFunction + org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat + org.apache.flink.api.java.hadoop.mapred.HadoopOutputFormat + org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat + org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat + org.apache.flink.api.scala.hadoop.mapred.HadoopInputFormat + org.apache.flink.api.scala.hadoop.mapred.HadoopOutputFormat + org.apache.flink.api.scala.hadoop.mapreduce.HadoopInputFormat + org.apache.flink.api.scala.hadoop.mapreduce.HadoopOutputFormat + + public + false + true + true + false + true + false + true + + true + + ${rootDir}/${japicmp.outputDir}/${project.artifactId} + + + org.apache.flink + flink-annotations + ${project.version} + + + + + + verify + + cmp + + + + + + + + + \ No newline at end of file