Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,151 @@
/*
*
* 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.bookkeeper.proto;

import io.netty.util.Recycler;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.common.util.MathUtils;
import org.apache.bookkeeper.net.BookieId;
import org.slf4j.MDC;

class AddCompletion extends CompletionValue implements BookkeeperInternalCallbacks.WriteCallback {
Copy link
Contributor

Choose a reason for hiding this comment

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

Refactoring by pulling the classes out masks the actual fix here. Can you, for this fix, leave the refactoring aside?

Copy link
Contributor

Choose a reason for hiding this comment

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

Actually, after reviewing a bit, it kind of makes sense to split the classes in this PR, since it forces all classes to be static.


static AddCompletion acquireAddCompletion(final CompletionKey key,
final BookkeeperInternalCallbacks.WriteCallback originalCallback,
final Object originalCtx,
final long ledgerId, final long entryId,
PerChannelBookieClient perChannelBookieClient) {
AddCompletion completion = ADD_COMPLETION_RECYCLER.get();
completion.reset(key, originalCallback, originalCtx, ledgerId, entryId, perChannelBookieClient);
return completion;
}

final Recycler.Handle<AddCompletion> handle;

CompletionKey key = null;
BookkeeperInternalCallbacks.WriteCallback originalCallback = null;

AddCompletion(Recycler.Handle<AddCompletion> handle) {
super("Add", null, -1, -1, null);
this.handle = handle;
}

void reset(final CompletionKey key,
final BookkeeperInternalCallbacks.WriteCallback originalCallback,
final Object originalCtx,
final long ledgerId, final long entryId,
PerChannelBookieClient perChannelBookieClient) {
this.key = key;
this.originalCallback = originalCallback;
this.ctx = originalCtx;
this.ledgerId = ledgerId;
this.entryId = entryId;
this.startTime = org.apache.bookkeeper.common.util.MathUtils.nowInNano();

this.opLogger = perChannelBookieClient.addEntryOpLogger;
this.timeoutOpLogger = perChannelBookieClient.addTimeoutOpLogger;
this.perChannelBookieClient = perChannelBookieClient;
this.mdcContextMap = perChannelBookieClient.preserveMdcForTaskExecution ? MDC.getCopyOfContextMap() : null;
}

@Override
public void release() {
this.ctx = null;
this.opLogger = null;
this.timeoutOpLogger = null;
this.perChannelBookieClient = null;
this.mdcContextMap = null;
handle.recycle(this);
}

@Override
public void writeComplete(int rc, long ledgerId, long entryId,
BookieId addr,
Object ctx) {
logOpResult(rc);
originalCallback.writeComplete(rc, ledgerId, entryId, addr, ctx);
key.release();
this.release();
}

@Override
boolean maybeTimeout() {
if (MathUtils.elapsedNanos(startTime) >= perChannelBookieClient.addEntryTimeoutNanos) {
timeout();
return true;
} else {
return false;
}
}

@Override
public void errorOut() {
errorOut(BKException.Code.BookieHandleNotAvailableException);
}

@Override
public void errorOut(final int rc) {
errorOutAndRunCallback(
() -> writeComplete(rc, ledgerId, entryId, perChannelBookieClient.bookieId, ctx));
}

@Override
public void setOutstanding() {
perChannelBookieClient.addEntryOutstanding.inc();
}

@Override
public void handleV2Response(
long ledgerId, long entryId, BookkeeperProtocol.StatusCode status,
BookieProtocol.Response response) {
perChannelBookieClient.addEntryOutstanding.dec();
handleResponse(ledgerId, entryId, status);
}

@Override
public void handleV3Response(
BookkeeperProtocol.Response response) {
perChannelBookieClient.addEntryOutstanding.dec();
BookkeeperProtocol.AddResponse addResponse = response.getAddResponse();
BookkeeperProtocol.StatusCode status = response.getStatus() == BookkeeperProtocol.StatusCode.EOK
? addResponse.getStatus() : response.getStatus();
handleResponse(addResponse.getLedgerId(), addResponse.getEntryId(),
status);
}

private void handleResponse(long ledgerId, long entryId,
BookkeeperProtocol.StatusCode status) {
if (LOG.isDebugEnabled()) {
logResponse(status, "ledger", ledgerId, "entry", entryId);
}

int rc = convertStatus(status, BKException.Code.WriteException);
writeComplete(rc, ledgerId, entryId, perChannelBookieClient.bookieId, ctx);
}

private static final Recycler<AddCompletion> ADD_COMPLETION_RECYCLER = new Recycler<AddCompletion>() {
@Override
protected AddCompletion newObject(Recycler.Handle<AddCompletion> handle) {
return new AddCompletion(handle);
}
};
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
/*
*
* 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.bookkeeper.proto;

import static org.apache.bookkeeper.client.LedgerHandle.INVALID_ENTRY_ID;

import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.util.ByteBufList;

class BatchedReadCompletion extends CompletionValue {

final BookkeeperInternalCallbacks.BatchedReadEntryCallback cb;

public BatchedReadCompletion(final CompletionKey key,
final BookkeeperInternalCallbacks.BatchedReadEntryCallback originalCallback,
final Object originalCtx,
long ledgerId, final long entryId,
PerChannelBookieClient perChannelBookieClient) {
super("BatchedRead", originalCtx, ledgerId, entryId, perChannelBookieClient);
this.opLogger = perChannelBookieClient.readEntryOpLogger;
this.timeoutOpLogger = perChannelBookieClient.readTimeoutOpLogger;
this.cb = (rc, ledgerId1, startEntryId, bufList, ctx) -> {
logOpResult(rc);
originalCallback.readEntriesComplete(rc,
ledgerId1, entryId,
bufList, originalCtx);
key.release();
};
}

@Override
public void errorOut() {
errorOut(BKException.Code.BookieHandleNotAvailableException);
}

@Override
public void errorOut(final int rc) {
errorOutAndRunCallback(
() -> cb.readEntriesComplete(rc, ledgerId,
entryId, null, ctx));
}

@Override
public void handleV2Response(long ledgerId,
long entryId,
BookkeeperProtocol.StatusCode status,
BookieProtocol.Response response) {

perChannelBookieClient.readEntryOutstanding.dec();
if (!(response instanceof BookieProtocol.BatchedReadResponse)) {
return;
}
BookieProtocol.BatchedReadResponse readResponse = (BookieProtocol.BatchedReadResponse) response;
handleBatchedReadResponse(ledgerId, entryId, status, readResponse.getData(),
INVALID_ENTRY_ID, -1L);
}

@Override
public void handleV3Response(BookkeeperProtocol.Response response) {
// V3 protocol haven't supported batched read yet.
}

private void handleBatchedReadResponse(long ledgerId,
long entryId,
BookkeeperProtocol.StatusCode status,
ByteBufList buffers,
long maxLAC, // max known lac piggy-back from bookies
long lacUpdateTimestamp) { // the timestamp when the lac is updated.
int rc = convertStatus(status, BKException.Code.ReadException);

if (maxLAC > INVALID_ENTRY_ID && (ctx instanceof BookkeeperInternalCallbacks.ReadEntryCallbackCtx)) {
((BookkeeperInternalCallbacks.ReadEntryCallbackCtx) ctx).setLastAddConfirmed(maxLAC);
}
if (lacUpdateTimestamp > -1L && (ctx instanceof ReadLastConfirmedAndEntryContext)) {
((ReadLastConfirmedAndEntryContext) ctx).setLacUpdateTimestamp(lacUpdateTimestamp);
}
cb.readEntriesComplete(rc, ledgerId, entryId, buffers, ctx);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
*
* 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.bookkeeper.proto;

import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType;

abstract class CompletionKey {
OperationType operationType;

CompletionKey(OperationType operationType) {
this.operationType = operationType;
}

public void release() {}
}
Loading