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

KAFKA-15018: Failing offset flush for EOS when secondary offset store writes fails for tombstone records #13801

Merged
merged 28 commits into from
May 7, 2024

Conversation

vamossagar12
Copy link
Contributor

@vamossagar12 vamossagar12 commented Jun 2, 2023

Problem statement described in ticket

This PR explicitly fails the offset flush for cases when the dual write fails on secondary store. This PR does the same thing as prescribed in the ticket to write first to secondary stores in cases of tombstone records. Also, while originally the flushes didn't care about the outcome of the writes to secondary stores, this PR fails marks the flush as failed for cases when offsets with tombstone records also don't get written successfully to secondary stores. This is because Tombstone events happening are rare events as per ticket. And the probability of them failing on secondary and succeeding on primary is even lesser than that. Considering these things, this PR errs on the side of correctness for these rare events which can lead to inconsistent state. Note that post this PR, offset flus becomes a 3 step process when tombstone offsets are present.

  1. Only tombstone offsets are flushed to the secondary store in a synchronous manner.
  2. Only if 1 succeeds, then all the supplied offsets are flushed to Primary store and the remaining non tombstone offsets are flushed to the secondary store. Note that as earlier, any failures in this latter flush to the secondary store are ignored.

@vamossagar12
Copy link
Contributor Author

I will review the tests once they run. Still tagging the 2 reviewers.

@yashmayya
Copy link
Contributor

@vamossagar12 looks like there are checkstyle failures

log.warn("Failed to write offsets to secondary backing store", secondaryWriteError);
} else {
log.debug("Successfully flushed offsets to secondary backing store");
}
//primaryWriteError is null at this point, and we don't care about secondaryWriteError
callback.onCompletion(null, ignored);
Copy link
Contributor

Choose a reason for hiding this comment

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

Isn't this going to cause us to always block on writes to the secondary store which is something we want to avoid?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah. Good catch. That doesn’t need to happen in this case. Will remove it

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I have removed this. I needed to tweak the callback logic a little but because eventually it needs to be invoked only once so that OffsetStorageWriter#handleFinishWrite doesn't ignore the flush responses. Let me know how the changes are looking now.

@vamossagar12
Copy link
Contributor Author

@vamossagar12 looks like there are checkstyle failures

Oh didn’t check that before taggging. Will check

@C0urante
Copy link
Contributor

C0urante commented Jun 2, 2023

@vamossagar12 Thanks for the PR. I'd like to wait until Yash has had a chance to review before taking a look, so I've removed myself as a reviewer. Feel free to add me back once he (or someone else familiar with Connect) has had a chance to review.

@C0urante C0urante removed their request for review June 2, 2023 13:57
if (secondaryStore != null && containsTombstones && secondaryWriteError != null) {
callback.onCompletion(secondaryWriteError, ignored);
} else {
callback.onCompletion(primaryWriteError, ignored);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

There's one case here where the write to secondary store is successful but fails to primary store. In such a case, there would be discrepancies in the 2 offsets topic but since that would also lead to the failure of offset commit, that discrepancy is still different from the originally reported issue. Let me know if that makes sense.

@vamossagar12 vamossagar12 requested a review from yashmayya June 14, 2023 11:00
@vamossagar12
Copy link
Contributor Author

The build was aborted. But the failed tests seem unrelated as such to this PR.

Copy link
Contributor

@yashmayya yashmayya left a comment

Choose a reason for hiding this comment

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

Thanks @vamossagar12, I've left a few more comments on the non-testing parts; I'll go over the tests in a subsequent pass.

@vamossagar12
Copy link
Contributor Author

Thanks @yashmayya , I addressed your comments and added a response for one of the questions. Let me know if that makes sense. Regarding #13801 (comment), I am still undecided but I feel it's somewhat ok to have it in OffsetStorageWriterTest still since that's the interface that the tasks actually interact with. We can get to this later on.

@vamossagar12
Copy link
Contributor Author

Thanks @yashmayya . I addressed the rest of the comments that you had.

@vamossagar12
Copy link
Contributor Author

vamossagar12 commented Jul 7, 2023

MirrorConnectorsIntegrationBaseTest#testOffsetTranslationBehindReplicationFlow is the only test which is relevant wrt the changes in this PR. I ran it locally a couple of times and it passed on both the occasions. Is it known to be a flaky test?

Copy link
Contributor

@yashmayya yashmayya left a comment

Choose a reason for hiding this comment

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

Thanks @vamossagar12, this is looking much better now!

@vamossagar12
Copy link
Contributor Author

Thanks @vamossagar12, this is looking much better now!

Thanks Yash.. I addressed your comments.

@yashmayya
Copy link
Contributor

Thanks @vamossagar12, I don't have any other comments other than the currently open ones - primarily the timeout case for when exactly-once support is enabled, the ConnectorOffsetBackingStore::set Javadoc, and the unit tests being in OffsetStorageWriterTest versus a dedicated ConnectorOffsetBackingStoreTest class (this one isn't blocking though). @C0urante could you please take a look at this whenever you get a chance?

Copy link
Contributor

@C0urante C0urante left a comment

Choose a reason for hiding this comment

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

Thanks Sagar! I had a feeling the implementation for this would be a little tricky, appreciate you taking a stab at it.

I've left some inline suggestions and have some higher-level thoughts/suggestions as well:

  1. Can we update the description to remove the parts that were copied from the Jira ticket? It's fine to just link to the ticket, and that also makes it easier to distinguish between the information already contained there, and new information contained only in this PR.
  2. Can we skip writes for tombstone records to the secondary store in our second write (the one that takes place after we've written to the primary store)? It's a bit of an optimization, and a bit of a readability improvement: it explicitly calls out the three-step process that we want to engage in when offsets contain tombstones where we first write tombstones to the secondary store, then we write everything to the primary store, and then finally we write non-tombstones to the secondary store.

@vamossagar12
Copy link
Contributor Author

Hey Chris, sorry for the long delay on this. I finally got a chance to verify the code that you provided and it makes sense. I agree that so far I was only thinking about either having 2 separate futures such that one waits for the other or trying to chain futures like CompletableFutures. However, the version you have provided is pretty straight forward and all the new tests passed OOB for me.

Regarding

I think the only question left is whether out-of-order writes are possible because of how things are chained

I am assuming that for non-exactly-once source tasks, you are referring to scenarios when offset flushes are triggered and when flush operations finish out of order. I reviewed the code and I can see that this is being checked in handleFinishWrite which does not complete the flush in case the currently completed flush isn't the current one. For any other erroneous cases, cancelFlush is being invoked ( as you mentioned).

Copy link
Contributor

@C0urante C0urante left a comment

Choose a reason for hiding this comment

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

Thanks Sagar, this is looking pretty good. I've made a brief pass on the testing parts; once this initial stuff is taken care of I can do a deeper dive.

vamossagar12 and others added 4 commits April 10, 2024 16:43
@vamossagar12
Copy link
Contributor Author

vamossagar12 commented Apr 10, 2024

Chris, I started changing the tests in alignment with the comments (i.e using AtomicBoolean, AtomicReference and removing try-catch block). I noticed an interesting issue with testFlushFailureWhenWritesToPrimaryStoreFailsAndSecondarySucceedsForTombstoneRecords test. What's happening is that when we do a get on the future returned in this case, that doesn't throw an exception. I debugged it and I think the problem is because in this case, when the primary store fails, we set the callback to error correctly. However, because the secondary store write doesn't fail, when it's callback gets invoked from here, eventually the producer callback's onCompletion sets it to a non-error from here. The net effect is that the .get() call on the future doesn't return an error which isn't right. Let me know if that makes sense. Meanwhile I would spend some more time on it to see how can it be fixed.

@vamossagar12
Copy link
Contributor Author

vamossagar12 commented Apr 11, 2024

@C0urante , I think I figured out the reason for the above failure with testFlushFailureWhenWritesToPrimaryStoreFailsAndSecondarySucceedsForTombstoneRecords. The problem was that from ConnectOFfsetBackingStore#set we were returning the callback created here but when there is no write error to the secondary store, the exception object is null because of which get doesn't return an error. The primary store write error works fine but that's not the future the caller is waiting on.

I have fixed this by creating a new FutureCallback object and making that as the underlying callback for SetCallbackFuture. The important thing is that it overrides the get() methods (and onCompletion as well) so that the caller of the method, waits on this future and now I can control throwing an exception when the primary store write fails and secondary store passes (or throw nothing if both pass). Let me know what do you think about this.

super.get(timeout, unit);
if (primaryWriteError.get() != null) {
if (primaryWriteError.get() instanceof TimeoutException) {
throw (TimeoutException) primaryWriteError.get();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I am slightly on the fence if we need to handle this case or not, because in ConvertingFutureCallback#result I see that any exception other than CancellationException is wrapped in ExecutionException.

@C0urante
Copy link
Contributor

Thanks Sagar, great catch! I suspected this would be a gnarly one to tackle but it's turning out to be even harder than I thought.

I think there's still an issue with the current state of the PR. It looks like we aren't blocking on the future returned by setPrimaryThenSecondary, which means that we may spuriously return early from get in the future we're returning from ConnectorOffsetBackingStore::set if the write to the primary store hasn't completed yet. I believe this is missed by tests because the producer writes we mock out all take place synchronously; maybe we can use the MockProducer more idiomatically to simulate records being ack'd after calls to MockProducer::send have returned?

I've sketched a new kind of Future implementation that seems to do the trick, though I haven't tested it rigorously:

private class ChainedOffsetWriteFuture implements Future<Void> {

    private final OffsetBackingStore primaryStore;
    private final OffsetBackingStore secondaryStore;
    private final Map<ByteBuffer, ByteBuffer> completeOffsets;
    private final Map<ByteBuffer, ByteBuffer> regularOffsets;
    private final Callback<Void> callback;
    private final AtomicReference<Throwable> writeError;
    private final CountDownLatch completed;

    public ChainedOffsetWriteFuture(
            OffsetBackingStore primaryStore,
            OffsetBackingStore secondaryStore,
            Map<ByteBuffer, ByteBuffer> completeOffsets,
            Map<ByteBuffer, ByteBuffer> regularOffsets,
            Map<ByteBuffer, ByteBuffer> tombstoneOffsets,
            Callback<Void> callback
    ) {
        this.primaryStore = primaryStore;
        this.secondaryStore = secondaryStore;
        this.completeOffsets = completeOffsets;
        this.regularOffsets = regularOffsets;
        this.callback = callback;
        this.writeError = new AtomicReference<>();
        this.completed = new CountDownLatch(1);

        secondaryStore.set(tombstoneOffsets, this::onFirstWrite);
    }

    private void onFirstWrite(Throwable error, Void ignored) {
        if (error != null) {
            log.trace("Skipping offsets write to primary store because secondary tombstone write has failed", error);
            try (LoggingContext context = loggingContext()) {
                callback.onCompletion(error, ignored);
                writeError.compareAndSet(null, error);
                completed.countDown();
            }
            return;
        }
        setPrimaryThenSecondary(primaryStore, secondaryStore, completeOffsets, regularOffsets, this::onSecondWrite);
    }

    private void onSecondWrite(Throwable error, Void ignored) {
        callback.onCompletion(error, ignored);
        writeError.compareAndSet(null, error);
        completed.countDown();
    }

    @Override
    public boolean cancel(boolean mayInterruptIfRunning) {
        return false;
    }

    @Override
    public boolean isCancelled() {
        return false;
    }

    @Override
    public boolean isDone() {
        return completed.getCount() == 0;
    }

    @Override
    public Void get() throws InterruptedException, ExecutionException {
        completed.await();
        if (writeError.get() != null) {
            throw new ExecutionException(writeError.get());
        }
        return null;
    }

    @Override
    public Void get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
        if (!completed.await(timeout, unit)) {
            throw new TimeoutException("Failed to complete offset write in time");
        }
        if (writeError.get() != null) {
            throw new ExecutionException(writeError.get());
        }
        return null;
    }
}

(I've omitted an implementation of cancel and isCancelled for now since I'm not sure it's really necessary, but LMK if I've missed a case where this would make a difference.)

The new class can be used at the end of ConnectorOffsetBackingStore::set like this:

if (secondaryStore != null && !tombstoneOffsets.isEmpty()) {
    return new ChainedOffsetWriteFuture(
            primaryStore,
            secondaryStore,
            values,
            regularOffsets,
            tombstoneOffsets,
            callback
    );
} else {
    return setPrimaryThenSecondary(primaryStore, secondaryStore, values, regularOffsets, callback);
}

@vamossagar12
Copy link
Contributor Author

Thanks Chris! I ran through the scenarios in the test and I can see that it handles the cases correctly. Regarding, cancel I don't see the future returned from set being cancelled explicitly so we can live w/o implementations of cancel and isCancelled.
Also, I have now updated the tests so that I control when should a record be returned, throw and error or a timeout. MockProducer provided some great hooks to do the same. I added a couple of more tests which even test the timeout scenario and the tests throw a timeout until all futures return promptly (error or o/w). Let me know how this is looking now. Thanks!

Copy link
Contributor

@C0urante C0urante left a comment

Choose a reason for hiding this comment

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

Thanks Sagar, this is looking great.

The one thing left I'd like to see is testing coverage for three-step writes. Right now we have great coverage for two-step writes (secondary tombstones then primary, primary then secondary non-tombstones, etc.), but it looks like we don't have any coverage for a mix of tombstone and non-tombstone offsets. Would it be possible to add cases for these kinds of scenarios?

@vamossagar12
Copy link
Contributor Author

Thanks @C0urante , good catch, yeah those are missing. I have modified some of the tests to consider all the 3 types of offset records. I added another test for the case when write to secondary store times out for regular offsets. Let me know if these look ok coverage-wise.

Copy link
Contributor

@C0urante C0urante left a comment

Choose a reason for hiding this comment

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

LGTM, thanks @vamossagar12!

@C0urante C0urante merged commit 525b9b1 into apache:trunk May 7, 2024
1 check failed
gongxuanzhang pushed a commit to gongxuanzhang/kafka that referenced this pull request Jun 12, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants