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
Add epoch for connection handler to handle create producer timeout. #5571
Changes from 1 commit
08fd3ed
edae5d1
760b5b3
38ebf2a
56764ce
7fc26ea
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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.pulsar.client.api; | ||
|
||
import org.apache.pulsar.client.impl.ProducerImpl; | ||
import org.testng.Assert; | ||
import org.testng.annotations.AfterMethod; | ||
import org.testng.annotations.BeforeMethod; | ||
import org.testng.annotations.Test; | ||
|
||
public class ProducerCreationTest extends ProducerConsumerBase { | ||
|
||
@BeforeMethod | ||
@Override | ||
protected void setup() throws Exception { | ||
super.internalSetup(); | ||
super.producerBaseSetup(); | ||
} | ||
|
||
@AfterMethod | ||
@Override | ||
protected void cleanup() throws Exception { | ||
super.internalCleanup(); | ||
} | ||
|
||
@Test | ||
public void testExactlyOnceWithProducerNameSpecified() throws PulsarClientException { | ||
Producer<byte[]> producer1 = pulsarClient.newProducer() | ||
.topic("testExactlyOnceWithProducerNameSpecified") | ||
.producerName("p-name-1") | ||
.create(); | ||
|
||
Assert.assertNotNull(producer1); | ||
|
||
Producer<byte[]> producer2 = pulsarClient.newProducer() | ||
.topic("testExactlyOnceWithProducerNameSpecified") | ||
.producerName("p-name-2") | ||
.create(); | ||
|
||
Assert.assertNotNull(producer2); | ||
|
||
try { | ||
pulsarClient.newProducer() | ||
.topic("testExactlyOnceWithProducerNameSpecified") | ||
.producerName("p-name-2") | ||
.create(); | ||
Assert.fail("should be failed"); | ||
} catch (PulsarClientException.ProducerBusyException e) { | ||
//ok here | ||
} | ||
} | ||
|
||
@Test | ||
public void testGeneratedNameProducerReconnect() throws PulsarClientException, InterruptedException { | ||
ProducerImpl<byte[]> producer = (ProducerImpl<byte[]>) pulsarClient.newProducer() | ||
.topic("testGeneratedNameProducerReconnect") | ||
.create(); | ||
Assert.assertTrue(producer.isConnected()); | ||
//simulate create producer timeout. | ||
Thread.sleep(3000); | ||
|
||
producer.getConnectionHandler().connectionClosed(producer.getConnectionHandler().getClientCnx()); | ||
Assert.assertFalse(producer.isConnected()); | ||
Thread.sleep(3000); | ||
Assert.assertEquals(producer.getConnectionHandler().getEpoch(), 1); | ||
Assert.assertTrue(producer.isConnected()); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -38,6 +38,11 @@ public interface Producer<T> extends Closeable { | |
*/ | ||
String getProducerName(); | ||
|
||
/** | ||
* @return the name of producer is generated or user specified | ||
*/ | ||
boolean isGeneratedName(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do we need to expose this to producer api? I think this is an implementation detail, which should be hidden in the implementation. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ok, currently no need to expose to the interface |
||
|
||
/** | ||
* Sends a message. | ||
* | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,19 +20,22 @@ | |
|
||
import java.util.concurrent.TimeUnit; | ||
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; | ||
|
||
import com.google.common.annotations.VisibleForTesting; | ||
import org.apache.pulsar.client.api.PulsarClientException; | ||
import org.apache.pulsar.client.impl.HandlerState.State; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
class ConnectionHandler { | ||
public class ConnectionHandler { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: add There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ok, let me try |
||
private static final AtomicReferenceFieldUpdater<ConnectionHandler, ClientCnx> CLIENT_CNX_UPDATER = | ||
AtomicReferenceFieldUpdater.newUpdater(ConnectionHandler.class, ClientCnx.class, "clientCnx"); | ||
@SuppressWarnings("unused") | ||
private volatile ClientCnx clientCnx = null; | ||
|
||
protected final HandlerState state; | ||
protected final Backoff backoff; | ||
protected long epoch = 0L; | ||
|
||
interface Connection { | ||
void connectionFailed(PulsarClientException exception); | ||
|
@@ -94,11 +97,13 @@ protected void reconnectLater(Throwable exception) { | |
state.setState(State.Connecting); | ||
state.client.timer().newTimeout(timeout -> { | ||
log.info("[{}] [{}] Reconnecting after connection was closed", state.topic, state.getHandlerName()); | ||
++epoch; | ||
grabCnx(); | ||
}, delayMs, TimeUnit.MILLISECONDS); | ||
} | ||
|
||
protected void connectionClosed(ClientCnx cnx) { | ||
@VisibleForTesting | ||
public void connectionClosed(ClientCnx cnx) { | ||
if (CLIENT_CNX_UPDATER.compareAndSet(this, cnx, null)) { | ||
if (!isValidStateForReconnection()) { | ||
log.info("[{}] [{}] Ignoring reconnection request (state: {})", state.topic, state.getHandlerName(), state.getState()); | ||
|
@@ -110,6 +115,7 @@ protected void connectionClosed(ClientCnx cnx) { | |
delayMs / 1000.0); | ||
state.client.timer().newTimeout(timeout -> { | ||
log.info("[{}] [{}] Reconnecting after timeout", state.topic, state.getHandlerName()); | ||
++epoch; | ||
grabCnx(); | ||
}, delayMs, TimeUnit.MILLISECONDS); | ||
} | ||
|
@@ -127,7 +133,8 @@ protected boolean isRetriableError(PulsarClientException e) { | |
return e instanceof PulsarClientException.LookupException; | ||
} | ||
|
||
protected ClientCnx getClientCnx() { | ||
@VisibleForTesting | ||
public ClientCnx getClientCnx() { | ||
return CLIENT_CNX_UPDATER.get(this); | ||
} | ||
|
||
|
@@ -153,5 +160,10 @@ private boolean isValidStateForReconnection() { | |
return false; | ||
} | ||
|
||
@VisibleForTesting | ||
public long getEpoch() { | ||
return epoch; | ||
} | ||
|
||
private static final Logger log = LoggerFactory.getLogger(ConnectionHandler.class); | ||
} |
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.
Can we avoid iterating over the producers set? I.e. can you change the hash set to a hash map?