Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[#25887] fix(JmsIO): issue with multiple connection open #25887 #25945

Merged
merged 4 commits into from
Apr 5, 2023
Merged

[#25887] fix(JmsIO): issue with multiple connection open #25887 #25945

merged 4 commits into from
Apr 5, 2023

Conversation

Amraneze
Copy link
Contributor

@Amraneze Amraneze commented Mar 23, 2023

  • Issue related to multiple connection being open for each bundle
  • Add integration test using jms-qpid for JmsIO

Fixes #25887

Related to PR#25886

There are two points handled in this PR:

  • The issue that we encountered that the pipeline open 6k ports for each VM. With 18 workers, we had 108k ports opened. After checking the issue, we found out that creating a connection in startBundle is not a good approach. Especially that the client that we use qpid-jms-client open a connection per CPU. Having 6 CPUs, the JMS client will open 6 connections. The solution that we tested, was to create a connection only in setup cycle and recreate it, if it failed, during the process element cycle. The open connections are not a leaked connections, because we can see the logs of the connections being closed an opened at the start of each bundle.
  • We added integration tests for the client qpid-jms-client. We want to have multiple tests for each Jms client.

Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Mention the appropriate issue in your description (for example: addresses #123), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment fixes #<ISSUE NUMBER> instead.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI.

@Amraneze
Copy link
Contributor Author

@Abacn I pushed this PR to fix the issue and more integration test. We are working on having a pool connection to determine the connection that we need. Something like this.

 private static class JmsConnectionPool<T> implements Serializable {

  private static final long serialVersionUID = 1L;
  private static final int DEFAULT_MAX_POOL_SIZE = 10;
  private static final int DEFAULT_INITIAL_POOL_SIZE = 20;

  private JmsIO.Write<T> spec;
  private final int maxPoolSize;
  private final int initialPoolSize;
  private List<JmsConnection<T>> jmsConnectionPool;
  private List<JmsConnection<T>> usedJmsConnections = new ArrayList<>();

  JmsConnectionPool(JmsIO.Write<T> spec, List<JmsConnection<T>> jmsConnectionPool) {
    this.spec = spec;
    this.jmsConnectionPool = jmsConnectionPool;
    this.maxPoolSize = Optional.ofNullable(spec.getMaxPoolSize()).orElse(DEFAULT_MAX_POOL_SIZE);
    this.initialPoolSize = Optional.ofNullable(spec.getInitialPoolSize()).orElse(DEFAULT_INITIAL_POOL_SIZE);
  }

  static <T> JmsConnectionPool<T> create(JmsIO.Write<T> spec) {
    int initialPoolSize = Optional.ofNullable(spec.getInitialPoolSize()).orElse(DEFAULT_INITIAL_POOL_SIZE);
    List<JmsConnection<T>> jmsConnectionPool = new ArrayList<>(initialPoolSize);
    for (int i = 0; i < initialPoolSize; i++) {
      jmsConnectionPool.add(new JmsConnection<>(spec));
    }
    return new JmsConnectionPool<>(spec, jmsConnectionPool);
  }

  JmsConnection<T> getConnection() throws JmsIOException {
    if (jmsConnectionPool.isEmpty()) {
      if (usedJmsConnections.size() < maxPoolSize) {
        jmsConnectionPool.add(new JmsConnection<>(spec));
      } else {
        throw new JmsIOException("Maximum pool connection size has been reached");
      }
    }

    JmsConnection<T> jmsConnection = jmsConnectionPool
            .remove(jmsConnectionPool.size() - 1);

    usedJmsConnections.add(jmsConnection);
    return jmsConnection;
  }

  public boolean releaseConnection(JmsConnection<T> jmsConnection) {
    jmsConnectionPool.add(jmsConnection);
    return usedJmsConnections.remove(jmsConnection);
  }

  public boolean closeConnection(JmsConnection<T> jmsConnection) {
    jmsConnection.close();
    jmsConnectionPool.remove(jmsConnection);
    return usedJmsConnections.remove(jmsConnection);
  }

  public void shutdown() throws JMSException {
    usedJmsConnections.forEach(this::releaseConnection);
    for (JmsConnection<T> jmsConnection : jmsConnectionPool) {
      jmsConnection.close();
    }
    jmsConnectionPool.clear();
  }
}

The function closeConnection will be called inside of this.connection.setExceptionListener. What do you think about it ? We want also to create latency or issues with other projects using JmsIO

@codecov
Copy link

codecov bot commented Mar 23, 2023

Codecov Report

Merging #25945 (feaf32b) into master (7ee74d2) will decrease coverage by 0.06%.
The diff coverage is n/a.

@@            Coverage Diff             @@
##           master   #25945      +/-   ##
==========================================
- Coverage   71.42%   71.37%   -0.06%     
==========================================
  Files         782      783       +1     
  Lines      102856   102946      +90     
==========================================
+ Hits        73470    73473       +3     
- Misses      27910    27997      +87     
  Partials     1476     1476              
Flag Coverage Δ
python 79.86% <ø> (-0.10%) ⬇️

Flags with carried forward coverage won't be shown. Click here to find out more.

see 12 files with indirect coverage changes

📣 We’re building smart automated test selection to slash your CI/CD build times. Learn more

@Amraneze
Copy link
Contributor Author

@Abacn I had an issue with source compilation. The client that I used qpid-jms-client is compiled using JDK 11 but the tests are running using JDK 8. I downgraded the version to 0.61.0 compiled using JDK 8 in the commit 73bb7f8.

@Abacn
Copy link
Contributor

Abacn commented Mar 23, 2023

The forward fix is heavy. To avoid introducing new risk is it possible to have a lightweight fix at first? For example, Does only close the producer and session at finishBundle (but keep session and connection open) mitigate the pressure to the server? Per documentation "A connection could represent an open TCP/IP socket between a client and a provider service daemon." So the port use should be per connection base.

Connection Pool is a feature request and can be considered later.

@Abacn
Copy link
Contributor

Abacn commented Mar 23, 2023

Also, integration test goes to JmsIOIT and has its own gradle task. The guideline of Beam website may help with design and implement integration tests: https://beam.apache.org/documentation/io/io-standards/#integration-tests

@Amraneze
Copy link
Contributor Author

Amraneze commented Mar 23, 2023

The forward fix is heavy. To avoid introducing new risk is it possible to have a lightweight fix at first? For example, Does only close the producer and session at finishBundle (but keep session and connection open) mitigate the pressure to the server? Per documentation "A connection could represent an open TCP/IP socket between a client and a provider service daemon." So the port use should be per connection base.

And about the creating of connection, should we keep it at setup or startBundle ?

Connection Pool is a feature request and can be considered later.

Yes, it would be nice to have.

@github-actions
Copy link
Contributor

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @kennknowles for label java.
R: @damccorm for label build.
R: @ahmedabu98 for label io.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

@Abacn
Copy link
Contributor

Abacn commented Mar 23, 2023

And about the creating of connection, should we keep it at setup or startBundle ?

At setup and destroy in tearDown, as the original implementation. Then only one port will be used for one worker thread.

@github-actions
Copy link
Contributor

Reminder, please take a look at this pr: @kennknowles @damccorm @ahmedabu98

- Issue related to multiple connection being open for each bundle
- Add integration test using jms-qpid for JmsIO

Fixes #25887

Co-Authored-By: Amrane Ait Zeouay <[email protected]>
Copy link
Contributor

@Abacn Abacn left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks. the changes on main scope looks good to me. For the test left a couple of comments. If we want the integration test running on ci, we'll need to add the task here:

jms: [
    ':sdks:java:io:jms:integrationTest',
  ],

@github-actions github-actions bot added infra and removed infra labels Apr 4, 2023
@github-actions github-actions bot added infra and removed infra labels Apr 4, 2023
Copy link
Contributor

@Abacn Abacn left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, just have a few comments about clean up. I think we can still get it in by the 2.47.0 cut which is tomorrow. Have you tested in your environement that this PR resolves the port occupying issue?

Verified integration test locally:

Test | Duration | Result
-- | -- | --
testPublishingThenReadingAll[with client class class org.apache.activemq.ActiveMQConnectionFactory] | 33.851s | passed
testPublishingThenReadingAll[with client class class org.apache.qpid.jms.JmsConnectionFactory] | 34.059s | passed

@@ -1026,7 +1027,7 @@ public void start() throws JMSException {
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

may replace the direct assignment of producer with startProducer() thus make it clear that producer is opened in single code path.

Also, I see " isProducerNeedsToBeCreated" is removed in several places and connect() is only called in DoFn's setup. Can we get rid of this flag now?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We added the flag in case of a failed connection it will create a new connection based on exceptionListener. Do you think it would be better to check with the producer if it's null ?

if (producer == null) {
// open connection
connection.setExceptionListener(exception -> {
    connectionCounter.inc();
    // if there is an issue with the connection, we will close session, connection & producer so it can be recreated it
    close();
});
// create new producer
}

Copy link
Contributor

@Abacn Abacn Apr 4, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Both is fine, if the flag is used then it should be consistent. For now if closeProducer() is called, producer is closed and set to null, but isProducerNeedsToBeCreated is still false.


void close() {
try {
if (producer != null) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

may call closeProducer() here. Making producer is opened / closed by single code path

@Amraneze
Copy link
Contributor Author

Amraneze commented Apr 4, 2023

Thanks, just have a few comments about clean up. I think we can still get it in by the 2.47.0 cut which is tomorrow. Have you tested in your environement that this PR resolves the port occupying issue?

When the version 2.47.0 will be published? So I can know if I will have time to test in my environment. For the issue with new connection it was fixed.

@Abacn
Copy link
Contributor

Abacn commented Apr 4, 2023

Thanks, just have a few comments about clean up. I think we can still get it in by the 2.47.0 cut which is tomorrow. Have you tested in your environement that this PR resolves the port occupying issue?

When the version 2.47.0 will be published? So I can know if I will have time to test in my environment. For the issue with new connection it was fixed.

Release cut is tomorrow, Apr 5th. Release date targetted to early May.

@Amraneze
Copy link
Contributor Author

Amraneze commented Apr 4, 2023

Thanks, just have a few comments about clean up. I think we can still get it in by the 2.47.0 cut which is tomorrow. Have you tested in your environement that this PR resolves the port occupying issue?

When the version 2.47.0 will be published? So I can know if I will have time to test in my environment. For the issue with new connection it was fixed.

Release cut is tomorrow, Apr 5th. Release date targetted to early May.

I'm not really confident about it. I didn't test the republish functionality which is a major thing for us. The problem that I can see is that in case of a failed connection, the DoFn thread will retry the bundle for X times during a duration Y. But, there is no recreating the connection again. I can suggest two possible ways to do it:

1 - inside of the exception listener

connection.setExceptionListener(exception -> {
  failedConnectionCounter.inc();
  // Make sure to free the failed connection 
  close();
  // Recreate a new connection with a new producer
  connect();
});

2 - inside of the loop

connection.setExceptionListener(exception -> {
  failedConnectionCounter.inc();
  // Make sure to free the failed connection 
  close();
});
...
void connect () {
   if (producer == null) {
   ....
   }
}
...

void publishMessage() {
 while(true) {
  //Recreate a connection if the producer is null
  connect();
  ...
 }
}

What do you think about this?

@Abacn
Copy link
Contributor

Abacn commented Apr 4, 2023

Generally, the backOff retry intermittent error (that a retry in place can success).

If your concern is the persistent error (a dead connection) it should be already handled by runner. That is, when the exception is popping up, the runner will create a new work item, which includes initializing a new DoFn instance, to retry the bundle.

Handling all errors within DoFn, theoretically, it works, but not encouraged because it adds complexity to the implementation.

If republish need more test, can we aim to fix the original issue (high port occupacy) that was a regression in 2.46.0 at the moment?

@Abacn
Copy link
Contributor

Abacn commented Apr 4, 2023

For the comment #25945 (comment)

1 - inside of the exception listener

is there a potential racing condition to operate connection inside connection.exceptionListener?

2 - inside of the loop

This looks good at first glance. Though I still suggest to improvement one thing at a time, to get at lease a stable version first.

Also, if we call close and connect inside @ProcessElement (other than @setup and @teardown), they need to be marked as synchronized methods

@github-actions github-actions bot added infra and removed infra labels Apr 5, 2023
@Amraneze
Copy link
Contributor Author

Amraneze commented Apr 5, 2023

For the comment #25945 (comment)

1 - inside of the exception listener

is there a potential racing condition to operate connection inside connection.exceptionListener?

2 - inside of the loop

This looks good at first glance. Though I still suggest to improvement one thing at a time, to get at lease a stable version first.

Also, if we call close and connect inside @ProcessElement (other than @setup and @teardown), they need to be marked as synchronized methods

I will try to see which solution have better performance. Should we add it with PoolConnection ?

Just FYI, when there is an exception while publishing, the bundle is not republished because the message is sent to an output. That was the previous implementation and I wanted to keep backward compatibility. So if we remove that, we can just throw an exception and let Dataflow to retry the bundle without the retry policy.

Here is a graph that explains the logic of JmsIO.
Brainstorming

@github-actions github-actions bot added infra and removed infra labels Apr 5, 2023
@Abacn
Copy link
Contributor

Abacn commented Apr 5, 2023

test passed though not reflected on GitHub UI. Merging for now

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Task]: Add integration test for JmsIO
2 participants