fapaul commented on a change in pull request #16333:
URL: https://github.com/apache/flink/pull/16333#discussion_r662325588
##########
File path:
flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java
##########
@@ -157,6 +157,28 @@ public void throwExceptionIfConnectionFactoryReturnNull()
throws Exception {
}
}
+ @Test
+ public void testResourceCleanupOnOpenFailure() throws Exception {
+ RMQConnectionConfig connectionConfig =
Mockito.mock(RMQConnectionConfig.class);
+ ConnectionFactory connectionFactory =
Mockito.mock(ConnectionFactory.class);
+ Connection connection = Mockito.mock(Connection.class);
+
Mockito.when(connectionConfig.getConnectionFactory()).thenReturn(connectionFactory);
+ Mockito.when(connectionConfig.getHost()).thenReturn("hostDummy");
+ Mockito.when(connectionFactory.newConnection()).thenReturn(connection);
+ Mockito.when(connection.createChannel()).thenThrow(new IOException());
+
+ RMQSource<String> rmqSource =
+ new RMQSource<>(
+ connectionConfig, "queueDummy", true, new
StringDeserializationScheme());
+ try {
+ rmqSource.open(new Configuration());
+ } catch (RuntimeException ex) {
+ assertEquals(
+ "Cannot create RMQ connection with queueDummy at
hostDummy", ex.getMessage());
+ }
Review comment:
I think it is a bit cleaner to use `Assert.assertThrows`
##########
File path:
flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java
##########
@@ -177,6 +199,46 @@ public void testOpenCallDeclaresQueueInStandardMode()
throws Exception {
Mockito.verify(channel).queueDeclare(RMQTestSource.QUEUE_NAME, true,
false, false, null);
}
+ @Test
+ public void testResourceCleanupOnClose() throws Exception {
+ FunctionInitializationContext mockContext = getMockContext();
+
+ RMQConnectionConfig connectionConfig =
Mockito.mock(RMQConnectionConfig.class);
+ ConnectionFactory connectionFactory =
Mockito.mock(ConnectionFactory.class);
+ Connection connection = Mockito.mock(Connection.class);
+ Channel channel = Mockito.mock(Channel.class);
+
+
Mockito.when(connectionConfig.getConnectionFactory()).thenReturn(connectionFactory);
+ Mockito.when(connectionFactory.newConnection()).thenReturn(connection);
+ Mockito.when(connectionConfig.getHost()).thenReturn("hostDummy");
+ Mockito.when(connection.createChannel()).thenReturn(channel);
+ Mockito.doThrow(new IOException("Consumer cancel
error")).when(channel).basicCancel(any());
+ Mockito.doThrow(new IOException("Channel
error")).when(channel).close();
+ Mockito.doThrow(new IOException("Connection
error")).when(connection).close();
+
+ RMQSource<String> rmqSource = new
RMQMockedRuntimeTestSource(connectionConfig);
+ rmqSource.initializeState(mockContext);
+ rmqSource.open(new Configuration());
+
+ try {
+ rmqSource.close();
+ } catch (RuntimeException ex) {
+ assertEquals(
+ "Error while cancelling RMQ consumer on queueDummy at
hostDummy",
+ ex.getMessage());
+ assertEquals(2, ex.getSuppressed().length);
+ assertEquals(
+ "Error while closing RMQ channel with queueDummy at
hostDummy",
+ ex.getSuppressed()[0].getMessage());
+ assertEquals(
+ "Error while closing RMQ connection with queueDummy at
hostDummy",
+ ex.getSuppressed()[1].getMessage());
+ }
Review comment:
Same here, you can use `Assert.assertThrows`
##########
File path:
flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
##########
@@ -273,44 +276,56 @@ public void open(Configuration config) throws Exception {
@Override
public void close() throws Exception {
super.close();
+ Exception exception = null;
try {
if (consumer != null && channel != null) {
channel.basicCancel(consumer.getConsumerTag());
}
} catch (IOException e) {
- throw new RuntimeException(
- "Error while cancelling RMQ consumer on "
- + queueName
- + " at "
- + rmqConnectionConfig.getHost(),
- e);
+ exception =
+ new RuntimeException(
+ "Error while cancelling RMQ consumer on "
+ + queueName
+ + " at "
+ + rmqConnectionConfig.getHost(),
+ e);
}
try {
if (channel != null) {
channel.close();
}
} catch (IOException e) {
- throw new RuntimeException(
- "Error while closing RMQ channel with "
- + queueName
- + " at "
- + rmqConnectionConfig.getHost(),
- e);
+ exception =
+ ExceptionUtils.firstOrSuppressed(
+ new RuntimeException(
+ "Error while closing RMQ channel with "
+ + queueName
+ + " at "
+ + rmqConnectionConfig.getHost(),
+ e),
Review comment:
Nit: For the `channel` and `connection` you can use `IOUtils.closeAll()`
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]