-
Notifications
You must be signed in to change notification settings - Fork 4.2k
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
Changes from 3 commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
42ccfb5
[#25887] fix(JmsIO): issue with multiple connection open #25887
Amraneze 7977cdb
[#25887] fix(JmsIO): create new CommonJms instance instead of extendi…
Amraneze eae4611
[#25887] fix(JmsIO): replacing flag with producer null verification #…
Amraneze feaf32b
[#25887] fix(JmsIO): replace google timestamp with java instant #25887
Amraneze File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
133 changes: 133 additions & 0 deletions
133
sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/CommonJms.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,133 @@ | ||
/* | ||
* 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.beam.sdk.io.jms; | ||
|
||
import java.io.Serializable; | ||
import java.nio.charset.StandardCharsets; | ||
import java.util.ArrayList; | ||
import java.util.List; | ||
import javax.jms.BytesMessage; | ||
import javax.jms.ConnectionFactory; | ||
import javax.jms.Message; | ||
import org.apache.activemq.ActiveMQConnectionFactory; | ||
import org.apache.activemq.broker.BrokerPlugin; | ||
import org.apache.activemq.broker.BrokerService; | ||
import org.apache.activemq.security.AuthenticationUser; | ||
import org.apache.activemq.security.SimpleAuthenticationPlugin; | ||
import org.apache.activemq.store.memory.MemoryPersistenceAdapter; | ||
import org.apache.activemq.transport.TransportFactory; | ||
import org.apache.activemq.transport.amqp.AmqpTransportFactory; | ||
|
||
/** | ||
* A common test fixture to create a broker and connection factories for {@link JmsIOIT} & {@link | ||
* JmsIOTest}. | ||
*/ | ||
public class CommonJms implements Serializable { | ||
Amraneze marked this conversation as resolved.
Show resolved
Hide resolved
|
||
private static final String BROKER_WITHOUT_PREFETCH_PARAM = "?jms.prefetchPolicy.all=0&"; | ||
|
||
static final String USERNAME = "test_user"; | ||
static final String PASSWORD = "test_password"; | ||
static final String QUEUE = "test_queue"; | ||
static final String TOPIC = "test_topic"; | ||
|
||
private final String brokerUrl; | ||
private final Integer brokerPort; | ||
private final String forceAsyncAcksParam; | ||
private transient BrokerService broker; | ||
|
||
protected ConnectionFactory connectionFactory; | ||
protected final Class<? extends ConnectionFactory> connectionFactoryClass; | ||
protected ConnectionFactory connectionFactoryWithSyncAcksAndWithoutPrefetch; | ||
|
||
public CommonJms( | ||
String brokerUrl, | ||
Integer brokerPort, | ||
String forceAsyncAcksParam, | ||
Class<? extends ConnectionFactory> connectionFactoryClass) { | ||
this.brokerUrl = brokerUrl; | ||
this.brokerPort = brokerPort; | ||
this.forceAsyncAcksParam = forceAsyncAcksParam; | ||
this.connectionFactoryClass = connectionFactoryClass; | ||
} | ||
|
||
void startBroker() throws Exception { | ||
broker = new BrokerService(); | ||
broker.setUseJmx(false); | ||
broker.setPersistenceAdapter(new MemoryPersistenceAdapter()); | ||
TransportFactory.registerTransportFactory("amqp", new AmqpTransportFactory()); | ||
if (connectionFactoryClass != ActiveMQConnectionFactory.class) { | ||
broker.addConnector(String.format("%s:%d?transport.transformer=jms", brokerUrl, brokerPort)); | ||
} else { | ||
broker.addConnector(brokerUrl); | ||
} | ||
broker.setBrokerName("localhost"); | ||
broker.setPopulateJMSXUserID(true); | ||
broker.setUseAuthenticatedPrincipalForJMSXUserID(true); | ||
broker.getManagementContext().setCreateConnector(false); | ||
|
||
// enable authentication | ||
List<AuthenticationUser> users = new ArrayList<>(); | ||
// username and password to use to connect to the broker. | ||
// This user has users privilege (able to browse, consume, produce, list destinations) | ||
users.add(new AuthenticationUser(USERNAME, PASSWORD, "users")); | ||
SimpleAuthenticationPlugin plugin = new SimpleAuthenticationPlugin(users); | ||
BrokerPlugin[] plugins = new BrokerPlugin[] {plugin}; | ||
broker.setPlugins(plugins); | ||
|
||
broker.start(); | ||
broker.waitUntilStarted(); | ||
|
||
// create JMS connection factory | ||
connectionFactory = connectionFactoryClass.getConstructor(String.class).newInstance(brokerUrl); | ||
connectionFactoryWithSyncAcksAndWithoutPrefetch = | ||
connectionFactoryClass | ||
.getConstructor(String.class) | ||
.newInstance(brokerUrl + BROKER_WITHOUT_PREFETCH_PARAM + forceAsyncAcksParam); | ||
} | ||
|
||
void stopBroker() throws Exception { | ||
broker.stop(); | ||
broker.waitUntilStopped(); | ||
broker = null; | ||
} | ||
|
||
Class<? extends ConnectionFactory> getConnectionFactoryClass() { | ||
return this.connectionFactoryClass; | ||
} | ||
|
||
ConnectionFactory getConnectionFactory() { | ||
return this.connectionFactory; | ||
} | ||
|
||
ConnectionFactory getConnectionFactoryWithSyncAcksAndWithoutPrefetch() { | ||
return this.connectionFactoryWithSyncAcksAndWithoutPrefetch; | ||
} | ||
|
||
/** A test class that maps a {@link javax.jms.BytesMessage} into a {@link String}. */ | ||
public static class BytesMessageToStringMessageMapper implements JmsIO.MessageMapper<String> { | ||
|
||
@Override | ||
public String mapMessage(Message message) throws Exception { | ||
BytesMessage bytesMessage = (BytesMessage) message; | ||
|
||
byte[] bytes = new byte[(int) bytesMessage.getBodyLength()]; | ||
|
||
return new String(bytes, StandardCharsets.UTF_8); | ||
} | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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 ?
There was a problem hiding this comment.
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.