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

Fix Jms drop record #30218

Merged
merged 10 commits into from Feb 7, 2024
Expand Up @@ -19,11 +19,12 @@

import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import javax.jms.JMSException;
import javax.jms.Message;
import javax.jms.MessageConsumer;
import javax.jms.Session;
import org.apache.beam.sdk.io.UnboundedSource;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.checkerframework.checker.nullness.qual.Nullable;
Expand All @@ -39,87 +40,67 @@ class JmsCheckpointMark implements UnboundedSource.CheckpointMark, Serializable

private static final Logger LOG = LoggerFactory.getLogger(JmsCheckpointMark.class);

private Instant oldestMessageTimestamp = Instant.now();
private transient List<Message> messages = new ArrayList<>();
private Instant oldestMessageTimestamp;
private transient @Nullable Message lastMessage;
private transient @Nullable MessageConsumer consumer;
private transient @Nullable Session session;

@VisibleForTesting transient boolean discarded = false;

@VisibleForTesting final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();

JmsCheckpointMark() {}
private JmsCheckpointMark(
Instant oldestMessageTimestamp,
@Nullable Message lastMessage,
@Nullable MessageConsumer consumer,
@Nullable Session session) {
this.oldestMessageTimestamp = oldestMessageTimestamp;
this.lastMessage = lastMessage;
this.consumer = consumer;
this.session = session;
}

void add(Message message) throws Exception {
lock.writeLock().lock();
/** Acknowledge all outstanding message. */
@Override
public void finalizeCheckpoint() {
try {
if (discarded) {
throw new IllegalStateException(
String.format(
"Attempting to add message %s to checkpoint that is discarded.", message));
}
Instant currentMessageTimestamp = new Instant(message.getJMSTimestamp());
if (currentMessageTimestamp.isBefore(oldestMessageTimestamp)) {
oldestMessageTimestamp = currentMessageTimestamp;
// Jms spec will implicitly acknowledge _all_ messaged already received by the same
// session if one message in this session is being acknowledged.
if (lastMessage != null) {
lastMessage.acknowledge();
}
messages.add(message);
} finally {
lock.writeLock().unlock();
} catch (JMSException e) {
// The effect of this is message not get acknowledged and thus will be redelivered. It is
// not fatal, so we just raise error log. Similar below.
LOG.error(
"Failed to acknowledge the message. Will redeliver and might cause duplication.", e);
}
}

Instant getOldestMessageTimestamp() {
lock.readLock().lock();
try {
return this.oldestMessageTimestamp;
} finally {
lock.readLock().unlock();
// session is closed after message acknowledged otherwise other consumer may receive duplicate
// messages.
if (consumer != null) {
try {
consumer.close();
consumer = null;
} catch (JMSException e) {
LOG.info("Error closing JMS consumer. It may have already been closed.");
Copy link
Collaborator

Choose a reason for hiding this comment

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

infor or debug? Not sure the rule. :)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

For the latter two, as long as we successfully acknowledged the messages, there won't be duplicates for the messages in this checkpoint. So error in closing consumer or session is less harm, so I put it as info level here.

}
}
}

void discard() {
lock.writeLock().lock();
try {
this.discarded = true;
} finally {
lock.writeLock().unlock();
}
}

/**
* Acknowledge all outstanding message. Since we believe that messages will be delivered in
* timestamp order, and acknowledged messages will not be retried, the newest message in this
* batch is a good bound for future messages.
*/
@Override
public void finalizeCheckpoint() {
lock.writeLock().lock();
try {
if (discarded) {
messages.clear();
return;
// session needs to be closed after message acknowledged because the latter needs session remain
// active.
if (session != null) {
try {
session.close();
session = null;
} catch (JMSException e) {
LOG.info("Error closing JMS session. It may have already been closed.");
}
for (Message message : messages) {
try {
message.acknowledge();
Instant currentMessageTimestamp = new Instant(message.getJMSTimestamp());
if (currentMessageTimestamp.isAfter(oldestMessageTimestamp)) {
oldestMessageTimestamp = currentMessageTimestamp;
}
} catch (Exception e) {
LOG.error("Exception while finalizing message: ", e);
}
}
messages.clear();
} finally {
lock.writeLock().unlock();
}
}

// set an empty list to messages when deserialize
Copy link
Collaborator

Choose a reason for hiding this comment

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

update the comment?

private void readObject(java.io.ObjectInputStream stream)
throws IOException, ClassNotFoundException {
stream.defaultReadObject();
messages = new ArrayList<>();
discarded = false;
lastMessage = null;
session = null;
}

@Override
Expand All @@ -138,4 +119,90 @@ public boolean equals(@Nullable Object o) {
public int hashCode() {
return Objects.hash(oldestMessageTimestamp);
}

static Preparer newPreparer() {
return new Preparer();
}

/**
* A class preparing the immutable checkpoint. It is mutable so that new messages can be added.
*/
static class Preparer {
private Instant oldestMessageTimestamp = Instant.now();
private transient @Nullable Message lastMessage = null;

@VisibleForTesting transient boolean discarded = false;

@VisibleForTesting final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();

private Preparer() {}

void add(Message message) throws Exception {
lock.writeLock().lock();
try {
if (discarded) {
throw new IllegalStateException(
String.format(
"Attempting to add message %s to checkpoint that is discarded.", message));
}
Instant currentMessageTimestamp = new Instant(message.getJMSTimestamp());
if (currentMessageTimestamp.isBefore(oldestMessageTimestamp)) {
oldestMessageTimestamp = currentMessageTimestamp;
}
lastMessage = message;
} finally {
lock.writeLock().unlock();
}
}

Instant getOldestMessageTimestamp() {
lock.readLock().lock();
try {
return this.oldestMessageTimestamp;
} finally {
lock.readLock().unlock();
}
}

void discard() {
lock.writeLock().lock();
try {
this.discarded = true;
} finally {
lock.writeLock().unlock();
}
}

/**
* Create a new checkpoint mark based on the current preparer. This will reset the messages held
* by the preparer, and the owner of the preparer is responsible to create a new Jms session
* after this call.
*/
JmsCheckpointMark newCheckpoint(@Nullable MessageConsumer consumer, @Nullable Session session) {
JmsCheckpointMark checkpointMark;
lock.writeLock().lock();
try {
if (discarded) {
lastMessage = null;
checkpointMark = this.emptyCheckpoint();
} else {
checkpointMark =
new JmsCheckpointMark(oldestMessageTimestamp, lastMessage, consumer, session);
lastMessage = null;
oldestMessageTimestamp = Instant.now();
}
} finally {
lock.writeLock().unlock();
}
return checkpointMark;
}

JmsCheckpointMark emptyCheckpoint() {
return new JmsCheckpointMark(oldestMessageTimestamp, null, null, null);
}

boolean isEmpty() {
return lastMessage == null;
}
}
}