From adfd619ca0f5ec7fd8cfa919020df2a31e0b3944 Mon Sep 17 00:00:00 2001 From: Priyesh Narayanan Date: Wed, 13 Apr 2016 11:46:36 -0700 Subject: [PATCH 1/6] GetManager implementation with GetBlobInfo support --- .../com.github.ambry/config/RouterConfig.java | 16 + .../messageformat/BlobProperties.java | 15 + .../MockBlockingChannel.java | 4 +- .../ByteBufferSend.java | 6 +- .../DeleteRequestRegistrationCallback.java | 25 - .../GetBlobInfoOperation.java | 243 +++++++++ .../com.github.ambry.router/GetManager.java | 124 ++++- .../com.github.ambry.router/GetOperation.java | 204 ++++++++ .../NonBlockingRouter.java | 17 +- .../com.github.ambry.router/PutManager.java | 11 +- .../com.github.ambry.router/PutOperation.java | 13 +- ....java => RequestRegistrationCallback.java} | 8 +- .../com.github.ambry.router/RouterUtils.java | 2 +- .../DeleteManagerTest.java | 2 +- .../GetManagerTest.java | 172 +++++++ .../GetOperationTest.java | 474 ++++++++++++++++++ .../com.github.ambry.router/MockSelector.java | 4 +- .../com.github.ambry.router/MockServer.java | 220 ++++++-- .../MockServerLayout.java | 6 +- .../NonBlockingRouterTest.java | 2 +- .../PutManagerTest.java | 22 +- .../PutOperationTest.java | 11 +- build.gradle | 1 + 23 files changed, 1462 insertions(+), 140 deletions(-) rename {ambry-coordinator/src/test/java/com.github.ambry.coordinator => ambry-network/src/test/java/com.github.ambry.network}/ByteBufferSend.java (91%) delete mode 100644 ambry-router/src/main/java/com.github.ambry.router/DeleteRequestRegistrationCallback.java create mode 100644 ambry-router/src/main/java/com.github.ambry.router/GetBlobInfoOperation.java create mode 100644 ambry-router/src/main/java/com.github.ambry.router/GetOperation.java rename ambry-router/src/main/java/com.github.ambry.router/{PutRequestRegistrationCallback.java => RequestRegistrationCallback.java} (62%) create mode 100644 ambry-router/src/test/java/com.github.ambry.router/GetManagerTest.java create mode 100644 ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java diff --git a/ambry-api/src/main/java/com.github.ambry/config/RouterConfig.java b/ambry-api/src/main/java/com.github.ambry/config/RouterConfig.java index 9d440deca4..0348c94774 100644 --- a/ambry-api/src/main/java/com.github.ambry/config/RouterConfig.java +++ b/ambry-api/src/main/java/com.github.ambry/config/RouterConfig.java @@ -110,6 +110,20 @@ public class RouterConfig { @Default("2") public final int routerDeleteSuccessTarget; + /** + * The maximum number of parallel requests issued at a time by the get manager for a get operation on a chunk. + */ + @Config("router.get.request.parallelism") + @Default("2") + public final int routerGetRequestParallelism; + + /** + * The minimum number of successful responses required for a get operation on a chunk. + */ + @Config("router.get.success.target") + @Default("1") + public final int routerGetSuccessTarget; + /** * Create a RouterConfig instance. * @param verifiableProperties the properties map to refer to. @@ -131,5 +145,7 @@ public RouterConfig(VerifiableProperties verifiableProperties) { routerMaxSlippedPutAttempts = verifiableProperties.getInt("router.max.slipped.put.attempts", 1); routerDeleteRequestParallelism = verifiableProperties.getInt("router.delete.request.parallelism", 12); routerDeleteSuccessTarget = verifiableProperties.getInt("router.delete.success.target", 2); + routerGetRequestParallelism = verifiableProperties.getInt("router.get.request.parallelism", 2); + routerGetSuccessTarget = verifiableProperties.getInt("router.get.success.target", 1); } } diff --git a/ambry-api/src/main/java/com.github.ambry/messageformat/BlobProperties.java b/ambry-api/src/main/java/com.github.ambry/messageformat/BlobProperties.java index 79da3841a4..74499f9d7e 100644 --- a/ambry-api/src/main/java/com.github.ambry/messageformat/BlobProperties.java +++ b/ambry-api/src/main/java/com.github.ambry/messageformat/BlobProperties.java @@ -131,4 +131,19 @@ public String toString() { sb.append("]"); return sb.toString(); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null) { + return false; + } + + BlobProperties that = (BlobProperties) o; + return blobSize == that.blobSize && serviceId.equals(that.serviceId) && ownerId.equals(that.ownerId) && contentType + .equals(that.contentType) && isPrivate == that.isPrivate && timeToLiveInSeconds == that.timeToLiveInSeconds; + } } diff --git a/ambry-coordinator/src/test/java/com.github.ambry.coordinator/MockBlockingChannel.java b/ambry-coordinator/src/test/java/com.github.ambry.coordinator/MockBlockingChannel.java index c25d14ff0d..2c196ccf04 100644 --- a/ambry-coordinator/src/test/java/com.github.ambry.coordinator/MockBlockingChannel.java +++ b/ambry-coordinator/src/test/java/com.github.ambry.coordinator/MockBlockingChannel.java @@ -18,9 +18,10 @@ import com.github.ambry.messageformat.BlobOutput; import com.github.ambry.messageformat.BlobProperties; import com.github.ambry.messageformat.MessageFormatRecord; -import com.github.ambry.network.Send; import com.github.ambry.network.BlockingChannel; +import com.github.ambry.network.ByteBufferSend; import com.github.ambry.network.ChannelOutput; +import com.github.ambry.network.Send; import com.github.ambry.protocol.DeleteRequest; import com.github.ambry.protocol.DeleteResponse; import com.github.ambry.protocol.GetRequest; @@ -31,7 +32,6 @@ import com.github.ambry.protocol.RequestOrResponse; import com.github.ambry.store.MessageInfo; import com.github.ambry.utils.Crc32; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; diff --git a/ambry-coordinator/src/test/java/com.github.ambry.coordinator/ByteBufferSend.java b/ambry-network/src/test/java/com.github.ambry.network/ByteBufferSend.java similarity index 91% rename from ambry-coordinator/src/test/java/com.github.ambry.coordinator/ByteBufferSend.java rename to ambry-network/src/test/java/com.github.ambry.network/ByteBufferSend.java index ffa47e22ab..b3b54653de 100644 --- a/ambry-coordinator/src/test/java/com.github.ambry.coordinator/ByteBufferSend.java +++ b/ambry-network/src/test/java/com.github.ambry.network/ByteBufferSend.java @@ -11,9 +11,7 @@ * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. */ -package com.github.ambry.coordinator; - -import com.github.ambry.network.Send; +package com.github.ambry.network; import java.io.IOException; import java.nio.ByteBuffer; @@ -24,7 +22,7 @@ * A byte buffer version of Send that sends a materialized byte buffer. This breaks the contract of Send (only * materialize onto the network) and so is only suitable for use in tests. */ -class ByteBufferSend implements Send { +public class ByteBufferSend implements Send { private final ByteBuffer buffer; public ByteBufferSend(ByteBuffer byteBuffer) diff --git a/ambry-router/src/main/java/com.github.ambry.router/DeleteRequestRegistrationCallback.java b/ambry-router/src/main/java/com.github.ambry.router/DeleteRequestRegistrationCallback.java deleted file mode 100644 index 6f2544d5a4..0000000000 --- a/ambry-router/src/main/java/com.github.ambry.router/DeleteRequestRegistrationCallback.java +++ /dev/null @@ -1,25 +0,0 @@ -/** - * Copyright 2016 LinkedIn Corp. All rights reserved. - * - * Licensed 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. - */ -package com.github.ambry.router; - -import com.github.ambry.network.RequestInfo; - - -/** - * The callback to be used when delete requests are created and needs to be sent out. The {@link DeleteManager} passes this - * callback to the {@link DeleteOperation}. - */ -interface DeleteRequestRegistrationCallback { - public void registerRequestToSend(DeleteOperation deleteOperation, RequestInfo request); -} diff --git a/ambry-router/src/main/java/com.github.ambry.router/GetBlobInfoOperation.java b/ambry-router/src/main/java/com.github.ambry.router/GetBlobInfoOperation.java new file mode 100644 index 0000000000..f6b4dac77d --- /dev/null +++ b/ambry-router/src/main/java/com.github.ambry.router/GetBlobInfoOperation.java @@ -0,0 +1,243 @@ +/** + * Copyright 2016 LinkedIn Corp. All rights reserved. + * + * Licensed 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. + */ +package com.github.ambry.router; + +import com.github.ambry.clustermap.ClusterMap; +import com.github.ambry.clustermap.ReplicaId; +import com.github.ambry.commons.ResponseHandler; +import com.github.ambry.commons.ServerErrorCode; +import com.github.ambry.config.RouterConfig; +import com.github.ambry.messageformat.BlobInfo; +import com.github.ambry.messageformat.MessageFormatException; +import com.github.ambry.messageformat.MessageFormatFlags; +import com.github.ambry.messageformat.MessageFormatRecord; +import com.github.ambry.network.Port; +import com.github.ambry.network.PortType; +import com.github.ambry.network.RequestInfo; +import com.github.ambry.network.ResponseInfo; +import com.github.ambry.protocol.GetRequest; +import com.github.ambry.protocol.GetResponse; +import com.github.ambry.utils.ByteBufferInputStream; +import com.github.ambry.utils.Time; +import java.io.DataInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Iterator; +import java.util.Map; +import java.util.TreeMap; + + +/** + * GetBlobInfoOperation class is responsible for maintaining the state associated with a getBlobInfo operation, + * and completing it. A GetBlobInfo operation only needs to make requests for a single chunk to get the BlobInfo - + * which is either the only chunk in the case of a simple blob, or the metadata chunk in the case of composite blobs. + */ +class GetBlobInfoOperation extends GetOperation { + private BlobInfo blobInfo; + private final SimpleOperationTracker operationTracker; + // map of correlation id to the request metadata for every request issued for this operation. + protected final Map correlationIdToGetRequestInfo = new TreeMap(); + + /** + * Construct a GetBlobInfoOperation + * @param routerConfig the {@link RouterConfig} containing the configs for get operations. + * @param clusterMap the {@link ClusterMap} of the cluster + * @param responseHandler the {@link ResponseHandler} responsible for failure detection. + * @param blobIdStr the blob id associated with the operation in string form. + * @param futureResult the future that will contain the result of the operation. + * @param callback the callback that is to be called when the operation completes. + * @param time the Time instance to use. + * @throws RouterException if there is an error with any of the parameters, such as an invalid blob id. + */ + GetBlobInfoOperation(RouterConfig routerConfig, ClusterMap clusterMap, ResponseHandler responseHandler, + String blobIdStr, FutureResult futureResult, Callback callback, Time time) + throws RouterException { + super(routerConfig, clusterMap, responseHandler, blobIdStr, futureResult, callback, time); + operationTracker = new SimpleOperationTracker(routerConfig.routerDatacenterName, blobId.getPartition(), true, + routerConfig.routerGetSuccessTarget, routerConfig.routerGetRequestParallelism); + } + + /** + * Return the {@link MessageFormatFlags} to associate with a getBlobInfo operation. + * @return {@link MessageFormatFlags#BlobInfo} + */ + @Override + MessageFormatFlags getOperationFlag() { + return MessageFormatFlags.BlobInfo; + } + + /** + * For this operation, create and populate get requests (in the form of {@link RequestInfo}) to send out. + * @param requestRegistrationCallback the {@link RequestRegistrationCallback} to call for every request that gets + * created as part of this poll operation. + */ + @Override + void poll(RequestRegistrationCallback requestRegistrationCallback) { + //First, check if any of the existing requests have timed out. + Iterator> inFlightRequestsIterator = + correlationIdToGetRequestInfo.entrySet().iterator(); + while (inFlightRequestsIterator.hasNext()) { + Map.Entry entry = inFlightRequestsIterator.next(); + if (time.milliseconds() - entry.getValue().startTimeMs > routerConfig.routerRequestTimeoutMs) { + operationTracker.onResponse(entry.getValue().replicaId, false); + setOperationException( + new RouterException("Timed out waiting for responses", RouterErrorCode.OperationTimedOut)); + inFlightRequestsIterator.remove(); + } else { + // the entries are ordered by correlation id and time. Break on the first request that has not timed out. + break; + } + } + + checkAndMaybeComplete(); + if (isOperationComplete()) { + return; + } + + Iterator replicaIterator = operationTracker.getReplicaIterator(); + while (replicaIterator.hasNext()) { + ReplicaId replicaId = replicaIterator.next(); + String hostname = replicaId.getDataNodeId().getHostname(); + Port port = new Port(replicaId.getDataNodeId().getPort(), PortType.PLAINTEXT); + GetRequest getRequest = createGetRequest(blobId, getOperationFlag()); + RequestInfo request = new RequestInfo(hostname, port, getRequest); + int correlationId = getRequest.getCorrelationId(); + correlationIdToGetRequestInfo.put(correlationId, new GetRequestInfo(replicaId, time.milliseconds())); + requestRegistrationCallback.registerRequestToSend(this, request); + replicaIterator.remove(); + } + } + + /** + * Handle the body of the response: Deserialize and set the {@link BlobInfo} to return. + * @param payload the body of the response. + * @throws IOException if there is an IOException while deserializing the body. + * @throws MessageFormatException if there is a MessageFormatException while deserializing the body. + */ + void handleBody(InputStream payload) + throws IOException, MessageFormatException { + if (blobInfo == null) { + blobInfo = new BlobInfo(MessageFormatRecord.deserializeBlobProperties(payload), + MessageFormatRecord.deserializeUserMetadata(payload).array()); + } else { + // If the successTarget is 1, this case will never get executed. + // If it is more than 1, then, different responses will have to be reconciled in some way. Here is where that + // would be done. Since the store is immutable, currently we handle this by ignoring subsequent responses + // (if the successTarget is > 1). + } + } + + /** + * Handle the given {@link ResponseInfo} and update the status of the operation. + * Based on the status of the response, this involves handling the body, notifying the operation tracker, + * notifying the response handler and possibly setting the operationException. + * @param responseInfo the {@link ResponseInfo} to be handled. + */ + @Override + void handleResponse(ResponseInfo responseInfo) { + if (isOperationComplete()) { + return; + } + int correlationId = ((GetRequest) responseInfo.getRequest()).getCorrelationId(); + // Get the GetOperation that generated the request. + GetRequestInfo getRequestInfo = correlationIdToGetRequestInfo.remove(correlationId); + if (getRequestInfo == null) { + // Ignore right away. This associated operation has completed. + return; + } + if (responseInfo.getError() != null) { + setOperationException(new RouterException("Operation timed out", RouterErrorCode.OperationTimedOut)); + responseHandler.onRequestResponseException(getRequestInfo.replicaId, new IOException("NetworkClient error")); + operationTracker.onResponse(getRequestInfo.replicaId, false); + } else { + try { + GetResponse getResponse = GetResponse + .readFrom(new DataInputStream(new ByteBufferInputStream(responseInfo.getResponse())), clusterMap); + if (getResponse.getCorrelationId() != correlationId) { + // The NetworkClient associates a response with a request based on the fact that only one request is sent + // out over a connection id, and the response received on a connection id must be for the latest request + // sent over it. The check here ensures that is indeed the case. If not, log an error and fail this request. + // There is no other way to handle it. + logger.error("The correlation id in the GetResponse " + getResponse.getCorrelationId() + + " is not the same as the correlation id in the associated GetRequest: " + correlationId); + setOperationException( + new RouterException("Unexpected internal error", RouterErrorCode.UnexpectedInternalError)); + operationTracker.onResponse(getRequestInfo.replicaId, false); + // we do not notify the ResponseHandler responsible for failure detection as this is an unexpected error. + } else { + ServerErrorCode getError = getResponse.getError(); + if (getError == ServerErrorCode.No_Error) { + int partitionsInResponse = getResponse.getPartitionResponseInfoList().size(); + // Each get request issued by the router is for a single blob. + if (partitionsInResponse != 1) { + setOperationException(new RouterException("Unexpected number of partition responses, expected: 1, " + + "received: " + partitionsInResponse, RouterErrorCode.UnexpectedInternalError)); + operationTracker.onResponse(getRequestInfo.replicaId, false); + // Again, no need to notify the responseHandler. + } else { + getError = getResponse.getPartitionResponseInfoList().get(0).getErrorCode(); + responseHandler.onRequestResponseError(getRequestInfo.replicaId, getError); + if (getError == ServerErrorCode.No_Error) { + handleBody(getResponse.getInputStream()); + operationTracker.onResponse(getRequestInfo.replicaId, true); + } else { + // process and set the most relevant exception. + processServerError(getError); + if (getError == ServerErrorCode.Blob_Deleted || getError == ServerErrorCode.Blob_Expired) { + // this is a successful response and one that completes the operation regardless of whether the + // success target has been reached or not. + operationCompleted = true; + } else { + operationTracker.onResponse(getRequestInfo.replicaId, false); + } + } + } + } else { + responseHandler.onRequestResponseError(getRequestInfo.replicaId, getError); + operationTracker.onResponse(getRequestInfo.replicaId, false); + } + } + } catch (IOException e) { + // This should really not happen. Again, we do not notify the ResponseHandler responsible for failure + // detection. + setOperationException(new RouterException("Response deserialization received an unexpected error", e, + RouterErrorCode.UnexpectedInternalError)); + operationTracker.onResponse(getRequestInfo.replicaId, false); + } catch (MessageFormatException e) { + setOperationException(new RouterException("Response deserialization received an unexpected error", e, + RouterErrorCode.UnexpectedInternalError)); + operationTracker.onResponse(getRequestInfo.replicaId, false); + } + } + checkAndMaybeComplete(); + } + + /** + * Check whether the operation can be completed, if so complete it. + */ + private void checkAndMaybeComplete() { + if (operationTracker.isDone()) { + if (operationTracker.hasSucceeded()) { + operationException = null; + } + operationCompleted = true; + } + // this check is outside as the operation could get completed outside of the operation tracking if a + // Blob_Deleted error or Blob_Expired error was received (in which case the success target is not honored). + if (operationCompleted) { + NonBlockingRouter.completeOperation(operationFuture, operationCallback, blobInfo, operationException); + } + } +} + diff --git a/ambry-router/src/main/java/com.github.ambry.router/GetManager.java b/ambry-router/src/main/java/com.github.ambry.router/GetManager.java index 7c3c807ba4..93950df286 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/GetManager.java +++ b/ambry-router/src/main/java/com.github.ambry.router/GetManager.java @@ -19,8 +19,19 @@ import com.github.ambry.messageformat.BlobInfo; import com.github.ambry.network.RequestInfo; import com.github.ambry.network.ResponseInfo; +import com.github.ambry.protocol.GetRequest; +import com.github.ambry.protocol.RequestOrResponse; import com.github.ambry.utils.Time; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -28,6 +39,37 @@ * These methods have to be thread safe. */ class GetManager { + private static final Logger logger = LoggerFactory.getLogger(GetManager.class); + + private final Set getOperations; + private final Time time; + // This helps the GetManager quickly find the appropriate GetOperation to hand over the response to. + // Requests are added before they are sent out and get cleaned up as and when responses come in. + // Because there is a guaranteed response from the NetworkClient for every request sent out, entries + // get cleaned up periodically. + private final Map correlationIdToGetOperation = new HashMap(); + private final AtomicBoolean isOpen = new AtomicBoolean(true); + + // shared by all GetOperations + private final ClusterMap clusterMap; + private final RouterConfig routerConfig; + private final ResponseHandler responseHandler; + private final NonBlockingRouterMetrics routerMetrics; + + private class GetRequestRegistrationCallbackImpl implements RequestRegistrationCallback { + private List requestListToFill; + + @Override + public void registerRequestToSend(GetOperation getOperation, RequestInfo requestInfo) { + requestListToFill.add(requestInfo); + correlationIdToGetOperation.put(((RequestOrResponse) requestInfo.getRequest()).getCorrelationId(), getOperation); + } + }; + // A single callback as this will never get called concurrently. The list of request to fill will be set as + // appropriate before the callback is passed on to GetOperations, every time. + private final GetRequestRegistrationCallbackImpl requestRegistrationCallback = + new GetRequestRegistrationCallbackImpl(); + /** * Create a GetManager * @param clusterMap The {@link ClusterMap} of the cluster. @@ -38,29 +80,89 @@ class GetManager { */ GetManager(ClusterMap clusterMap, ResponseHandler responseHandler, RouterConfig routerConfig, NonBlockingRouterMetrics routerMetrics, Time time) { - //@todo + this.clusterMap = clusterMap; + this.responseHandler = responseHandler; + this.routerConfig = routerConfig; + this.routerMetrics = routerMetrics; + this.time = time; + getOperations = Collections.newSetFromMap(new ConcurrentHashMap()); } - void submitGetBlobOperation(String blobId, FutureResult futureResult, - Callback callback) { - //@todo + /** + * Submit an operation to get the BlobInfo associated with a blob asynchronously. + * @param blobId the blobId for which the BlobInfo is being requested, in string form. + * @param futureResult the {@link FutureResult} that contains the pending result of the operation. + * @param callback the {@link Callback} object to be called on completion of the operation. + */ + void submitGetBlobInfoOperation(String blobId, FutureResult futureResult, Callback callback) { + try { + GetBlobInfoOperation getBlobInfoOperation = + new GetBlobInfoOperation(routerConfig, clusterMap, responseHandler, blobId, futureResult, callback, time); + getOperations.add(getBlobInfoOperation); + } catch (RouterException e) { + NonBlockingRouter.completeOperation(futureResult, callback, null, e); + } } - void submitGetBlobInfoOperation(String blobId, FutureResult futureResult, - Callback callback) { - //@todo + /** + * Submit an operation to get a blob asynchronously. + * @param blobId the blobId for which the BlobInfo is being requested, in string form. + * @param futureResult the {@link FutureResult} that contains the pending result of the operation. + * @param callback the {@link Callback} object to be called on completion of the operation. + */ + void submitGetBlobOperation(String blobId, FutureResult futureResult, + Callback callback) { + // @todo } - void poll(List requests) { - //@todo + /** + * Creates and returns requests in the form of {@link RequestInfo} to be sent to data nodes in order to complete + * get operations. Since this is the only method guaranteed to be called periodically by the RequestResponseHandler + * thread in the {@link NonBlockingRouter} ({@link #handleResponse} gets called only if a + * response is received for a get operation), any error handling or operation completion and cleanup also usually + * gets done in the context of this method. + * @param requestListToFill list to be filled with the requests created + */ + void poll(List requestListToFill) { + Iterator getOperationIterator = getOperations.iterator(); + requestRegistrationCallback.requestListToFill = requestListToFill; + while (getOperationIterator.hasNext()) { + GetOperation op = getOperationIterator.next(); + op.poll(requestRegistrationCallback); + if (op.isOperationComplete()) { + getOperationIterator.remove(); + } + } } + /** + * Hands over the response to the associated GetOperation that issued the request. + * @param responseInfo the {@link ResponseInfo} containing the response. + */ void handleResponse(ResponseInfo responseInfo) { - // @todo + GetRequest getRequest = (GetRequest) responseInfo.getRequest(); + GetOperation getOperation = correlationIdToGetOperation.get(getRequest.getCorrelationId()); + if (getOperations.contains(getOperation)) { + getOperation.handleResponse(responseInfo); + if (getOperation.isOperationComplete()) { + getOperations.remove(getOperation); + } + } } + /** + * Close the GetManager. + * Complete all existing get operations. + */ void close() { - // @todo + Iterator iter = getOperations.iterator(); + while (iter.hasNext()) { + GetOperation op = iter.next(); + logger.trace("Aborting operation for blob id: ", op.getBlobIdStr()); + NonBlockingRouter.completeOperation(op.getFuture(), op.getCallback(), null, + new RouterException("Aborted operation because Router is closed", RouterErrorCode.RouterClosed)); + iter.remove(); + } } } diff --git a/ambry-router/src/main/java/com.github.ambry.router/GetOperation.java b/ambry-router/src/main/java/com.github.ambry.router/GetOperation.java new file mode 100644 index 0000000000..2e9ac6e804 --- /dev/null +++ b/ambry-router/src/main/java/com.github.ambry.router/GetOperation.java @@ -0,0 +1,204 @@ +/** + * Copyright 2016 LinkedIn Corp. All rights reserved. + * + * Licensed 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. + */ +package com.github.ambry.router; + +import com.github.ambry.clustermap.ClusterMap; +import com.github.ambry.clustermap.ReplicaId; +import com.github.ambry.commons.BlobId; +import com.github.ambry.commons.ResponseHandler; +import com.github.ambry.commons.ServerErrorCode; +import com.github.ambry.config.RouterConfig; +import com.github.ambry.messageformat.MessageFormatFlags; +import com.github.ambry.network.ResponseInfo; +import com.github.ambry.protocol.GetOptions; +import com.github.ambry.protocol.GetRequest; +import com.github.ambry.protocol.PartitionRequestInfo; +import com.github.ambry.utils.Time; +import java.util.ArrayList; +import java.util.List; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * An abstract class for a get operation. + * @param the type of the result of this operation. + */ +abstract class GetOperation { + final RouterConfig routerConfig; + final ClusterMap clusterMap; + final ResponseHandler responseHandler; + final FutureResult operationFuture; + final Callback operationCallback; + final BlobId blobId; + final Time time; + boolean operationCompleted = false; + Exception operationException; + + static final Logger logger = LoggerFactory.getLogger(GetOperation.class); + + /** + * Construct a GetOperation + * @param routerConfig the {@link RouterConfig} containing the configs for put operations. + * @param clusterMap the {@link ClusterMap} of the cluster + * @param responseHandler the {@link ResponseHandler} responsible for failure detection. + * @param blobIdStr the blobId of the associated blob in string form. + * @param futureResult the future that will contain the result of the operation. + * @param callback the callback that is to be called when the operation completes. + * @param time the {@link Time} instance to use. + * @throws RouterException if there is an error with any of the parameters, such as an invalid blob id. + */ + GetOperation(RouterConfig routerConfig, ClusterMap clusterMap, ResponseHandler responseHandler, String blobIdStr, + FutureResult futureResult, Callback callback, Time time) + throws RouterException { + this.routerConfig = routerConfig; + this.clusterMap = clusterMap; + this.responseHandler = responseHandler; + this.operationFuture = futureResult; + this.operationCallback = callback; + this.time = time; + blobId = RouterUtils.getBlobIdFromString(blobIdStr, clusterMap); + } + + /** + * Return the {@link FutureResult} associated with this operation. + * @return the {@link FutureResult} associated with this operation. + */ + FutureResult getFuture() { + return operationFuture; + } + + /** + * Return the {@link Callback} associated with this operation. + * @return the {@link Callback} associated with this operation. + */ + Callback getCallback() { + return operationCallback; + } + + /** + * The exception associated with this operation if it failed; null otherwise. + * @return exception associated with this operation if it failed; null otherwise. + */ + Exception getOperationException() { + return operationException; + } + + /** + * Return the blob id string + * @return return the blob id string + */ + String getBlobIdStr() { + return blobId.getID(); + } + + /** + * returns whether the operation has completed. + * @return whether the operation has completed. + */ + boolean isOperationComplete() { + return operationCompleted; + } + + /** + * Return the {@link MessageFormatFlags} to associate with the requests for this operation. + * @return the {@link MessageFormatFlags} to associate with the requests for this operation. + */ + abstract MessageFormatFlags getOperationFlag(); + + /** + * For this operation, create and populate get requests to send out. + * @param requestRegistrationCallback the {@link RequestRegistrationCallback} to call for every request that gets + * created as part of this poll operation. + */ + abstract void poll(RequestRegistrationCallback requestRegistrationCallback); + + /** + * Handle the given {@link ResponseInfo} received for a request that was sent out. + * @param responseInfo the {@link ResponseInfo} to be handled. + */ + abstract void handleResponse(ResponseInfo responseInfo); + + /** + * Process the given {@link ServerErrorCode} and set operation status accordingly. + * @param errorCode the {@link ServerErrorCode} to process. + */ + void processServerError(ServerErrorCode errorCode) { + switch (errorCode) { + case Blob_Deleted: + logger.trace("Requested blob was deleted"); + setOperationException(new RouterException("Server returned: " + errorCode, RouterErrorCode.BlobDeleted)); + break; + case Blob_Expired: + logger.trace("Requested blob is expired"); + setOperationException(new RouterException("Server returned: " + errorCode, RouterErrorCode.BlobExpired)); + break; + case Blob_Not_Found: + logger.trace("Requested blob was not found on this server"); + setOperationException(new RouterException("Server returned: " + errorCode, RouterErrorCode.BlobDoesNotExist)); + break; + default: + setOperationException(new RouterException("Server returned: " + errorCode, RouterErrorCode.AmbryUnavailable)); + break; + } + } + + /** + * Set the exception associated with this operation. + * A Blob_Deleted or Blob_Expired error overrides any other error previously received exception. + * @param exception the RouterException to possibly set. + */ + void setOperationException(RouterException exception) { + if (operationException == null || exception.getErrorCode() == RouterErrorCode.BlobDeleted + || exception.getErrorCode() == RouterErrorCode.BlobExpired) { + operationException = exception; + } + } + + /** + * Create and return the {@link GetRequest} associated with the given blobId. + * @return the crated {@link GetRequest}. + * @param blobId The {@link BlobId} for which the GetRequest is being created. + * @param flag The {@link MessageFormatFlags} to be set with the GetRequest. + * @return the created GetRequest. + */ + protected GetRequest createGetRequest(BlobId blobId, MessageFormatFlags flag) { + ArrayList blobIds = new ArrayList(1); + blobIds.add(blobId); + List partitionRequestInfoList = new ArrayList(); + PartitionRequestInfo partitionRequestInfo = new PartitionRequestInfo(blobId.getPartition(), blobIds); + partitionRequestInfoList.add(partitionRequestInfo); + return new GetRequest(NonBlockingRouter.correlationIdGenerator.incrementAndGet(), routerConfig.routerHostname, flag, + partitionRequestInfoList, GetOptions.None); + } +} + +/** + * A class that holds information about the get requests sent out. + */ +class GetRequestInfo { + ReplicaId replicaId; + long startTimeMs; + + /** + * Construct a GetRequestInfo + * @param replicaId the replica to which this request is being sent. + * @param startTimeMs the time at which this request was created. + */ + GetRequestInfo(ReplicaId replicaId, long startTimeMs) { + this.replicaId = replicaId; + this.startTimeMs = startTimeMs; + } +} + diff --git a/ambry-router/src/main/java/com.github.ambry.router/NonBlockingRouter.java b/ambry-router/src/main/java/com.github.ambry.router/NonBlockingRouter.java index 293149b802..7718a3d571 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/NonBlockingRouter.java +++ b/ambry-router/src/main/java/com.github.ambry.router/NonBlockingRouter.java @@ -59,6 +59,7 @@ class NonBlockingRouter implements Router { private static final Logger logger = LoggerFactory.getLogger(NonBlockingRouter.class); private static final AtomicInteger currentOperationsCount = new AtomicInteger(0); + static final int MAX_IN_MEM_CHUNKS = 4; static final int SHUTDOWN_WAIT_MS = 10 * Time.MsPerSec; static final AtomicInteger correlationIdGenerator = new AtomicInteger(0); @@ -277,10 +278,24 @@ boolean isOpen() { * Return an approximate count of the number of operations submitted to the router that are not yet completed. * @return (approximate) number of operations being handled at the time of this call. */ - int getOperationsCount() { + static int getOperationsCount() { return currentOperationsCount.get(); } + /** + * Increment the count of currently running operations + */ + static void incrementOperationsCount() { + currentOperationsCount.incrementAndGet(); + } + + /** + * Decrement the count of currently running operations + */ + static void decrementOperationsCount() { + currentOperationsCount.decrementAndGet(); + } + /** * Completes a router operation by invoking the {@code callback} and setting the {@code futureResult} with * {@code operationResult} (if any) and {@code exception} (if any). diff --git a/ambry-router/src/main/java/com.github.ambry.router/PutManager.java b/ambry-router/src/main/java/com.github.ambry.router/PutManager.java index f9331e238e..fca7c915a2 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/PutManager.java +++ b/ambry-router/src/main/java/com.github.ambry.router/PutManager.java @@ -60,17 +60,16 @@ class PutManager { private final ResponseHandler responseHandler; private final NonBlockingRouterMetrics routerMetrics; - private class PutRequestRegistrationCallbackImpl implements PutRequestRegistrationCallback { + private class PutRequestRegistrationCallbackImpl implements RequestRegistrationCallback { private List requestListToFill; @Override public void registerRequestToSend(PutOperation putOperation, RequestInfo requestInfo) { requestListToFill.add(requestInfo); - correlationIdToPutOperation.put(((RequestOrResponse) requestInfo.getRequest()).getCorrelationId(), putOperation); + correlationIdToPutOperation + .put(((RequestOrResponse) requestInfo.getRequest()).getCorrelationId(), putOperation); } - } - - ; + }; // A single callback as this will never get called concurrently. The list of request to fill will be set as // appropriate before the callback is passed on to the PutOperations, every time. private final PutRequestRegistrationCallbackImpl requestRegistrationCallback = @@ -142,7 +141,7 @@ void poll(List requestListToFill) { } /** - * Hands over the response to the associated PutChunk that issued the request. + * Hands over the response to the associated PutOperation that issued the request. * @param responseInfo the {@link ResponseInfo} containing the response. */ void handleResponse(ResponseInfo responseInfo) { diff --git a/ambry-router/src/main/java/com.github.ambry.router/PutOperation.java b/ambry-router/src/main/java/com.github.ambry.router/PutOperation.java index 8b7e4b403a..5adbbc8520 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/PutOperation.java +++ b/ambry-router/src/main/java/com.github.ambry.router/PutOperation.java @@ -114,7 +114,6 @@ class PutOperation { // To find the PutChunk to hand over the response quickly. private final Map correlationIdToPutChunk = new HashMap(); - private static final int MAX_IN_MEM_CHUNKS = 4; private static final Logger logger = LoggerFactory.getLogger(PutOperation.class); /** @@ -164,7 +163,7 @@ class PutOperation { chunkCounter = -1; // Initialize chunks - putChunks = new PutChunk[Math.min(numDataChunks, MAX_IN_MEM_CHUNKS)]; + putChunks = new PutChunk[Math.min(numDataChunks, NonBlockingRouter.MAX_IN_MEM_CHUNKS)]; for (int i = 0; i < putChunks.length; i++) { putChunks[i] = new PutChunk(); } @@ -195,10 +194,10 @@ boolean isOperationComplete() { /** * For this operation, create and populate put requests for chunks (in the form of {@link RequestInfo}) to * send out. - * @param requestRegistrationCallback the {@link PutRequestRegistrationCallback} to call for every request that gets created - * as part of this poll operation. + * @param requestRegistrationCallback the {@link RequestRegistrationCallback} to call for every request that gets + * created as part of this poll operation. */ - void poll(PutRequestRegistrationCallback requestRegistrationCallback) { + void poll(RequestRegistrationCallback requestRegistrationCallback) { if (operationCompleted) { return; } @@ -662,10 +661,10 @@ void checkAndMaybeComplete() { * status of the operation and anything else that needs to be done within this PutChunk. The callers guarantee * that this method is called on all the PutChunks of an operation until either the operation, * or the chunk operation is completed. - * @param requestFillCallback the {@link PutRequestRegistrationCallback} to call for every request that gets created as + * @param requestFillCallback the {@link RequestRegistrationCallback} to call for every request that gets created as * part of this poll operation. */ - void poll(PutRequestRegistrationCallback requestFillCallback) { + void poll(RequestRegistrationCallback requestFillCallback) { maybeFreeDefunctBuffers(); //First, check if any of the existing requests have timed out. Iterator> inFlightRequestsIterator = diff --git a/ambry-router/src/main/java/com.github.ambry.router/PutRequestRegistrationCallback.java b/ambry-router/src/main/java/com.github.ambry.router/RequestRegistrationCallback.java similarity index 62% rename from ambry-router/src/main/java/com.github.ambry.router/PutRequestRegistrationCallback.java rename to ambry-router/src/main/java/com.github.ambry.router/RequestRegistrationCallback.java index f13864913f..0a4fd15a99 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/PutRequestRegistrationCallback.java +++ b/ambry-router/src/main/java/com.github.ambry.router/RequestRegistrationCallback.java @@ -17,10 +17,10 @@ /** - * The callback to be used when put requests are created and needs to be sent out. The {@link PutManager} passes this - * callback to the {@link PutOperation} and the {@link PutOperation} uses this callback when requests are created and + * The callback to be used when requests are created and needs to be sent out. The operation manager passes this + * callback to the associated operation class and the operation uses this callback when requests are created and * need to be sent out. */ -public interface PutRequestRegistrationCallback { - public void registerRequestToSend(PutOperation putOperation, RequestInfo request); +public interface RequestRegistrationCallback { + public void registerRequestToSend(T routerOperation, RequestInfo request); } diff --git a/ambry-router/src/main/java/com.github.ambry.router/RouterUtils.java b/ambry-router/src/main/java/com.github.ambry.router/RouterUtils.java index c3e83ed22a..05fe0866f5 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/RouterUtils.java +++ b/ambry-router/src/main/java/com.github.ambry.router/RouterUtils.java @@ -23,7 +23,7 @@ /** * This is a utility class used by Router. */ -public class RouterUtils { +class RouterUtils { private static Logger logger = LoggerFactory.getLogger(RestUtils.class); diff --git a/ambry-router/src/test/java/com.github.ambry.router/DeleteManagerTest.java b/ambry-router/src/test/java/com.github.ambry.router/DeleteManagerTest.java index 0024912815..8d190fe50e 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/DeleteManagerTest.java +++ b/ambry-router/src/test/java/com.github.ambry.router/DeleteManagerTest.java @@ -424,7 +424,7 @@ private void assertCloseCleanup() { TestUtils.numThreadsByThisName("ChunkFillerThread")); Assert.assertEquals("No RequestResponseHandler should be running after the router is closed", 0, TestUtils.numThreadsByThisName("RequestResponseHandlerThread")); - Assert.assertEquals("All operations should have completed", 0, router.getOperationsCount()); + Assert.assertEquals("All operations should have completed", 0, NonBlockingRouter.getOperationsCount()); } /** diff --git a/ambry-router/src/test/java/com.github.ambry.router/GetManagerTest.java b/ambry-router/src/test/java/com.github.ambry.router/GetManagerTest.java new file mode 100644 index 0000000000..b2f45fc3c1 --- /dev/null +++ b/ambry-router/src/test/java/com.github.ambry.router/GetManagerTest.java @@ -0,0 +1,172 @@ +/** + * Copyright 2016 LinkedIn Corp. All rights reserved. + * + * Licensed 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. + */ +package com.github.ambry.router; + +import com.codahale.metrics.MetricRegistry; +import com.github.ambry.clustermap.MockClusterMap; +import com.github.ambry.commons.ByteBufferReadableStreamChannel; +import com.github.ambry.commons.LoggingNotificationSystem; +import com.github.ambry.config.RouterConfig; +import com.github.ambry.config.VerifiableProperties; +import com.github.ambry.messageformat.BlobInfo; +import com.github.ambry.messageformat.BlobProperties; +import com.github.ambry.utils.MockTime; +import com.github.ambry.utils.Utils; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Properties; +import java.util.Random; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicReference; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + + +public class GetManagerTest { + private final MockServerLayout mockServerLayout; + private final MockTime mockTime = new MockTime(); + private final MockClusterMap mockClusterMap; + // this is a reference to the state used by the mockSelector. just allows tests to manipulate the state. + private AtomicReference mockSelectorState = new AtomicReference(); + private NonBlockingRouter router; + + private int chunkSize; + private int requestParallelism; + private int successTarget; + private final Random random = new Random(); + + // Request params; + BlobProperties putBlobProperties; + byte[] putUserMetadata; + byte[] putContent; + ReadableStreamChannel putChannel; + + private static final int MAX_PORTS_PLAIN_TEXT = 3; + private static final int MAX_PORTS_SSL = 3; + private static final int CHECKOUT_TIMEOUT_MS = 1000; + + /** + * Pre-initialization common to all tests. + */ + public GetManagerTest() + throws Exception { + // random chunkSize in the range [1, 1 MB] + chunkSize = random.nextInt(1024 * 1024) + 1; + requestParallelism = 3; + successTarget = 2; + mockSelectorState.set(MockSelectorState.Good); + mockClusterMap = new MockClusterMap(); + mockServerLayout = new MockServerLayout(mockClusterMap); + } + + /** + * Every test in this class should leave the router closed in the end. Some tests do additional checks after + * closing the router. This is just a guard to ensure that the tests are not broken (which helped when developing + * these tests). + */ + @After + public void postCheck() { + Assert.assertFalse("Router should be closed at the end of each test", router.isOpen()); + Assert.assertEquals("Router operations count must be zero", 0, NonBlockingRouter.getOperationsCount()); + } + + /** + * Tests get blob info of simple blobs + * @throws Exception + */ + @Test + public void testSimpleBlobGetBlobInfoSuccess() + throws Exception { + router = getNonBlockingRouter(); + setOperationParams(chunkSize); + String blobId = router.putBlob(putBlobProperties, putUserMetadata, putChannel).get(); + BlobInfo blobInfo = router.getBlobInfo(blobId).get(); + Assert.assertEquals("Blob properties should match", putBlobProperties, blobInfo.getBlobProperties()); + Assert.assertArrayEquals("User metadata should match", putUserMetadata, blobInfo.getUserMetadata()); + router.close(); + } + + /** + * Tests get blob info of composite blobs + * @throws Exception + */ + @Test + public void testCompositeBlobGetBlobInfoSuccess() + throws Exception { + router = getNonBlockingRouter(); + setOperationParams(chunkSize * 6 + 11); + String blobId = router.putBlob(putBlobProperties, putUserMetadata, putChannel).get(); + BlobInfo blobInfo = router.getBlobInfo(blobId).get(); + Assert.assertEquals("Blob properties should match", putBlobProperties, blobInfo.getBlobProperties()); + Assert.assertArrayEquals("User metadata should match", putUserMetadata, blobInfo.getUserMetadata()); + router.close(); + } + + /** + * Tests the failure case where poll throws and closes the router. This also tests the case where the GetManager + * gets closed with active operations, and ensures that operations get completed with the appropriate error. + * @throws Exception + */ + @Test + public void testFailureOnAllPollThatSends() + throws Exception { + router = getNonBlockingRouter(); + setOperationParams(chunkSize); + String blobId = router.putBlob(putBlobProperties, putUserMetadata, putChannel).get(); + try { + mockSelectorState.set(MockSelectorState.ThrowExceptionOnSend); + router.getBlobInfo(blobId).get(); + Assert.fail("operation should have thrown"); + } catch (ExecutionException e) { + RouterException routerException = (RouterException) e.getCause(); + Assert.assertEquals("Exception received should be router closed error", routerException.getErrorCode(), + RouterErrorCode.RouterClosed); + } + } + + /** + * @return Return a {@link NonBlockingRouter} created with default {@link VerifiableProperties} + */ + private NonBlockingRouter getNonBlockingRouter() + throws IOException { + Properties properties = new Properties(); + properties.setProperty("router.hostname", "localhost"); + properties.setProperty("router.datacenter.name", "DC1"); + properties.setProperty("router.max.put.chunk.size.bytes", Integer.toString(chunkSize)); + properties.setProperty("router.put.request.parallelism", Integer.toString(requestParallelism)); + properties.setProperty("router.put.success.target", Integer.toString(successTarget)); + VerifiableProperties vProps = new VerifiableProperties(properties); + router = new NonBlockingRouter(new RouterConfig(vProps), new NonBlockingRouterMetrics(new MetricRegistry()), + new MockNetworkClientFactory(vProps, mockSelectorState, MAX_PORTS_PLAIN_TEXT, MAX_PORTS_SSL, + CHECKOUT_TIMEOUT_MS, mockServerLayout, mockTime), new LoggingNotificationSystem(), mockClusterMap, + mockTime); + return router; + } + + /** + * Set operation parameters for the blob that will be put and got. + * @param blobSize the blob size for the blob that will be put and got. + */ + private void setOperationParams(int blobSize) { + putBlobProperties = + new BlobProperties(blobSize, "serviceId", "memberId", "contentType", false, Utils.Infinite_Time); + putUserMetadata = new byte[10]; + random.nextBytes(putUserMetadata); + putContent = new byte[blobSize]; + random.nextBytes(putContent); + putChannel = new ByteBufferReadableStreamChannel(ByteBuffer.wrap(putContent)); + } +} + diff --git a/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java b/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java new file mode 100644 index 0000000000..e22f942d8c --- /dev/null +++ b/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java @@ -0,0 +1,474 @@ +/** + * Copyright 2016 LinkedIn Corp. All rights reserved. + * + * Licensed 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. + */ +package com.github.ambry.router; + +import com.codahale.metrics.MetricRegistry; +import com.github.ambry.clustermap.MockClusterMap; +import com.github.ambry.commons.BlobId; +import com.github.ambry.commons.ByteBufferReadableStreamChannel; +import com.github.ambry.commons.LoggingNotificationSystem; +import com.github.ambry.commons.ResponseHandler; +import com.github.ambry.commons.ServerErrorCode; +import com.github.ambry.config.RouterConfig; +import com.github.ambry.config.VerifiableProperties; +import com.github.ambry.messageformat.BlobInfo; +import com.github.ambry.messageformat.BlobProperties; +import com.github.ambry.network.NetworkClient; +import com.github.ambry.network.NetworkClientErrorCode; +import com.github.ambry.network.RequestInfo; +import com.github.ambry.network.ResponseInfo; +import com.github.ambry.protocol.RequestOrResponse; +import com.github.ambry.utils.MockTime; +import com.github.ambry.utils.Utils; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.concurrent.atomic.AtomicReference; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + + +public class GetOperationTest { + private static final int MAX_PORTS_PLAIN_TEXT = 3; + private static final int MAX_PORTS_SSL = 3; + private static final int CHECKOUT_TIMEOUT_MS = 1000; + + private int requestParallelism = 2; + private int successTarget = 1; + private final RouterConfig routerConfig; + private final MockClusterMap mockClusterMap; + private final MockServerLayout mockServerLayout; + private final int replicasCount; + private final AtomicReference mockSelectorState = new AtomicReference<>(); + private final ResponseHandler responseHandler; + private final MockNetworkClientFactory networkClientFactory; + private final NetworkClient networkClient; + private final MockTime time; + private final Map correlationIdToGetOperation = new HashMap<>(); + private final NonBlockingRouter router; + private final int BLOB_SIZE = 100; + private final int BLOB_USER_METADATA_SIZE = 10; + private final Random random = new Random(); + private final String blobIdStr; + private final BlobProperties blobProperties; + private final byte[] userMetadata; + private final byte[] putContent; + private final GetTestRequestRegistrationCallbackImpl requestRegistrationCallback = + new GetTestRequestRegistrationCallbackImpl(); + private final FutureResult operationFuture = new FutureResult<>(); + private final AtomicReference operationResult = new AtomicReference<>(null); + private final AtomicReference operationException = new AtomicReference<>(null); + private final Callback operationCallback = new Callback() { + @Override + public void onCompletion(BlobInfo result, Exception exception) { + operationResult.set(result); + operationException.set(exception); + } + }; + + private class GetTestRequestRegistrationCallbackImpl implements RequestRegistrationCallback { + private List requestListToFill; + + @Override + public void registerRequestToSend(GetOperation getOperation, RequestInfo requestInfo) { + requestListToFill.add(requestInfo); + correlationIdToGetOperation.put(((RequestOrResponse) requestInfo.getRequest()).getCorrelationId(), getOperation); + } + } + + ; + + public GetOperationTest() + throws Exception { + VerifiableProperties vprops = getNonBlockingRouterProperties(); + routerConfig = new RouterConfig(vprops); + mockClusterMap = new MockClusterMap(); + mockServerLayout = new MockServerLayout(mockClusterMap); + replicasCount = mockClusterMap.getWritablePartitionIds().get(0).getReplicaIds().size(); + responseHandler = new ResponseHandler(mockClusterMap); + time = new MockTime(); + networkClientFactory = new MockNetworkClientFactory(vprops, mockSelectorState, MAX_PORTS_PLAIN_TEXT, MAX_PORTS_SSL, + CHECKOUT_TIMEOUT_MS, mockServerLayout, time); + router = new NonBlockingRouter(new RouterConfig(vprops), new NonBlockingRouterMetrics(new MetricRegistry()), + networkClientFactory, new LoggingNotificationSystem(), mockClusterMap, time); + blobProperties = new BlobProperties(BLOB_SIZE, "serviceId", "memberId", "contentType", false, Utils.Infinite_Time); + userMetadata = new byte[BLOB_USER_METADATA_SIZE]; + random.nextBytes(userMetadata); + putContent = new byte[BLOB_SIZE]; + random.nextBytes(putContent); + ReadableStreamChannel putChannel = new ByteBufferReadableStreamChannel(ByteBuffer.wrap(putContent)); + blobIdStr = router.putBlob(blobProperties, userMetadata, putChannel).get(); + router.close(); + networkClient = networkClientFactory.getNetworkClient(); + } + + @Before + public void before() { + // almost all tests create and complete an operation outside of the router. Router's count of active operations + // will get decremented when that happens. Since the operation is not submitted via the router in this test, + // the count would never have been incremented. It is done here so that the count is ultimately zero when each test + // completes. + NonBlockingRouter.incrementOperationsCount(); + } + + @After + public void after() { + if (networkClient != null) { + networkClient.close(); + } + Assert.assertEquals("All operations should have completed", 0, NonBlockingRouter.getOperationsCount()); + } + + /** + * Test {@link GetBlobInfoOperation} instantiation and validate the get methods. + * @throws Exception + */ + @Test + public void testInstantiation() + throws Exception { + String blobIdStr = (new BlobId(mockClusterMap.getWritablePartitionIds().get(0))).getID(); + // test a bad case + GetBlobInfoOperation op; + try { + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, "invalid_id", operationFuture, + operationCallback, time); + Assert.fail("Instantiation of GetBlobInfo operation with an invalid blob id must fail"); + } catch (RouterException e) { + Assert + .assertEquals("Unexpected exception received on creating GetBlobInfoOperation", RouterErrorCode.InvalidBlobId, + e.getErrorCode()); + } + + // test a good case + op = new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, + operationCallback, time); + + Assert.assertEquals("Callback must match", operationCallback, op.getCallback()); + Assert.assertEquals("Futures must match", operationFuture, op.getFuture()); + Assert.assertEquals("Blob ids must match", blobIdStr, op.getBlobIdStr()); + // count was incremented, but this operation was not completed, so decrement. + NonBlockingRouter.decrementOperationsCount(); + } + + /** + * Test basic successful operation completion, by polling and handing over responses to the BlobInfo operation. + * @throws Exception + */ + @Test + public void testPollAndResponseHandling() + throws Exception { + GetBlobInfoOperation op = + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, + operationCallback, time); + ArrayList requestListToFill = new ArrayList<>(); + requestRegistrationCallback.requestListToFill = requestListToFill; + op.poll(requestRegistrationCallback); + Assert.assertEquals("There should only be as many requests at this point as requestParallelism", requestParallelism, + correlationIdToGetOperation.size()); + + List responses = sendAndWaitForResponses(requestListToFill); + for (ResponseInfo response : responses) { + op.handleResponse(response); + if (op.isOperationComplete()) { + break; + } + } + Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); + assertSuccess(); + } + + /** + * Test the case where all requests time out within the GetOperation. + * @throws Exception + */ + @Test + public void testRouterRequestTimeoutAllFailure() + throws Exception { + GetBlobInfoOperation op = + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, + operationCallback, time); + requestRegistrationCallback.requestListToFill = new ArrayList<>(); + op.poll(requestRegistrationCallback); + while (!op.isOperationComplete()) { + time.sleep(routerConfig.routerRequestTimeoutMs + 1); + op.poll(requestRegistrationCallback); + } + // At this time requests would have been created for all replicas, as none of them were delivered, + // and cross-colo proxying is enabled by default. + Assert.assertEquals("Must have attempted sending requests to all replicas", replicasCount, + correlationIdToGetOperation.size()); + Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); + Assert.assertTrue(operationFuture.isDone()); + Assert.assertEquals(operationException.get(), op.getOperationException()); + RouterException routerException = (RouterException) operationException.get(); + Assert.assertEquals(RouterErrorCode.OperationTimedOut, routerException.getErrorCode()); + } + + /** + * Test the case where all requests time out within the NetworkClient. + * @throws Exception + */ + @Test + public void testNetworkClientTimeoutAllFailure() + throws Exception { + GetBlobInfoOperation op = + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, + operationCallback, time); + ArrayList requestListToFill = new ArrayList<>(); + requestRegistrationCallback.requestListToFill = requestListToFill; + + while (!op.isOperationComplete()) { + op.poll(requestRegistrationCallback); + List responses = sendAndWaitForResponses(requestListToFill); + requestListToFill.clear(); + for (ResponseInfo response : responses) { + ResponseInfo fakeResponse = + new ResponseInfo(response.getRequest(), NetworkClientErrorCode.NetworkError, response.getResponse()); + op.handleResponse(fakeResponse); + if (op.isOperationComplete()) { + break; + } + } + } + + Assert.assertEquals(replicasCount, correlationIdToGetOperation.size()); + Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); + Assert.assertTrue(operationFuture.isDone()); + RouterException routerException = (RouterException) operationException.get(); + Assert.assertEquals(RouterErrorCode.OperationTimedOut, routerException.getErrorCode()); + } + + /** + * Test the case where every server returns Blob_Not_Found. All servers must have been contacted, + * due to cross-colo proxying. + * @throws Exception + */ + @Test + public void testBlobNotFoundCase() + throws Exception { + GetBlobInfoOperation op = + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, + operationCallback, time); + ArrayList requestListToFill = new ArrayList<>(); + requestRegistrationCallback.requestListToFill = requestListToFill; + + for (MockServer server : mockServerLayout.getMockServers()) { + server.setServerErrorForAllRequests(ServerErrorCode.Blob_Not_Found); + } + + while (!op.isOperationComplete()) { + op.poll(requestRegistrationCallback); + List responses = sendAndWaitForResponses(requestListToFill); + for (ResponseInfo response : responses) { + op.handleResponse(response); + if (op.isOperationComplete()) { + break; + } + } + } + + Assert.assertEquals("Must have attempted sending requests to all replicas", replicasCount, + correlationIdToGetOperation.size()); + Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); + Assert.assertTrue(operationFuture.isDone()); + RouterException routerException = (RouterException) operationException.get(); + Assert.assertEquals(RouterErrorCode.BlobDoesNotExist, routerException.getErrorCode()); + } + + /** + * Test the case with Blob_Not_Found errors from most servers, and Blob_Deleted at just one server. The latter + * should be the exception received for the operation. + * @throws Exception + */ + @Test + public void testErrorPrecedenceWithBlobDeletedCase() + throws Exception { + GetBlobInfoOperation op = + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, + operationCallback, time); + ArrayList requestListToFill = new ArrayList<>(); + requestRegistrationCallback.requestListToFill = requestListToFill; + + int i = 0; + int indexToSetBlobDeleted = random.nextInt(replicasCount); + for (MockServer server : mockServerLayout.getMockServers()) { + if (i == indexToSetBlobDeleted) { + server.setServerErrorForAllRequests(ServerErrorCode.Blob_Deleted); + } else { + server.setServerErrorForAllRequests(ServerErrorCode.Blob_Not_Found); + } + i++; + } + + while (!op.isOperationComplete()) { + op.poll(requestRegistrationCallback); + List responses = sendAndWaitForResponses(requestListToFill); + for (ResponseInfo response : responses) { + op.handleResponse(response); + if (op.isOperationComplete()) { + break; + } + } + } + + Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); + Assert.assertTrue(operationFuture.isDone()); + RouterException routerException = (RouterException) operationException.get(); + Assert.assertEquals(RouterErrorCode.BlobDeleted, routerException.getErrorCode()); + } + + /** + * Test the case with Blob_Not_Found errors from most servers, and Blob_Expired at just one server. The latter + * should be the exception received for the operation. + * @throws Exception + */ + @Test + public void testErrorPrecedenceWithBlobExpiredCase() + throws Exception { + GetBlobInfoOperation op = + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, + operationCallback, time); + ArrayList requestListToFill = new ArrayList<>(); + requestRegistrationCallback.requestListToFill = requestListToFill; + + int i = 0; + int indexToSetBlobDeleted = random.nextInt(replicasCount); + for (MockServer server : mockServerLayout.getMockServers()) { + if (i == indexToSetBlobDeleted) { + server.setServerErrorForAllRequests(ServerErrorCode.Blob_Expired); + } else { + server.setServerErrorForAllRequests(ServerErrorCode.Blob_Not_Found); + } + i++; + } + + while (!op.isOperationComplete()) { + op.poll(requestRegistrationCallback); + List responses = sendAndWaitForResponses(requestListToFill); + for (ResponseInfo response : responses) { + op.handleResponse(response); + if (op.isOperationComplete()) { + break; + } + } + } + + Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); + Assert.assertTrue(operationFuture.isDone()); + RouterException routerException = (RouterException) operationException.get(); + Assert.assertEquals(RouterErrorCode.BlobExpired, routerException.getErrorCode()); + } + + /** + * Test the case with multiple errors (server level and partition level) from multiple servers, + * with just one server returning a successful response. The operation should succeed. + * @throws Exception + */ + @Test + public void testSuccessInThePresenceOfVariousErrors() + throws Exception { + GetBlobInfoOperation op = + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, + operationCallback, time); + ArrayList requestListToFill = new ArrayList<>(); + requestRegistrationCallback.requestListToFill = requestListToFill; + + ArrayList mockServers = new ArrayList<>(mockServerLayout.getMockServers()); + // set the status to various server level or partition level errors (not Blob_Deleted or Blob_Expired). + mockServers.get(0).setServerErrorForAllRequests(ServerErrorCode.Unknown_Error); + mockServers.get(1).setServerErrorForAllRequests(ServerErrorCode.Unknown_Error); + mockServers.get(2).setServerErrorForAllRequests(ServerErrorCode.IO_Error); + mockServers.get(3).setServerErrorForAllRequests(ServerErrorCode.No_Error); + mockServers.get(4).setServerErrorForAllRequests(ServerErrorCode.Data_Corrupt); + mockServers.get(5).setServerErrorForAllRequests(ServerErrorCode.Blob_Not_Found); + mockServers.get(6).setServerErrorForAllRequests(ServerErrorCode.No_Error); + mockServers.get(7).setServerErrorForAllRequests(ServerErrorCode.Disk_Unavailable); + mockServers.get(8).setServerErrorForAllRequests(ServerErrorCode.Unknown_Error); + + // set the status of one of the servers in the local datacenter to success (we depend on an actual put of the blob + // in order to get back the BlobInfo, and the put would have gone only to the local datacenter). + for (int i = 0; i < mockServers.size(); i++) { + MockServer mockServer = mockServers.get(i); + if (mockServer.getDataCenter().equals(routerConfig.routerDatacenterName)) { + mockServer.setServerErrorForAllRequests(ServerErrorCode.No_Error); + break; + } + } + + while (!op.isOperationComplete()) { + op.poll(requestRegistrationCallback); + List responses = sendAndWaitForResponses(requestListToFill); + for (ResponseInfo response : responses) { + op.handleResponse(response); + if (op.isOperationComplete()) { + break; + } + } + } + + Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); + assertSuccess(); + } + + /** + * Submit all the requests that were handed over by the operation and wait until a response is received for every + * one of them. + * @param requestList the list containing the requests handed over by the operation. + * @return the list of responses from the network client. + * @throws IOException + */ + private List sendAndWaitForResponses(List requestList) + throws IOException { + List responseList = new ArrayList<>(); + int sendCount = requestList.size(); + responseList.addAll(networkClient.sendAndPoll(requestList)); + requestList.clear(); + while (responseList.size() < sendCount) { + responseList.addAll(networkClient.sendAndPoll(requestList)); + } + return responseList; + } + + /** + * Assert that the operation is complete and successful. + */ + private void assertSuccess() { + Assert.assertTrue(operationFuture.isDone()); + Assert.assertEquals("Null expected", null, operationException.get()); + BlobInfo blobInfo = operationResult.get(); + Assert.assertEquals("Blob properties must be the same", blobProperties, blobInfo.getBlobProperties()); + Assert.assertArrayEquals("User metadata must be the same", userMetadata, blobInfo.getUserMetadata()); + } + + /** + * Get the properties for the {@link NonBlockingRouter}. + * @return the constructed properties. + */ + private VerifiableProperties getNonBlockingRouterProperties() { + Properties properties = new Properties(); + properties.setProperty("router.hostname", "localhost"); + properties.setProperty("router.datacenter.name", "DC1"); + properties.setProperty("router.get.request.parallelism", Integer.toString(requestParallelism)); + properties.setProperty("router.get.success.target", Integer.toString(successTarget)); + return new VerifiableProperties(properties); + } +} + diff --git a/ambry-router/src/test/java/com.github.ambry.router/MockSelector.java b/ambry-router/src/test/java/com.github.ambry.router/MockSelector.java index 2906128a5d..39b9e9b0bc 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/MockSelector.java +++ b/ambry-router/src/test/java/com.github.ambry.router/MockSelector.java @@ -59,7 +59,7 @@ class MockSelector extends Selector { // we don't need the actual selector, close it. super.close(); this.serverLayout = serverLayout; - this.state = state == null ? new AtomicReference(MockSelectorState.Good) : state; + this.state = state == null ? new AtomicReference(MockSelectorState.Good) : state; this.time = time; } @@ -203,7 +203,7 @@ enum MockSelectorState { */ ThrowExceptionOnSend, /** - * A state that causes all poll calls to throw an IOException regardless of whethere there are sends to perform or + * A state that causes all poll calls to throw an IOException regardless of whether there are sends to perform or * not. */ ThrowExceptionOnAllPoll, diff --git a/ambry-router/src/test/java/com.github.ambry.router/MockServer.java b/ambry-router/src/test/java/com.github.ambry.router/MockServer.java index 5f1e0b8853..8c1544fe4d 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/MockServer.java +++ b/ambry-router/src/test/java/com.github.ambry.router/MockServer.java @@ -13,20 +13,33 @@ */ package com.github.ambry.router; +import com.github.ambry.clustermap.ClusterMap; import com.github.ambry.commons.ServerErrorCode; +import com.github.ambry.messageformat.BlobProperties; +import com.github.ambry.messageformat.MessageFormatRecord; import com.github.ambry.network.BoundedByteBufferReceive; +import com.github.ambry.network.ByteBufferSend; import com.github.ambry.network.Send; import com.github.ambry.protocol.DeleteRequest; import com.github.ambry.protocol.DeleteResponse; +import com.github.ambry.protocol.GetRequest; +import com.github.ambry.protocol.GetResponse; +import com.github.ambry.protocol.PartitionRequestInfo; +import com.github.ambry.protocol.PartitionResponseInfo; import com.github.ambry.protocol.PutRequest; import com.github.ambry.protocol.PutResponse; import com.github.ambry.protocol.RequestOrResponse; import com.github.ambry.protocol.RequestOrResponseType; +import com.github.ambry.store.MessageInfo; +import com.github.ambry.store.StoreKey; import com.github.ambry.utils.ByteBufferChannel; -import java.io.ByteArrayOutputStream; -import java.io.DataOutputStream; +import com.github.ambry.utils.ByteBufferInputStream; +import com.github.ambry.utils.Crc32; +import com.github.ambry.utils.Utils; +import java.io.DataInputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.HashMap; import java.util.LinkedList; import java.util.List; @@ -39,10 +52,17 @@ */ class MockServer { private ServerErrorCode hardError = null; - private LinkedList putErrors = new LinkedList(); + private LinkedList serverErrors = new LinkedList(); private final Map blobs = new ConcurrentHashMap(); private final HashMap blobIdToServerErrorCode = new HashMap(); private boolean shouldRespond = true; + private final ClusterMap clusterMap; + private final String dataCenter; + + MockServer(ClusterMap clusterMap, String dataCenter) { + this.clusterMap = clusterMap; + this.dataCenter = dataCenter; + } /** * Take in a request in the form of {@link Send} and return a response in the form of a @@ -53,54 +73,137 @@ class MockServer { */ public MockBoundedByteBufferReceive send(Send send) throws IOException { - RequestOrResponseType type = ((RequestOrResponse) send).getRequestType(); if (!shouldRespond) { return null; } + ServerErrorCode serverError = + hardError != null ? hardError : serverErrors.size() > 0 ? serverErrors.poll() : ServerErrorCode.No_Error; + RequestOrResponseType type = ((RequestOrResponse) send).getRequestType(); + RequestOrResponse response; switch (type) { case PutRequest: - PutRequest putRequest = (PutRequest) send; - final ServerErrorCode putError = - hardError != null ? hardError : putErrors.size() > 0 ? putErrors.poll() : ServerErrorCode.No_Error; - if (putError == ServerErrorCode.No_Error) { - updateBlobMap(putRequest); - } - return new MockBoundedByteBufferReceive( - new PutResponse(putRequest.getCorrelationId(), putRequest.getClientId(), putError) { - ByteBuffer getPayload() - throws IOException { - ByteArrayOutputStream bStream = new ByteArrayOutputStream(); - DataOutputStream dStream = new DataOutputStream(bStream); - dStream.writeShort((short) RequestOrResponseType.PutResponse.ordinal()); - dStream.writeShort(versionId); - dStream.writeInt(correlationId); - dStream.writeInt(0); // avoiding adding clientId - dStream.writeShort((short) putError.ordinal()); - return ByteBuffer.wrap(bStream.toByteArray()); - } - }.getPayload()); - + response = makePutResponse((PutRequest) send, serverError); + break; + case GetRequest: + response = makeGetResponse((GetRequest) send, serverError); + break; case DeleteRequest: - final DeleteRequest deleteRequest = (DeleteRequest) send; - final String blobIdString = deleteRequest.getBlobId().getID(); - final ServerErrorCode error = getErrorFromBlobIdStr(blobIdString); - return new MockBoundedByteBufferReceive( - new DeleteResponse(deleteRequest.getCorrelationId(), deleteRequest.getClientId(), error) { - ByteBuffer getPayload() - throws IOException { - ByteArrayOutputStream bStream = new ByteArrayOutputStream(); - DataOutputStream dStream = new DataOutputStream(bStream); - dStream.writeShort((short) RequestOrResponseType.DeleteResponse.ordinal()); - dStream.writeShort(versionId); - dStream.writeInt(correlationId); - dStream.writeInt(0); // avoiding adding clientId - dStream.writeShort((short) error.ordinal()); - return ByteBuffer.wrap(bStream.toByteArray()); - } - }.getPayload()); + response = makeDeleteResponse((DeleteRequest) send, serverError); + break; default: throw new IOException("Unknown request type received"); } + ByteBufferChannel channel = new ByteBufferChannel(ByteBuffer.allocate((int) response.sizeInBytes())); + response.writeTo(channel); + ByteBuffer payload = channel.getBuffer(); + payload.flip(); + // read the size off. the size is used by the client to allocate the buffer and then interpret the response. + // MockServer abstracts it away at a level above that and returns the "allocated buffer". + payload.getLong(); + return new MockBoundedByteBufferReceive(payload); + } + + PutResponse makePutResponse(PutRequest putRequest, ServerErrorCode putError) + throws IOException { + if (putError == ServerErrorCode.No_Error) { + updateBlobMap(putRequest); + } + return new PutResponse(putRequest.getCorrelationId(), putRequest.getClientId(), putError); + } + + GetResponse makeGetResponse(GetRequest getRequest, ServerErrorCode getError) + throws IOException { + GetResponse getResponse; + if (getError == ServerErrorCode.No_Error) { + List infos = getRequest.getPartitionInfoList(); + if (infos.size() != 1 || infos.get(0).getBlobIds().size() != 1) { + getError = ServerErrorCode.Unknown_Error; + } + } + + ServerErrorCode serverError; + ServerErrorCode partitionError; + // getError could be at the server level or the partition level. For partition level errors, + // handle set it in the partitionResponseInfo + if (getError == ServerErrorCode.No_Error || getError == ServerErrorCode.Blob_Expired + || getError == ServerErrorCode.Blob_Deleted || getError == ServerErrorCode.Blob_Not_Found) { + partitionError = getError; + serverError = ServerErrorCode.No_Error; + } else { + serverError = getError; + // does not matter. + partitionError = ServerErrorCode.No_Error; + } + + if (serverError == ServerErrorCode.No_Error) { + int byteBufferSize; + ByteBuffer byteBuffer; + StoreKey key = getRequest.getPartitionInfoList().get(0).getBlobIds().get(0); + if (blobs.containsKey(key.getID())) { + ByteBuffer buf = blobs.get(key.getID()).duplicate(); + buf.getLong(); // read off the size + buf.getShort(); // read off the type. + PutRequest originalBlobPutReq = + PutRequest.readFrom(new DataInputStream(new ByteBufferInputStream(buf)), clusterMap); + switch (getRequest.getMessageFormatFlag()) { + case BlobInfo: + BlobProperties blobProperties = originalBlobPutReq.getBlobProperties(); + ByteBuffer userMetadata = originalBlobPutReq.getUsermetadata(); + byteBufferSize = MessageFormatRecord.BlobProperties_Format_V1.getBlobPropertiesRecordSize(blobProperties) + + MessageFormatRecord.UserMetadata_Format_V1.getUserMetadataSize(userMetadata); + byteBuffer = ByteBuffer.allocate(byteBufferSize); + MessageFormatRecord.BlobProperties_Format_V1.serializeBlobPropertiesRecord(byteBuffer, blobProperties); + MessageFormatRecord.UserMetadata_Format_V1.serializeUserMetadataRecord(byteBuffer, userMetadata); + break; + case Blob: + byteBufferSize = + (int) MessageFormatRecord.Blob_Format_V2.getBlobRecordSize((int) originalBlobPutReq.getBlobSize()); + byteBuffer = ByteBuffer.allocate(byteBufferSize); + MessageFormatRecord.Blob_Format_V2 + .serializePartialBlobRecord(byteBuffer, (int) originalBlobPutReq.getBlobSize(), + originalBlobPutReq.getBlobType()); + byteBuffer.put( + Utils.readBytesFromStream(originalBlobPutReq.getBlobStream(), (int) originalBlobPutReq.getBlobSize())); + Crc32 crc = new Crc32(); + crc.update(byteBuffer.array(), 0, byteBuffer.position()); + byteBuffer.putLong(crc.getValue()); + break; + default: + throw new IOException("GetRequest flag is not supported: " + getRequest.getMessageFormatFlag()); + } + } else { + if (partitionError == ServerErrorCode.No_Error) { + partitionError = ServerErrorCode.Blob_Not_Found; + } + byteBuffer = ByteBuffer.allocate(0); + byteBufferSize = 0; + } + + byteBuffer.flip(); + ByteBufferSend responseSend = new ByteBufferSend(byteBuffer); + List messageInfoList = new ArrayList(1); + List partitionResponseInfoList = new ArrayList(); + messageInfoList.add(new MessageInfo(key, byteBufferSize)); + PartitionResponseInfo partitionResponseInfo = + partitionError == ServerErrorCode.No_Error ? new PartitionResponseInfo( + getRequest.getPartitionInfoList().get(0).getPartition(), messageInfoList) + : new PartitionResponseInfo(getRequest.getPartitionInfoList().get(0).getPartition(), partitionError); + partitionResponseInfoList.add(partitionResponseInfo); + getResponse = new GetResponse(getRequest.getCorrelationId(), getRequest.getClientId(), partitionResponseInfoList, + responseSend, serverError); + } else { + getResponse = new GetResponse(getRequest.getCorrelationId(), getRequest.getClientId(), serverError); + } + return getResponse; + } + + DeleteResponse makeDeleteResponse(DeleteRequest deleteRequest, ServerErrorCode deleteError) + throws IOException { + String blobIdString = deleteRequest.getBlobId().getID(); + if (deleteError == ServerErrorCode.No_Error) { + deleteError = getErrorFromBlobIdStr(blobIdString); + } + return new DeleteResponse(deleteRequest.getCorrelationId(), deleteRequest.getClientId(), deleteError); } /** @@ -108,6 +211,7 @@ ByteBuffer getPayload() * @param putRequest the PutRequest * @throws IOException if there was an error reading the contents of the given PutRequest. */ + private void updateBlobMap(PutRequest putRequest) throws IOException { String id = putRequest.getBlobId().getID(); @@ -126,34 +230,42 @@ Map getBlobs() { } /** - * Set the error for each request from this point onwards that affects subsequent PutRequests sent to this node + * Return the datacenter name of this server. + * @return the datacenter name. + */ + String getDataCenter() { + return dataCenter; + } + + /** + * Set the error for each request from this point onwards that affects subsequent requests sent to this node * (until/unless the next set or reset error method is invoked). * Each request from the list is used exactly once and in order. So, if the list contains {No_Error, Unknown_Error, * Disk_Error}, then the first, second and third requests would receive No_Error, * Unknown_Error and Disk_Error respectively. Once the errors are exhausted, the default No_Error is assumed for * all further requests until the next call to this method. - * @param putErrors the list of errors that affects subsequent PutRequests. + * @param serverErrors the list of errors that affects subsequent PutRequests. */ - public void setPutErrors(List putErrors) { - this.putErrors.clear(); - this.putErrors.addAll(putErrors); + public void setServerErrors(List serverErrors) { + this.serverErrors.clear(); + this.serverErrors.addAll(serverErrors); } /** * Set the error to be set in the responses for all requests from this point onwards (until/unless another set or * reset method for errors is invoked). - * @param putError the error to set from this point onwards. + * @param serverError the error to set from this point onwards. */ - public void setPutErrorForAllRequests(ServerErrorCode putError) { - this.hardError = putError; + public void setServerErrorForAllRequests(ServerErrorCode serverError) { + this.hardError = serverError; } /** - * Clear the error for subsequent PutRequests. That is all responses from this point onwards will be successful + * Clear the error for subsequent requests. That is all responses from this point onwards will be successful * ({@link ServerErrorCode#No_Error}) until/unless another set error method is invoked. */ - public void resetPutErrors() { - this.putErrors.clear(); + public void resetServerErrors() { + this.serverErrors.clear(); this.hardError = null; } @@ -161,7 +273,7 @@ public void resetPutErrors() { * Set whether or not the server would send response back. * @param shouldRespond {@code true} if the server responds. */ - public void setShouldRespond (boolean shouldRespond) { + public void setShouldRespond(boolean shouldRespond) { this.shouldRespond = shouldRespond; } diff --git a/ambry-router/src/test/java/com.github.ambry.router/MockServerLayout.java b/ambry-router/src/test/java/com.github.ambry.router/MockServerLayout.java index b6329df282..c1276a5164 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/MockServerLayout.java +++ b/ambry-router/src/test/java/com.github.ambry.router/MockServerLayout.java @@ -35,6 +35,9 @@ class MockServerLayout { public MockServerLayout(ClusterMap clusterMap) { this.mockServers = new HashMap(); this.clustermap = clusterMap; + for (DataNodeId dataNodeId : clusterMap.getDataNodeIds()) { + mockServers.put(dataNodeId, new MockServer(clusterMap, dataNodeId.getDatacenterName())); + } } /** @@ -45,9 +48,6 @@ public MockServerLayout(ClusterMap clusterMap) { */ public synchronized MockServer getMockServer(String host, int port) { DataNodeId dataNodeId = clustermap.getDataNodeId(host, port); - if (!mockServers.containsKey(dataNodeId)) { - mockServers.put(dataNodeId, new MockServer()); - } return mockServers.get(dataNodeId); } diff --git a/ambry-router/src/test/java/com.github.ambry.router/NonBlockingRouterTest.java b/ambry-router/src/test/java/com.github.ambry.router/NonBlockingRouterTest.java index 5ba9f182b2..f7410923d0 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/NonBlockingRouterTest.java +++ b/ambry-router/src/test/java/com.github.ambry.router/NonBlockingRouterTest.java @@ -163,7 +163,7 @@ private void assertExpectedThreadCounts(int expectedCount) { if (expectedCount == 0) { Assert.assertFalse("Router should be closed if there are no worker threads running", router.isOpen()); Assert - .assertEquals("All operations should have completed if the router is closed", 0, router.getOperationsCount()); + .assertEquals("All operations should have completed if the router is closed", 0, NonBlockingRouter.getOperationsCount()); } } diff --git a/ambry-router/src/test/java/com.github.ambry.router/PutManagerTest.java b/ambry-router/src/test/java/com.github.ambry.router/PutManagerTest.java index f835f027f9..527e41dd43 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/PutManagerTest.java +++ b/ambry-router/src/test/java/com.github.ambry.router/PutManagerTest.java @@ -187,7 +187,7 @@ public void testFailureOnAllSends() * Tests a failure scenario where selector poll throws an exception when there is anything to send. */ @Test - public void testFailureOnAllSend() + public void testFailureOnAllPollThatSends() throws Exception { requestAndResultsList.clear(); requestAndResultsList.add(new RequestAndResult(chunkSize * 5)); @@ -202,7 +202,7 @@ public void testFailureOnAllSend() TestUtils.numThreadsByThisName("ChunkFillerThread")); Assert.assertEquals("No RequestResponseHandler threads should be running after the router is closed", 0, TestUtils.numThreadsByThisName("RequestResponseHandlerThread")); - Assert.assertEquals("All operations should have completed", 0, router.getOperationsCount()); + Assert.assertEquals("All operations should have completed", 0, NonBlockingRouter.getOperationsCount()); } /** @@ -237,7 +237,7 @@ public void testPutWithAllNodesFailure() String host = dataNodeIds.get(i).getHostname(); int port = dataNodeIds.get(i).getPort(); MockServer server = mockServerLayout.getMockServer(host, port); - server.setPutErrorForAllRequests(ServerErrorCode.Unknown_Error); + server.setServerErrorForAllRequests(ServerErrorCode.Unknown_Error); } Exception expectedException = new RouterException("", RouterErrorCode.AmbryUnavailable); submitPutsAndAssertFailure(expectedException, true, false); @@ -264,7 +264,7 @@ public void testOneNodeFailurePutSuccess() String host = dataNodeIds.get(i).getHostname(); int port = dataNodeIds.get(i).getPort(); MockServer server = mockServerLayout.getMockServer(host, port); - server.setPutErrorForAllRequests(ServerErrorCode.Unknown_Error); + server.setServerErrorForAllRequests(ServerErrorCode.Unknown_Error); } } submitPutsAndAssertSuccess(true); @@ -286,7 +286,7 @@ public void testPutWithTwoNodesFailure() String host = dataNodeIds.get(i).getHostname(); int port = dataNodeIds.get(i).getPort(); MockServer server = mockServerLayout.getMockServer(host, port); - server.setPutErrorForAllRequests(ServerErrorCode.Unknown_Error); + server.setServerErrorForAllRequests(ServerErrorCode.Unknown_Error); } } Exception expectedException = new RouterException("", RouterErrorCode.AmbryUnavailable); @@ -315,7 +315,7 @@ public void testSlippedPutsSuccess() serverErrorList.add(ServerErrorCode.No_Error); for (DataNodeId dataNodeId : dataNodeIds) { MockServer server = mockServerLayout.getMockServer(dataNodeId.getHostname(), dataNodeId.getPort()); - server.setPutErrors(serverErrorList); + server.setServerErrors(serverErrorList); } submitPutsAndAssertSuccess(true); } @@ -341,7 +341,7 @@ public void testLaterChunkFailure() } for (DataNodeId dataNodeId : dataNodeIds) { MockServer server = mockServerLayout.getMockServer(dataNodeId.getHostname(), dataNodeId.getPort()); - server.setPutErrors(serverErrorList); + server.setServerErrors(serverErrorList); } Exception expectedException = new RouterException("", RouterErrorCode.AmbryUnavailable); submitPutsAndAssertFailure(expectedException, true, false); @@ -483,7 +483,7 @@ public void testRouterClosingOnChunkFillerThreadException() // Ensure that the existing operation was completed. requestAndResultsList.get(0).result.await(); - Assert.assertEquals("All operations should have completed", 0, router.getOperationsCount()); + Assert.assertEquals("All operations should have completed", 0, NonBlockingRouter.getOperationsCount()); Assert.assertTrue("Router should still be open", router.isOpen()); // Now submit another job and ensure that the router gets closed. @@ -501,7 +501,7 @@ public void testRouterClosingOnChunkFillerThreadException() TestUtils.numThreadsByThisName("ChunkFillerThread")); Assert.assertEquals("No RequestResponseHandler should be running after the router is closed", 0, TestUtils.numThreadsByThisName("RequestResponseHandlerThread")); - Assert.assertEquals("All operations should have completed", 0, router.getOperationsCount()); + Assert.assertEquals("All operations should have completed", 0, NonBlockingRouter.getOperationsCount()); } /** @@ -540,7 +540,7 @@ public void testRouterClosingOnRequestResponseHandlerThreadException() TestUtils.numThreadsByThisName("ChunkFillerThread")); Assert.assertEquals("No RequestResponseHandler should be running after the router is closed", 0, TestUtils.numThreadsByThisName("RequestResponseHandlerThread")); - Assert.assertEquals("All operations should have completed", 0, router.getOperationsCount()); + Assert.assertEquals("All operations should have completed", 0, NonBlockingRouter.getOperationsCount()); } // Methods used by the tests @@ -774,7 +774,7 @@ private void assertCloseCleanup() { TestUtils.numThreadsByThisName("ChunkFillerThread")); Assert.assertEquals("No RequestResponseHandler should be running after the router is closed", 0, TestUtils.numThreadsByThisName("RequestResponseHandlerThread")); - Assert.assertEquals("All operations should have completed", 0, router.getOperationsCount()); + Assert.assertEquals("All operations should have completed", 0, NonBlockingRouter.getOperationsCount()); } private class RequestAndResult { diff --git a/ambry-router/src/test/java/com.github.ambry.router/PutOperationTest.java b/ambry-router/src/test/java/com.github.ambry.router/PutOperationTest.java index f739e1dca4..ced97aa2aa 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/PutOperationTest.java +++ b/ambry-router/src/test/java/com.github.ambry.router/PutOperationTest.java @@ -42,13 +42,13 @@ public class PutOperationTest { private final RouterConfig routerConfig; - private final MockClusterMap mockClusterMap; + private final MockClusterMap mockClusterMap = new MockClusterMap(); private final ResponseHandler responseHandler; private final Time time; private final Map correlationIdToPutOperation = new TreeMap<>(); - private final MockServer mockServer = new MockServer(); + private final MockServer mockServer = new MockServer(mockClusterMap, ""); - private class PutTestRequestRegistrationCallbackImpl implements PutRequestRegistrationCallback { + private class PutTestRequestRegistrationCallbackImpl implements RequestRegistrationCallback { private List requestListToFill; @Override @@ -56,9 +56,7 @@ public void registerRequestToSend(PutOperation putOperation, RequestInfo request requestListToFill.add(requestInfo); correlationIdToPutOperation.put(((RequestOrResponse) requestInfo.getRequest()).getCorrelationId(), putOperation); } - } - - ; + }; private final PutTestRequestRegistrationCallbackImpl requestRegistrationCallback = new PutTestRequestRegistrationCallbackImpl(); @@ -77,7 +75,6 @@ public PutOperationTest() properties.setProperty("router.put.success.target", Integer.toString(successTarget)); VerifiableProperties vProps = new VerifiableProperties(properties); routerConfig = new RouterConfig(vProps); - mockClusterMap = new MockClusterMap(); responseHandler = new ResponseHandler(mockClusterMap); time = new MockTime(); } diff --git a/build.gradle b/build.gradle index 1db10aa2db..5c8b300c40 100644 --- a/build.gradle +++ b/build.gradle @@ -154,6 +154,7 @@ project(':ambry-coordinator') { project(':ambry-protocol'), project(':ambry-network') testCompile project(':ambry-clustermap').sourceSets.test.output + testCompile project(':ambry-network').sourceSets.test.output } } From eeca9a92714356cc8af273b7575aa039475d5f9a Mon Sep 17 00:00:00 2001 From: Priyesh Narayanan Date: Wed, 4 May 2016 14:18:39 -0700 Subject: [PATCH 2/6] Address review comments and fix merge conflicts --- .../com.github.ambry/config/RouterConfig.java | 8 + .../messageformat/BlobProperties.java | 15 -- .../PartitionRequestInfo.java | 4 +- .../DeleteManager.java | 4 +- .../DeleteOperation.java | 8 +- .../GetBlobInfoOperation.java | 159 +++++++++-------- .../com.github.ambry.router/GetManager.java | 19 +- .../com.github.ambry.router/GetOperation.java | 51 +++--- .../NonBlockingRouter.java | 17 +- .../com.github.ambry.router/PutManager.java | 9 +- .../com.github.ambry.router/PutOperation.java | 14 +- .../GetManagerTest.java | 26 +-- .../GetOperationTest.java | 168 +++++++----------- .../com.github.ambry.router/MockServer.java | 11 +- .../PutOperationTest.java | 23 +-- .../RouterTestHelpers.java | 37 ++++ 16 files changed, 289 insertions(+), 284 deletions(-) create mode 100644 ambry-router/src/test/java/com.github.ambry.router/RouterTestHelpers.java diff --git a/ambry-api/src/main/java/com.github.ambry/config/RouterConfig.java b/ambry-api/src/main/java/com.github.ambry/config/RouterConfig.java index 0348c94774..a1f83076e4 100644 --- a/ambry-api/src/main/java/com.github.ambry/config/RouterConfig.java +++ b/ambry-api/src/main/java/com.github.ambry/config/RouterConfig.java @@ -124,6 +124,13 @@ public class RouterConfig { @Default("1") public final int routerGetSuccessTarget; + /** + * Indicates whether get operations are allowed to make requests to nodes in remote data centers. + */ + @Config("router.get.cross.dc.enabled") + @Default("true") + public final boolean routerGetCrossDcEnabled; + /** * Create a RouterConfig instance. * @param verifiableProperties the properties map to refer to. @@ -147,5 +154,6 @@ public RouterConfig(VerifiableProperties verifiableProperties) { routerDeleteSuccessTarget = verifiableProperties.getInt("router.delete.success.target", 2); routerGetRequestParallelism = verifiableProperties.getInt("router.get.request.parallelism", 2); routerGetSuccessTarget = verifiableProperties.getInt("router.get.success.target", 1); + routerGetCrossDcEnabled = verifiableProperties.getBoolean("router.get.cross.dc.enabled", true); } } diff --git a/ambry-api/src/main/java/com.github.ambry/messageformat/BlobProperties.java b/ambry-api/src/main/java/com.github.ambry/messageformat/BlobProperties.java index 74499f9d7e..79da3841a4 100644 --- a/ambry-api/src/main/java/com.github.ambry/messageformat/BlobProperties.java +++ b/ambry-api/src/main/java/com.github.ambry/messageformat/BlobProperties.java @@ -131,19 +131,4 @@ public String toString() { sb.append("]"); return sb.toString(); } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null) { - return false; - } - - BlobProperties that = (BlobProperties) o; - return blobSize == that.blobSize && serviceId.equals(that.serviceId) && ownerId.equals(that.ownerId) && contentType - .equals(that.contentType) && isPrivate == that.isPrivate && timeToLiveInSeconds == that.timeToLiveInSeconds; - } } diff --git a/ambry-protocol/src/main/java/com.github.ambry.protocol/PartitionRequestInfo.java b/ambry-protocol/src/main/java/com.github.ambry.protocol/PartitionRequestInfo.java index b1dd75bcf3..bface0e4be 100644 --- a/ambry-protocol/src/main/java/com.github.ambry.protocol/PartitionRequestInfo.java +++ b/ambry-protocol/src/main/java/com.github.ambry.protocol/PartitionRequestInfo.java @@ -31,12 +31,12 @@ public class PartitionRequestInfo { private final PartitionId partitionId; - private final ArrayList blobIds; + private final List blobIds; private long totalIdSize; private static final int Blob_Id_Count_Size_InBytes = 4; - public PartitionRequestInfo(PartitionId partitionId, ArrayList blobIds) { + public PartitionRequestInfo(PartitionId partitionId, List blobIds) { this.partitionId = partitionId; this.blobIds = blobIds; totalIdSize = 0; diff --git a/ambry-router/src/main/java/com.github.ambry.router/DeleteManager.java b/ambry-router/src/main/java/com.github.ambry.router/DeleteManager.java index 701d4a2ce0..8a984ff64c 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/DeleteManager.java +++ b/ambry-router/src/main/java/com.github.ambry.router/DeleteManager.java @@ -54,7 +54,7 @@ class DeleteManager { /** * Used by a {@link DeleteOperation} to associate a {@code CorrelationId} to a {@link DeleteOperation}. */ - private class DeleteRequestRegistrationCallbackImpl implements DeleteRequestRegistrationCallback { + private class DeleteRequestRegistrationCallbackImpl implements RequestRegistrationCallback { private List requestListToFill; @Override @@ -163,9 +163,9 @@ void close() { Iterator iter = deleteOperations.iterator(); while (iter.hasNext()) { DeleteOperation deleteOperation = iter.next(); + iter.remove(); NonBlockingRouter.completeOperation(deleteOperation.getFutureResult(), deleteOperation.getCallback(), null, new RouterException("Aborted operation because Router is closed.", RouterErrorCode.RouterClosed)); - iter.remove(); } } } diff --git a/ambry-router/src/main/java/com.github.ambry.router/DeleteOperation.java b/ambry-router/src/main/java/com.github.ambry.router/DeleteOperation.java index 34cc6fad65..ee510e8d39 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/DeleteOperation.java +++ b/ambry-router/src/main/java/com.github.ambry.router/DeleteOperation.java @@ -95,10 +95,10 @@ class DeleteOperation { /** * Gets a list of {@link DeleteRequest} for sending to replicas. - * @param requestFillCallback the {@link DeleteRequestRegistrationCallback} to call for every request + * @param requestFillCallback the {@link RequestRegistrationCallback} to call for every request * that gets created as part of this poll operation. */ - void poll(DeleteRequestRegistrationCallback requestFillCallback) { + void poll(RequestRegistrationCallback requestFillCallback) { cleanupExpiredInflightRequests(); checkAndMaybeComplete(); if (isOperationComplete()) { @@ -178,8 +178,8 @@ void handleResponse(ResponseInfo responseInfo) { * A wrapper class that is used to check if a request has been expired. */ private class InflightRequestInfo { - private final long submissionTime; - private final ReplicaId replica; + final long submissionTime; + final ReplicaId replica; InflightRequestInfo(long submissionTime, ReplicaId replica) { this.submissionTime = submissionTime; diff --git a/ambry-router/src/main/java/com.github.ambry.router/GetBlobInfoOperation.java b/ambry-router/src/main/java/com.github.ambry.router/GetBlobInfoOperation.java index f6b4dac77d..43811d091c 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/GetBlobInfoOperation.java +++ b/ambry-router/src/main/java/com.github.ambry.router/GetBlobInfoOperation.java @@ -36,6 +36,8 @@ import java.util.Iterator; import java.util.Map; import java.util.TreeMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -44,10 +46,11 @@ * which is either the only chunk in the case of a simple blob, or the metadata chunk in the case of composite blobs. */ class GetBlobInfoOperation extends GetOperation { - private BlobInfo blobInfo; private final SimpleOperationTracker operationTracker; // map of correlation id to the request metadata for every request issued for this operation. - protected final Map correlationIdToGetRequestInfo = new TreeMap(); + private final Map correlationIdToGetRequestInfo = new TreeMap(); + + private static final Logger logger = LoggerFactory.getLogger(GetBlobInfoOperation.class); /** * Construct a GetBlobInfoOperation @@ -64,8 +67,9 @@ class GetBlobInfoOperation extends GetOperation { String blobIdStr, FutureResult futureResult, Callback callback, Time time) throws RouterException { super(routerConfig, clusterMap, responseHandler, blobIdStr, futureResult, callback, time); - operationTracker = new SimpleOperationTracker(routerConfig.routerDatacenterName, blobId.getPartition(), true, - routerConfig.routerGetSuccessTarget, routerConfig.routerGetRequestParallelism); + operationTracker = new SimpleOperationTracker(routerConfig.routerDatacenterName, blobId.getPartition(), + routerConfig.routerGetCrossDcEnabled, routerConfig.routerGetSuccessTarget, + routerConfig.routerGetRequestParallelism); } /** @@ -78,13 +82,9 @@ MessageFormatFlags getOperationFlag() { } /** - * For this operation, create and populate get requests (in the form of {@link RequestInfo}) to send out. - * @param requestRegistrationCallback the {@link RequestRegistrationCallback} to call for every request that gets - * created as part of this poll operation. + * Clean up requests sent out by this operation that have now timed out. */ - @Override - void poll(RequestRegistrationCallback requestRegistrationCallback) { - //First, check if any of the existing requests have timed out. + void cleanupExpiredInFlightRequests() { Iterator> inFlightRequestsIterator = correlationIdToGetRequestInfo.entrySet().iterator(); while (inFlightRequestsIterator.hasNext()) { @@ -99,6 +99,17 @@ void poll(RequestRegistrationCallback requestRegistrationCallback) break; } } + } + + /** + * For this operation, create and populate get requests (in the form of {@link RequestInfo}) to send out. + * @param requestRegistrationCallback the {@link RequestRegistrationCallback} to call for every request that gets + * created as part of this poll operation. + */ + @Override + void poll(RequestRegistrationCallback requestRegistrationCallback) { + //First, check if any of the existing requests have timed out. + cleanupExpiredInFlightRequests(); checkAndMaybeComplete(); if (isOperationComplete()) { @@ -119,25 +130,6 @@ void poll(RequestRegistrationCallback requestRegistrationCallback) } } - /** - * Handle the body of the response: Deserialize and set the {@link BlobInfo} to return. - * @param payload the body of the response. - * @throws IOException if there is an IOException while deserializing the body. - * @throws MessageFormatException if there is a MessageFormatException while deserializing the body. - */ - void handleBody(InputStream payload) - throws IOException, MessageFormatException { - if (blobInfo == null) { - blobInfo = new BlobInfo(MessageFormatRecord.deserializeBlobProperties(payload), - MessageFormatRecord.deserializeUserMetadata(payload).array()); - } else { - // If the successTarget is 1, this case will never get executed. - // If it is more than 1, then, different responses will have to be reconciled in some way. Here is where that - // would be done. Since the store is immutable, currently we handle this by ignoring subsequent responses - // (if the successTarget is > 1). - } - } - /** * Handle the given {@link ResponseInfo} and update the status of the operation. * Based on the status of the response, this involves handling the body, notifying the operation tracker, @@ -169,60 +161,86 @@ void handleResponse(ResponseInfo responseInfo) { // out over a connection id, and the response received on a connection id must be for the latest request // sent over it. The check here ensures that is indeed the case. If not, log an error and fail this request. // There is no other way to handle it. - logger.error("The correlation id in the GetResponse " + getResponse.getCorrelationId() - + " is not the same as the correlation id in the associated GetRequest: " + correlationId); setOperationException( - new RouterException("Unexpected internal error", RouterErrorCode.UnexpectedInternalError)); + new RouterException("The correlation id in the GetResponse " + getResponse.getCorrelationId() + + "is not the same as the correlation id in the associated GetRequest: " + correlationId, + RouterErrorCode.UnexpectedInternalError)); operationTracker.onResponse(getRequestInfo.replicaId, false); // we do not notify the ResponseHandler responsible for failure detection as this is an unexpected error. } else { - ServerErrorCode getError = getResponse.getError(); - if (getError == ServerErrorCode.No_Error) { - int partitionsInResponse = getResponse.getPartitionResponseInfoList().size(); - // Each get request issued by the router is for a single blob. - if (partitionsInResponse != 1) { - setOperationException(new RouterException("Unexpected number of partition responses, expected: 1, " + - "received: " + partitionsInResponse, RouterErrorCode.UnexpectedInternalError)); - operationTracker.onResponse(getRequestInfo.replicaId, false); - // Again, no need to notify the responseHandler. - } else { - getError = getResponse.getPartitionResponseInfoList().get(0).getErrorCode(); - responseHandler.onRequestResponseError(getRequestInfo.replicaId, getError); - if (getError == ServerErrorCode.No_Error) { - handleBody(getResponse.getInputStream()); - operationTracker.onResponse(getRequestInfo.replicaId, true); - } else { - // process and set the most relevant exception. - processServerError(getError); - if (getError == ServerErrorCode.Blob_Deleted || getError == ServerErrorCode.Blob_Expired) { - // this is a successful response and one that completes the operation regardless of whether the - // success target has been reached or not. - operationCompleted = true; - } else { - operationTracker.onResponse(getRequestInfo.replicaId, false); - } - } - } - } else { - responseHandler.onRequestResponseError(getRequestInfo.replicaId, getError); - operationTracker.onResponse(getRequestInfo.replicaId, false); - } + processGetResponse(getRequestInfo, getResponse); } - } catch (IOException e) { + } catch (IOException | MessageFormatException e) { // This should really not happen. Again, we do not notify the ResponseHandler responsible for failure // detection. setOperationException(new RouterException("Response deserialization received an unexpected error", e, RouterErrorCode.UnexpectedInternalError)); operationTracker.onResponse(getRequestInfo.replicaId, false); - } catch (MessageFormatException e) { - setOperationException(new RouterException("Response deserialization received an unexpected error", e, - RouterErrorCode.UnexpectedInternalError)); - operationTracker.onResponse(getRequestInfo.replicaId, false); } } checkAndMaybeComplete(); } + /** + * Process the actual GetResponse extracted from a {@link ResponseInfo} + * @param getRequestInfo the associated {@link RequestInfo} for which this response was received. + * @param getResponse the {@link GetResponse} extracted the {@link ResponseInfo} + * @throws IOException if there is an error during deserialization of the GetResponse. + * @throws MessageFormatException if there is an error during deserialization of the GetResponse. + */ + void processGetResponse(GetRequestInfo getRequestInfo, GetResponse getResponse) + throws IOException, MessageFormatException { + ServerErrorCode getError = getResponse.getError(); + if (getError == ServerErrorCode.No_Error) { + int partitionsInResponse = getResponse.getPartitionResponseInfoList().size(); + // Each get request issued by the router is for a single blob. + if (partitionsInResponse != 1) { + setOperationException(new RouterException("Unexpected number of partition responses, expected: 1, " + + "received: " + partitionsInResponse, RouterErrorCode.UnexpectedInternalError)); + operationTracker.onResponse(getRequestInfo.replicaId, false); + // Again, no need to notify the responseHandler. + } else { + getError = getResponse.getPartitionResponseInfoList().get(0).getErrorCode(); + responseHandler.onRequestResponseError(getRequestInfo.replicaId, getError); + if (getError == ServerErrorCode.No_Error) { + handleBody(getResponse.getInputStream()); + operationTracker.onResponse(getRequestInfo.replicaId, true); + } else { + // process and set the most relevant exception. + processServerError(getError); + if (getError == ServerErrorCode.Blob_Deleted || getError == ServerErrorCode.Blob_Expired) { + // this is a successful response and one that completes the operation regardless of whether the + // success target has been reached or not. + operationCompleted = true; + } else { + operationTracker.onResponse(getRequestInfo.replicaId, false); + } + } + } + } else { + responseHandler.onRequestResponseError(getRequestInfo.replicaId, getError); + operationTracker.onResponse(getRequestInfo.replicaId, false); + } + } + + /** + * Handle the body of the response: Deserialize and set the {@link BlobInfo} to return. + * @param payload the body of the response. + * @throws IOException if there is an IOException while deserializing the body. + * @throws MessageFormatException if there is a MessageFormatException while deserializing the body. + */ + private void handleBody(InputStream payload) + throws IOException, MessageFormatException { + if (operationResult == null) { + operationResult = new BlobInfo(MessageFormatRecord.deserializeBlobProperties(payload), + MessageFormatRecord.deserializeUserMetadata(payload).array()); + } else { + // If the successTarget is 1, this case will never get executed. + // If it is more than 1, then, different responses will have to be reconciled in some way. Here is where that + // would be done. Since the store is immutable, currently we handle this by ignoring subsequent responses. + } + } + /** * Check whether the operation can be completed, if so complete it. */ @@ -233,11 +251,6 @@ private void checkAndMaybeComplete() { } operationCompleted = true; } - // this check is outside as the operation could get completed outside of the operation tracking if a - // Blob_Deleted error or Blob_Expired error was received (in which case the success target is not honored). - if (operationCompleted) { - NonBlockingRouter.completeOperation(operationFuture, operationCallback, blobInfo, operationException); - } } } diff --git a/ambry-router/src/main/java/com.github.ambry.router/GetManager.java b/ambry-router/src/main/java/com.github.ambry.router/GetManager.java index 93950df286..937a762352 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/GetManager.java +++ b/ambry-router/src/main/java/com.github.ambry.router/GetManager.java @@ -29,7 +29,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,7 +47,6 @@ class GetManager { // Because there is a guaranteed response from the NetworkClient for every request sent out, entries // get cleaned up periodically. private final Map correlationIdToGetOperation = new HashMap(); - private final AtomicBoolean isOpen = new AtomicBoolean(true); // shared by all GetOperations private final ClusterMap clusterMap; @@ -64,7 +62,7 @@ public void registerRequestToSend(GetOperation getOperation, RequestInfo request requestListToFill.add(requestInfo); correlationIdToGetOperation.put(((RequestOrResponse) requestInfo.getRequest()).getCorrelationId(), getOperation); } - }; + } // A single callback as this will never get called concurrently. The list of request to fill will be set as // appropriate before the callback is passed on to GetOperations, every time. private final GetRequestRegistrationCallbackImpl requestRegistrationCallback = @@ -131,10 +129,22 @@ void poll(List requestListToFill) { op.poll(requestRegistrationCallback); if (op.isOperationComplete()) { getOperationIterator.remove(); + onComplete(op); } } } + /** + * Called for a {@link GetOperation} when the operation is complete. Any cleanup that the GetManager needs to do + * with respect to this operation will have to be done here. The GetManager also finishes the operation by + * performing the callback and notification. + * @param op the {@link PutOperation} that has completed. + */ + void onComplete(GetOperation op) { + NonBlockingRouter + .completeOperation(op.getFuture(), op.getCallback(), op.getOperationResult(), op.getOperationException()); + } + /** * Hands over the response to the associated GetOperation that issued the request. * @param responseInfo the {@link ResponseInfo} containing the response. @@ -146,6 +156,7 @@ void handleResponse(ResponseInfo responseInfo) { getOperation.handleResponse(responseInfo); if (getOperation.isOperationComplete()) { getOperations.remove(getOperation); + onComplete(getOperation); } } } @@ -158,10 +169,10 @@ void close() { Iterator iter = getOperations.iterator(); while (iter.hasNext()) { GetOperation op = iter.next(); + iter.remove(); logger.trace("Aborting operation for blob id: ", op.getBlobIdStr()); NonBlockingRouter.completeOperation(op.getFuture(), op.getCallback(), null, new RouterException("Aborted operation because Router is closed", RouterErrorCode.RouterClosed)); - iter.remove(); } } } diff --git a/ambry-router/src/main/java/com.github.ambry.router/GetOperation.java b/ambry-router/src/main/java/com.github.ambry.router/GetOperation.java index 2e9ac6e804..48d3a9410e 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/GetOperation.java +++ b/ambry-router/src/main/java/com.github.ambry.router/GetOperation.java @@ -26,6 +26,7 @@ import com.github.ambry.protocol.PartitionRequestInfo; import com.github.ambry.utils.Time; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,17 +37,18 @@ * @param the type of the result of this operation. */ abstract class GetOperation { - final RouterConfig routerConfig; - final ClusterMap clusterMap; - final ResponseHandler responseHandler; - final FutureResult operationFuture; - final Callback operationCallback; - final BlobId blobId; - final Time time; - boolean operationCompleted = false; - Exception operationException; - - static final Logger logger = LoggerFactory.getLogger(GetOperation.class); + protected final RouterConfig routerConfig; + protected final ClusterMap clusterMap; + protected final ResponseHandler responseHandler; + protected final FutureResult operationFuture; + protected final Callback operationCallback; + protected final BlobId blobId; + protected final Time time; + protected boolean operationCompleted = false; + protected Exception operationException; + protected T operationResult; + + private static final Logger logger = LoggerFactory.getLogger(GetOperation.class); /** * Construct a GetOperation @@ -95,6 +97,14 @@ Exception getOperationException() { return operationException; } + /** + * Return the result of the operation. + * @return the operation result. + */ + T getOperationResult() { + return operationResult; + } + /** * Return the blob id string * @return return the blob id string @@ -149,15 +159,17 @@ void processServerError(ServerErrorCode errorCode) { setOperationException(new RouterException("Server returned: " + errorCode, RouterErrorCode.BlobDoesNotExist)); break; default: - setOperationException(new RouterException("Server returned: " + errorCode, RouterErrorCode.AmbryUnavailable)); + setOperationException( + new RouterException("Server returned: " + errorCode, RouterErrorCode.UnexpectedInternalError)); break; } } /** * Set the exception associated with this operation. - * A Blob_Deleted or Blob_Expired error overrides any other error previously received exception. - * @param exception the RouterException to possibly set. + * A {@link ServerErrorCode#Blob_Deleted} or {@link ServerErrorCode#Blob_Expired} error overrides any other + * previously received exception. + * @param exception the {@link RouterException} to possibly set. */ void setOperationException(RouterException exception) { if (operationException == null || exception.getErrorCode() == RouterErrorCode.BlobDeleted @@ -168,14 +180,13 @@ void setOperationException(RouterException exception) { /** * Create and return the {@link GetRequest} associated with the given blobId. - * @return the crated {@link GetRequest}. - * @param blobId The {@link BlobId} for which the GetRequest is being created. + * @return the created {@link GetRequest}. + * @param blobId The {@link BlobId} for which the {@link GetRequest} is being created. * @param flag The {@link MessageFormatFlags} to be set with the GetRequest. * @return the created GetRequest. */ protected GetRequest createGetRequest(BlobId blobId, MessageFormatFlags flag) { - ArrayList blobIds = new ArrayList(1); - blobIds.add(blobId); + List blobIds = Collections.singletonList(blobId); List partitionRequestInfoList = new ArrayList(); PartitionRequestInfo partitionRequestInfo = new PartitionRequestInfo(blobId.getPartition(), blobIds); partitionRequestInfoList.add(partitionRequestInfo); @@ -188,8 +199,8 @@ protected GetRequest createGetRequest(BlobId blobId, MessageFormatFlags flag) { * A class that holds information about the get requests sent out. */ class GetRequestInfo { - ReplicaId replicaId; - long startTimeMs; + final ReplicaId replicaId; + final long startTimeMs; /** * Construct a GetRequestInfo diff --git a/ambry-router/src/main/java/com.github.ambry.router/NonBlockingRouter.java b/ambry-router/src/main/java/com.github.ambry.router/NonBlockingRouter.java index 7718a3d571..a46b43954c 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/NonBlockingRouter.java +++ b/ambry-router/src/main/java/com.github.ambry.router/NonBlockingRouter.java @@ -282,20 +282,6 @@ static int getOperationsCount() { return currentOperationsCount.get(); } - /** - * Increment the count of currently running operations - */ - static void incrementOperationsCount() { - currentOperationsCount.incrementAndGet(); - } - - /** - * Decrement the count of currently running operations - */ - static void decrementOperationsCount() { - currentOperationsCount.decrementAndGet(); - } - /** * Completes a router operation by invoking the {@code callback} and setting the {@code futureResult} with * {@code operationResult} (if any) and {@code exception} (if any). @@ -307,6 +293,7 @@ static void decrementOperationsCount() { */ static void completeOperation(FutureResult futureResult, Callback callback, T operationResult, Exception exception) { + currentOperationsCount.decrementAndGet(); try { futureResult.done(operationResult, exception); if (callback != null) { @@ -314,8 +301,6 @@ static void completeOperation(FutureResult futureResult, Callback call } } catch (Exception e) { logger.error("Exception caught during future and callback completion", e); - } finally { - currentOperationsCount.decrementAndGet(); } } diff --git a/ambry-router/src/main/java/com.github.ambry.router/PutManager.java b/ambry-router/src/main/java/com.github.ambry.router/PutManager.java index fca7c915a2..b25211e8a2 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/PutManager.java +++ b/ambry-router/src/main/java/com.github.ambry.router/PutManager.java @@ -66,10 +66,9 @@ private class PutRequestRegistrationCallbackImpl implements RequestRegistrationC @Override public void registerRequestToSend(PutOperation putOperation, RequestInfo requestInfo) { requestListToFill.add(requestInfo); - correlationIdToPutOperation - .put(((RequestOrResponse) requestInfo.getRequest()).getCorrelationId(), putOperation); + correlationIdToPutOperation.put(((RequestOrResponse) requestInfo.getRequest()).getCorrelationId(), putOperation); } - }; + } // A single callback as this will never get called concurrently. The list of request to fill will be set as // appropriate before the callback is passed on to the PutOperations, every time. private final PutRequestRegistrationCallbackImpl requestRegistrationCallback = @@ -167,7 +166,7 @@ void getIdsToDelete(List idsToDelete) { } /** - * Called by a {@link PutOperation} when the operation is complete. Any cleanup that the PutManager needs to do + * Called for a {@link PutOperation} when the operation is complete. Any cleanup that the PutManager needs to do * with respect to this operation will have to be done here. The PutManager also finishes the operation by * performing the callback and notification. * @param op the {@link PutOperation} that has completed. @@ -215,9 +214,9 @@ void completePendingOperations() { Iterator iter = putOperations.iterator(); while (iter.hasNext()) { PutOperation op = iter.next(); + iter.remove(); NonBlockingRouter.completeOperation(op.getFuture(), op.getCallback(), null, new RouterException("Aborted operation because Router is closed", RouterErrorCode.RouterClosed)); - iter.remove(); } } diff --git a/ambry-router/src/main/java/com.github.ambry.router/PutOperation.java b/ambry-router/src/main/java/com.github.ambry.router/PutOperation.java index 5adbbc8520..be60384cc8 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/PutOperation.java +++ b/ambry-router/src/main/java/com.github.ambry.router/PutOperation.java @@ -197,7 +197,7 @@ boolean isOperationComplete() { * @param requestRegistrationCallback the {@link RequestRegistrationCallback} to call for every request that gets * created as part of this poll operation. */ - void poll(RequestRegistrationCallback requestRegistrationCallback) { + void poll(RequestRegistrationCallback requestRegistrationCallback) { if (operationCompleted) { return; } @@ -664,7 +664,7 @@ void checkAndMaybeComplete() { * @param requestFillCallback the {@link RequestRegistrationCallback} to call for every request that gets created as * part of this poll operation. */ - void poll(RequestRegistrationCallback requestFillCallback) { + void poll(RequestRegistrationCallback requestFillCallback) { maybeFreeDefunctBuffers(); //First, check if any of the existing requests have timed out. Iterator> inFlightRequestsIterator = @@ -822,9 +822,9 @@ private void processServerError(ServerErrorCode error) { * A class that holds information about requests sent out by this PutChunk. */ private class ChunkPutRequestInfo { - private final ReplicaId replicaId; - private final PutRequest putRequest; - private final long startTimeMs; + final ReplicaId replicaId; + final PutRequest putRequest; + final long startTimeMs; /** * Construct a ChunkPutRequestInfo @@ -844,9 +844,9 @@ private class ChunkPutRequestInfo { */ private class DefunctBufferInfo { // the buffer that is now defunct, but not yet freed. - ByteBuffer buf; + final ByteBuffer buf; // Requests that are reading from this buffer. - List putRequests; + final List putRequests; /** * Construct a DefunctBufferInfo diff --git a/ambry-router/src/test/java/com.github.ambry.router/GetManagerTest.java b/ambry-router/src/test/java/com.github.ambry.router/GetManagerTest.java index b2f45fc3c1..35e8b16027 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/GetManagerTest.java +++ b/ambry-router/src/test/java/com.github.ambry.router/GetManagerTest.java @@ -13,7 +13,6 @@ */ package com.github.ambry.router; -import com.codahale.metrics.MetricRegistry; import com.github.ambry.clustermap.MockClusterMap; import com.github.ambry.commons.ByteBufferReadableStreamChannel; import com.github.ambry.commons.LoggingNotificationSystem; @@ -38,20 +37,18 @@ public class GetManagerTest { private final MockServerLayout mockServerLayout; private final MockTime mockTime = new MockTime(); private final MockClusterMap mockClusterMap; + private final Random random = new Random(); // this is a reference to the state used by the mockSelector. just allows tests to manipulate the state. - private AtomicReference mockSelectorState = new AtomicReference(); + private final AtomicReference mockSelectorState = new AtomicReference(); private NonBlockingRouter router; - private int chunkSize; private int requestParallelism; private int successTarget; - private final Random random = new Random(); - // Request params; - BlobProperties putBlobProperties; - byte[] putUserMetadata; - byte[] putContent; - ReadableStreamChannel putChannel; + private BlobProperties putBlobProperties; + private byte[] putUserMetadata; + private byte[] putContent; + private ReadableStreamChannel putChannel; private static final int MAX_PORTS_PLAIN_TEXT = 3; private static final int MAX_PORTS_SSL = 3; @@ -93,7 +90,8 @@ public void testSimpleBlobGetBlobInfoSuccess() setOperationParams(chunkSize); String blobId = router.putBlob(putBlobProperties, putUserMetadata, putChannel).get(); BlobInfo blobInfo = router.getBlobInfo(blobId).get(); - Assert.assertEquals("Blob properties should match", putBlobProperties, blobInfo.getBlobProperties()); + Assert.assertTrue("Blob properties should match", + RouterTestHelpers.haveEquivalentFields(putBlobProperties, blobInfo.getBlobProperties())); Assert.assertArrayEquals("User metadata should match", putUserMetadata, blobInfo.getUserMetadata()); router.close(); } @@ -109,7 +107,8 @@ public void testCompositeBlobGetBlobInfoSuccess() setOperationParams(chunkSize * 6 + 11); String blobId = router.putBlob(putBlobProperties, putUserMetadata, putChannel).get(); BlobInfo blobInfo = router.getBlobInfo(blobId).get(); - Assert.assertEquals("Blob properties should match", putBlobProperties, blobInfo.getBlobProperties()); + Assert.assertTrue("Blob properties should match", + RouterTestHelpers.haveEquivalentFields(putBlobProperties, blobInfo.getBlobProperties())); Assert.assertArrayEquals("User metadata should match", putUserMetadata, blobInfo.getUserMetadata()); router.close(); } @@ -125,8 +124,8 @@ public void testFailureOnAllPollThatSends() router = getNonBlockingRouter(); setOperationParams(chunkSize); String blobId = router.putBlob(putBlobProperties, putUserMetadata, putChannel).get(); + mockSelectorState.set(MockSelectorState.ThrowExceptionOnSend); try { - mockSelectorState.set(MockSelectorState.ThrowExceptionOnSend); router.getBlobInfo(blobId).get(); Assert.fail("operation should have thrown"); } catch (ExecutionException e) { @@ -148,7 +147,8 @@ private NonBlockingRouter getNonBlockingRouter() properties.setProperty("router.put.request.parallelism", Integer.toString(requestParallelism)); properties.setProperty("router.put.success.target", Integer.toString(successTarget)); VerifiableProperties vProps = new VerifiableProperties(properties); - router = new NonBlockingRouter(new RouterConfig(vProps), new NonBlockingRouterMetrics(new MetricRegistry()), + router = new NonBlockingRouter(new RouterConfig(vProps), + new NonBlockingRouterMetrics(mockClusterMap.getMetricRegistry()), new MockNetworkClientFactory(vProps, mockSelectorState, MAX_PORTS_PLAIN_TEXT, MAX_PORTS_SSL, CHECKOUT_TIMEOUT_MS, mockServerLayout, mockTime), new LoggingNotificationSystem(), mockClusterMap, mockTime); diff --git a/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java b/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java index e22f942d8c..eb9f9a8fbe 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java +++ b/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java @@ -42,7 +42,6 @@ import java.util.concurrent.atomic.AtomicReference; import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; @@ -50,6 +49,8 @@ public class GetOperationTest { private static final int MAX_PORTS_PLAIN_TEXT = 3; private static final int MAX_PORTS_SSL = 3; private static final int CHECKOUT_TIMEOUT_MS = 1000; + private static final int BLOB_SIZE = 100; + private static final int BLOB_USER_METADATA_SIZE = 10; private int requestParallelism = 2; private int successTarget = 1; @@ -61,11 +62,9 @@ public class GetOperationTest { private final ResponseHandler responseHandler; private final MockNetworkClientFactory networkClientFactory; private final NetworkClient networkClient; - private final MockTime time; + private final MockTime time = new MockTime(); private final Map correlationIdToGetOperation = new HashMap<>(); private final NonBlockingRouter router; - private final int BLOB_SIZE = 100; - private final int BLOB_USER_METADATA_SIZE = 10; private final Random random = new Random(); private final String blobIdStr; private final BlobProperties blobProperties; @@ -74,15 +73,6 @@ public class GetOperationTest { private final GetTestRequestRegistrationCallbackImpl requestRegistrationCallback = new GetTestRequestRegistrationCallbackImpl(); private final FutureResult operationFuture = new FutureResult<>(); - private final AtomicReference operationResult = new AtomicReference<>(null); - private final AtomicReference operationException = new AtomicReference<>(null); - private final Callback operationCallback = new Callback() { - @Override - public void onCompletion(BlobInfo result, Exception exception) { - operationResult.set(result); - operationException.set(exception); - } - }; private class GetTestRequestRegistrationCallbackImpl implements RequestRegistrationCallback { private List requestListToFill; @@ -104,7 +94,6 @@ public GetOperationTest() mockServerLayout = new MockServerLayout(mockClusterMap); replicasCount = mockClusterMap.getWritablePartitionIds().get(0).getReplicaIds().size(); responseHandler = new ResponseHandler(mockClusterMap); - time = new MockTime(); networkClientFactory = new MockNetworkClientFactory(vprops, mockSelectorState, MAX_PORTS_PLAIN_TEXT, MAX_PORTS_SSL, CHECKOUT_TIMEOUT_MS, mockServerLayout, time); router = new NonBlockingRouter(new RouterConfig(vprops), new NonBlockingRouterMetrics(new MetricRegistry()), @@ -120,15 +109,6 @@ public GetOperationTest() networkClient = networkClientFactory.getNetworkClient(); } - @Before - public void before() { - // almost all tests create and complete an operation outside of the router. Router's count of active operations - // will get decremented when that happens. Since the operation is not submitted via the router in this test, - // the count would never have been incremented. It is done here so that the count is ultimately zero when each test - // completes. - NonBlockingRouter.incrementOperationsCount(); - } - @After public void after() { if (networkClient != null) { @@ -145,8 +125,14 @@ public void after() { public void testInstantiation() throws Exception { String blobIdStr = (new BlobId(mockClusterMap.getWritablePartitionIds().get(0))).getID(); + Callback operationCallback = new Callback() { + @Override + public void onCompletion(BlobInfo result, Exception exception) { + // no op. + } + }; + // test a bad case - GetBlobInfoOperation op; try { new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, "invalid_id", operationFuture, operationCallback, time); @@ -158,14 +144,13 @@ public void testInstantiation() } // test a good case - op = new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, - operationCallback, time); + GetBlobInfoOperation op = + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, + operationCallback, time); Assert.assertEquals("Callback must match", operationCallback, op.getCallback()); Assert.assertEquals("Futures must match", operationFuture, op.getFuture()); Assert.assertEquals("Blob ids must match", blobIdStr, op.getBlobIdStr()); - // count was incremented, but this operation was not completed, so decrement. - NonBlockingRouter.decrementOperationsCount(); } /** @@ -176,8 +161,7 @@ public void testInstantiation() public void testPollAndResponseHandling() throws Exception { GetBlobInfoOperation op = - new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, - operationCallback, time); + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, null, time); ArrayList requestListToFill = new ArrayList<>(); requestRegistrationCallback.requestListToFill = requestListToFill; op.poll(requestRegistrationCallback); @@ -192,7 +176,7 @@ public void testPollAndResponseHandling() } } Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); - assertSuccess(); + assertSuccess(op); } /** @@ -203,8 +187,7 @@ public void testPollAndResponseHandling() public void testRouterRequestTimeoutAllFailure() throws Exception { GetBlobInfoOperation op = - new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, - operationCallback, time); + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, null, time); requestRegistrationCallback.requestListToFill = new ArrayList<>(); op.poll(requestRegistrationCallback); while (!op.isOperationComplete()) { @@ -216,9 +199,7 @@ public void testRouterRequestTimeoutAllFailure() Assert.assertEquals("Must have attempted sending requests to all replicas", replicasCount, correlationIdToGetOperation.size()); Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); - Assert.assertTrue(operationFuture.isDone()); - Assert.assertEquals(operationException.get(), op.getOperationException()); - RouterException routerException = (RouterException) operationException.get(); + RouterException routerException = (RouterException) op.getOperationException(); Assert.assertEquals(RouterErrorCode.OperationTimedOut, routerException.getErrorCode()); } @@ -230,29 +211,29 @@ public void testRouterRequestTimeoutAllFailure() public void testNetworkClientTimeoutAllFailure() throws Exception { GetBlobInfoOperation op = - new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, - operationCallback, time); + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, null, time); ArrayList requestListToFill = new ArrayList<>(); requestRegistrationCallback.requestListToFill = requestListToFill; while (!op.isOperationComplete()) { op.poll(requestRegistrationCallback); - List responses = sendAndWaitForResponses(requestListToFill); - requestListToFill.clear(); - for (ResponseInfo response : responses) { + for (RequestInfo requestInfo : requestListToFill) { ResponseInfo fakeResponse = - new ResponseInfo(response.getRequest(), NetworkClientErrorCode.NetworkError, response.getResponse()); + new ResponseInfo(requestInfo.getRequest(), NetworkClientErrorCode.NetworkError, null); op.handleResponse(fakeResponse); if (op.isOperationComplete()) { break; } } + requestListToFill.clear(); } - Assert.assertEquals(replicasCount, correlationIdToGetOperation.size()); + // At this time requests would have been created for all replicas, as none of them were delivered, + // and cross-colo proxying is enabled by default. + Assert.assertEquals("Must have attempted sending requests to all replicas", replicasCount, + correlationIdToGetOperation.size()); Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); - Assert.assertTrue(operationFuture.isDone()); - RouterException routerException = (RouterException) operationException.get(); + RouterException routerException = (RouterException) op.getOperationException(); Assert.assertEquals(RouterErrorCode.OperationTimedOut, routerException.getErrorCode()); } @@ -265,8 +246,7 @@ public void testNetworkClientTimeoutAllFailure() public void testBlobNotFoundCase() throws Exception { GetBlobInfoOperation op = - new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, - operationCallback, time); + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, null, time); ArrayList requestListToFill = new ArrayList<>(); requestRegistrationCallback.requestListToFill = requestListToFill; @@ -288,8 +268,7 @@ public void testBlobNotFoundCase() Assert.assertEquals("Must have attempted sending requests to all replicas", replicasCount, correlationIdToGetOperation.size()); Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); - Assert.assertTrue(operationFuture.isDone()); - RouterException routerException = (RouterException) operationException.get(); + RouterException routerException = (RouterException) op.getOperationException(); Assert.assertEquals(RouterErrorCode.BlobDoesNotExist, routerException.getErrorCode()); } @@ -299,65 +278,40 @@ public void testBlobNotFoundCase() * @throws Exception */ @Test - public void testErrorPrecedenceWithBlobDeletedCase() + public void testErrorPrecedenceWithBlobDeletedAndExpiredCase() throws Exception { - GetBlobInfoOperation op = - new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, - operationCallback, time); - ArrayList requestListToFill = new ArrayList<>(); - requestRegistrationCallback.requestListToFill = requestListToFill; - - int i = 0; - int indexToSetBlobDeleted = random.nextInt(replicasCount); - for (MockServer server : mockServerLayout.getMockServers()) { - if (i == indexToSetBlobDeleted) { - server.setServerErrorForAllRequests(ServerErrorCode.Blob_Deleted); - } else { - server.setServerErrorForAllRequests(ServerErrorCode.Blob_Not_Found); - } - i++; - } - - while (!op.isOperationComplete()) { - op.poll(requestRegistrationCallback); - List responses = sendAndWaitForResponses(requestListToFill); - for (ResponseInfo response : responses) { - op.handleResponse(response); - if (op.isOperationComplete()) { - break; + ServerErrorCode[] serverErrorCodesToTest = {ServerErrorCode.Blob_Deleted, ServerErrorCode.Blob_Expired}; + RouterErrorCode[] routerErrorCodesToExpect = {RouterErrorCode.BlobDeleted, RouterErrorCode.BlobExpired}; + for (int i = 0; i < serverErrorCodesToTest.length; i++) { + int indexToSetCustomError = random.nextInt(replicasCount); + ServerErrorCode[] serverErrorCodesInOrder = new ServerErrorCode[9]; + for (int j = 0; j < serverErrorCodesInOrder.length; j++) { + if (j == indexToSetCustomError) { + serverErrorCodesInOrder[j] = serverErrorCodesToTest[i]; + } else { + serverErrorCodesInOrder[j] = ServerErrorCode.Blob_Not_Found; } } + testErrorPrecedence(serverErrorCodesInOrder, routerErrorCodesToExpect[i]); } - - Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); - Assert.assertTrue(operationFuture.isDone()); - RouterException routerException = (RouterException) operationException.get(); - Assert.assertEquals(RouterErrorCode.BlobDeleted, routerException.getErrorCode()); } /** - * Test the case with Blob_Not_Found errors from most servers, and Blob_Expired at just one server. The latter - * should be the exception received for the operation. + * Help test error precedence. + * @param serverErrorCodesInOrder the list of error codes to set the mock servers with. + * @param expectedErrorCode the expected router error code for the operation. * @throws Exception */ - @Test - public void testErrorPrecedenceWithBlobExpiredCase() + private void testErrorPrecedence(ServerErrorCode[] serverErrorCodesInOrder, RouterErrorCode expectedErrorCode) throws Exception { GetBlobInfoOperation op = - new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, - operationCallback, time); + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, null, time); ArrayList requestListToFill = new ArrayList<>(); requestRegistrationCallback.requestListToFill = requestListToFill; int i = 0; - int indexToSetBlobDeleted = random.nextInt(replicasCount); for (MockServer server : mockServerLayout.getMockServers()) { - if (i == indexToSetBlobDeleted) { - server.setServerErrorForAllRequests(ServerErrorCode.Blob_Expired); - } else { - server.setServerErrorForAllRequests(ServerErrorCode.Blob_Not_Found); - } - i++; + server.setServerErrorForAllRequests(serverErrorCodesInOrder[i++]); } while (!op.isOperationComplete()) { @@ -371,10 +325,8 @@ public void testErrorPrecedenceWithBlobExpiredCase() } } - Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); - Assert.assertTrue(operationFuture.isDone()); - RouterException routerException = (RouterException) operationException.get(); - Assert.assertEquals(RouterErrorCode.BlobExpired, routerException.getErrorCode()); + RouterException routerException = (RouterException) op.getOperationException(); + Assert.assertEquals(expectedErrorCode, routerException.getErrorCode()); } /** @@ -386,8 +338,7 @@ public void testErrorPrecedenceWithBlobExpiredCase() public void testSuccessInThePresenceOfVariousErrors() throws Exception { GetBlobInfoOperation op = - new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, - operationCallback, time); + new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, null, time); ArrayList requestListToFill = new ArrayList<>(); requestRegistrationCallback.requestListToFill = requestListToFill; @@ -396,10 +347,10 @@ public void testSuccessInThePresenceOfVariousErrors() mockServers.get(0).setServerErrorForAllRequests(ServerErrorCode.Unknown_Error); mockServers.get(1).setServerErrorForAllRequests(ServerErrorCode.Unknown_Error); mockServers.get(2).setServerErrorForAllRequests(ServerErrorCode.IO_Error); - mockServers.get(3).setServerErrorForAllRequests(ServerErrorCode.No_Error); + mockServers.get(3).setServerErrorForAllRequests(ServerErrorCode.Blob_Not_Found); mockServers.get(4).setServerErrorForAllRequests(ServerErrorCode.Data_Corrupt); mockServers.get(5).setServerErrorForAllRequests(ServerErrorCode.Blob_Not_Found); - mockServers.get(6).setServerErrorForAllRequests(ServerErrorCode.No_Error); + mockServers.get(6).setServerErrorForAllRequests(ServerErrorCode.Blob_Not_Found); mockServers.get(7).setServerErrorForAllRequests(ServerErrorCode.Disk_Unavailable); mockServers.get(8).setServerErrorForAllRequests(ServerErrorCode.Unknown_Error); @@ -425,7 +376,7 @@ public void testSuccessInThePresenceOfVariousErrors() } Assert.assertTrue("Operation should be complete at this time", op.isOperationComplete()); - assertSuccess(); + assertSuccess(op); } /** @@ -448,13 +399,16 @@ private List sendAndWaitForResponses(List requestList } /** - * Assert that the operation is complete and successful. + * Assert that the operation is complete and successful. Note that the future completion and callback invocation + * happens outside of the GetOperation, so those are not checked here. But at this point, the operation result should + * be ready. + * @param op the {@link GetBlobInfoOperation} that should have completed. */ - private void assertSuccess() { - Assert.assertTrue(operationFuture.isDone()); - Assert.assertEquals("Null expected", null, operationException.get()); - BlobInfo blobInfo = operationResult.get(); - Assert.assertEquals("Blob properties must be the same", blobProperties, blobInfo.getBlobProperties()); + private void assertSuccess(GetBlobInfoOperation op) { + Assert.assertEquals("Null expected", null, op.getOperationException()); + BlobInfo blobInfo = op.getOperationResult(); + Assert.assertTrue("Blob properties must be the same", + RouterTestHelpers.haveEquivalentFields(blobProperties, blobInfo.getBlobProperties())); Assert.assertArrayEquals("User metadata must be the same", userMetadata, blobInfo.getUserMetadata()); } diff --git a/ambry-router/src/test/java/com.github.ambry.router/MockServer.java b/ambry-router/src/test/java/com.github.ambry.router/MockServer.java index 8c1544fe4d..1ae44ac02a 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/MockServer.java +++ b/ambry-router/src/test/java/com.github.ambry.router/MockServer.java @@ -124,14 +124,14 @@ GetResponse makeGetResponse(GetRequest getRequest, ServerErrorCode getError) ServerErrorCode serverError; ServerErrorCode partitionError; // getError could be at the server level or the partition level. For partition level errors, - // handle set it in the partitionResponseInfo + // set it in the partitionResponseInfo if (getError == ServerErrorCode.No_Error || getError == ServerErrorCode.Blob_Expired || getError == ServerErrorCode.Blob_Deleted || getError == ServerErrorCode.Blob_Not_Found) { partitionError = getError; serverError = ServerErrorCode.No_Error; } else { serverError = getError; - // does not matter. + // does not matter - this will not be checked if serverError is not No_Error. partitionError = ServerErrorCode.No_Error; } @@ -141,8 +141,10 @@ GetResponse makeGetResponse(GetRequest getRequest, ServerErrorCode getError) StoreKey key = getRequest.getPartitionInfoList().get(0).getBlobIds().get(0); if (blobs.containsKey(key.getID())) { ByteBuffer buf = blobs.get(key.getID()).duplicate(); - buf.getLong(); // read off the size - buf.getShort(); // read off the type. + // read off the size + buf.getLong(); + // read off the type. + buf.getShort(); PutRequest originalBlobPutReq = PutRequest.readFrom(new DataInputStream(new ByteBufferInputStream(buf)), clusterMap); switch (getRequest.getMessageFormatFlag()) { @@ -211,7 +213,6 @@ DeleteResponse makeDeleteResponse(DeleteRequest deleteRequest, ServerErrorCode d * @param putRequest the PutRequest * @throws IOException if there was an error reading the contents of the given PutRequest. */ - private void updateBlobMap(PutRequest putRequest) throws IOException { String id = putRequest.getBlobId().getID(); diff --git a/ambry-router/src/test/java/com.github.ambry.router/PutOperationTest.java b/ambry-router/src/test/java/com.github.ambry.router/PutOperationTest.java index ced97aa2aa..8645f547a1 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/PutOperationTest.java +++ b/ambry-router/src/test/java/com.github.ambry.router/PutOperationTest.java @@ -86,10 +86,11 @@ public PutOperationTest() @Test public void testSendIncomplete() throws Exception { + int numChunks = NonBlockingRouter.MAX_IN_MEM_CHUNKS + 1; BlobProperties blobProperties = - new BlobProperties(chunkSize * 5, "serviceId", "memberId", "contentType", false, Utils.Infinite_Time); + new BlobProperties(chunkSize * numChunks, "serviceId", "memberId", "contentType", false, Utils.Infinite_Time); byte[] userMetadata = new byte[10]; - byte[] content = new byte[chunkSize * 5]; + byte[] content = new byte[chunkSize * numChunks]; random.nextBytes(content); ReadableStreamChannel channel = new ByteBufferReadableStreamChannel(ByteBuffer.wrap(content)); FutureResult future = new FutureResult<>(); @@ -98,15 +99,15 @@ public void testSendIncomplete() null, time); List requestInfos = new ArrayList<>(); requestRegistrationCallback.requestListToFill = requestInfos; - // Since this channel is in memory, one call to fill chunks would end up filling the maximum number of PutChunks (4). + // Since this channel is in memory, one call to fill chunks would end up filling the maximum number of PutChunks. op.fillChunks(); // A poll should therefore return requestParallelism number of requests from each chunk op.poll(requestRegistrationCallback); - Assert.assertEquals(4 * requestParallelism, requestInfos.size()); + Assert.assertEquals(NonBlockingRouter.MAX_IN_MEM_CHUNKS * requestParallelism, requestInfos.size()); - // There are 5 data chunks for this blob (and a metadata chunk). The maximum number of in-memory PutChunk objects - // for an operation is 4. So, once the first chunk is completely sent out, the first PutChunk will be reused. What - // the test verifies is that the buffer of the first PutChunk does not get reused. It does this as follows: + // There are MAX_IN_MEM_CHUNKS + 1 data chunks for this blob (and a metadata chunk). + // Once the first chunk is completely sent out, the first PutChunk will be reused. What the test verifies is that + // the buffer of the first PutChunk does not get reused. It does this as follows: // For the first chunk, // 1. use first request to succeed the chunk (the successTarget is set to 1). // 2. read and store from the second for comparing later. @@ -132,14 +133,14 @@ public void testSendIncomplete() for (int i = 3; i < requestInfos.size(); i++) { op.handleResponse(getResponseInfo(requestInfos.get(i))); } - // fill the first PutChunk with the 5th chunk. + // fill the first PutChunk with the last chunk. op.fillChunks(); - // Verify that the 5th chunk was filled. + // Verify that the last chunk was filled. requestInfos.clear(); op.poll(requestRegistrationCallback); Assert.assertEquals(1 * requestParallelism, requestInfos.size()); - // Verify that that the buffer of the third request is not affected. + // Verify that the buffer of the third request is not affected. buf = ByteBuffer.allocate((int) savedRequest.sizeInBytes()); bufChannel = new ByteBufferChannel(buf); savedRequest.writeTo(bufChannel); @@ -178,7 +179,7 @@ private void resetCorrelationId(byte[] request) { } /** - * Get the {@link ResponseInfo} for the given {@link RequestInfo} using tha {@link MockServer} + * Get the {@link ResponseInfo} for the given {@link RequestInfo} using the {@link MockServer} * @param requestInfo the {@link RequestInfo} for which the response is to be returned. * @return the {@link ResponseInfo} the response for the request. * @throws IOException if there is an error sending the request. diff --git a/ambry-router/src/test/java/com.github.ambry.router/RouterTestHelpers.java b/ambry-router/src/test/java/com.github.ambry.router/RouterTestHelpers.java new file mode 100644 index 0000000000..6023514397 --- /dev/null +++ b/ambry-router/src/test/java/com.github.ambry.router/RouterTestHelpers.java @@ -0,0 +1,37 @@ +/** + * Copyright 2016 LinkedIn Corp. All rights reserved. + * + * Licensed 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. + */ +package com.github.ambry.router; + +import com.github.ambry.messageformat.BlobProperties; + + +/** + * Class with helper methods for testing the router. + */ +class RouterTestHelpers { + /** + * Test whether two {@link BlobProperties} have the same fields + * @return true if the fields are equivalent in the two {@link BlobProperties} + */ + static boolean haveEquivalentFields(BlobProperties a, BlobProperties b) { + return a.getBlobSize() == b.getBlobSize() && + a.getServiceId().equals(b.getServiceId()) && + a.getOwnerId().equals(b.getOwnerId()) && + a.getContentType().equals(b.getContentType()) && + a.isPrivate() == b.isPrivate() && + a.getTimeToLiveInSeconds() == b.getTimeToLiveInSeconds() && + a.getCreationTimeInMs() == b.getCreationTimeInMs(); + } +} + From 3c097d5da0877939353661be6110f491e42df1f9 Mon Sep 17 00:00:00 2001 From: Priyesh Narayanan Date: Fri, 6 May 2016 16:04:48 -0700 Subject: [PATCH 3/6] Refactor, other fixes --- .../DeleteManager.java | 31 +++++++--------- .../DeleteOperation.java | 16 ++++++--- .../GetBlobInfoOperation.java | 36 +++++++++---------- .../com.github.ambry.router/GetManager.java | 29 ++++++++------- .../com.github.ambry.router/GetOperation.java | 5 ++- .../com.github.ambry.router/PutManager.java | 29 ++++++++------- .../com.github.ambry.router/PutOperation.java | 31 ++++++++++------ .../GetManagerTest.java | 4 +-- .../GetOperationTest.java | 2 -- .../PutOperationTest.java | 3 +- 10 files changed, 96 insertions(+), 90 deletions(-) diff --git a/ambry-router/src/main/java/com.github.ambry.router/DeleteManager.java b/ambry-router/src/main/java/com.github.ambry.router/DeleteManager.java index 8a984ff64c..3fb83dd8ce 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/DeleteManager.java +++ b/ambry-router/src/main/java/com.github.ambry.router/DeleteManager.java @@ -25,11 +25,9 @@ import com.github.ambry.utils.Time; import java.util.Collections; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,7 +41,6 @@ class DeleteManager { private final HashMap correlationIdToDeleteOperation; private final NotificationSystem notificationSystem; private final Time time; - private final AtomicBoolean isOpen = new AtomicBoolean(true); private final ResponseHandler responseHandler; private final NonBlockingRouterMetrics routerMetrics; private final ClusterMap clusterMap; @@ -112,13 +109,12 @@ void submitDeleteBlobOperation(String blobIdString, FutureResult futureRes * @param requestListToFill list to be filled with the requests created. */ public void poll(List requestListToFill) { - Iterator deleteOperationIterator = deleteOperations.iterator(); requestRegistrationCallback.requestListToFill = requestListToFill; - while (deleteOperationIterator.hasNext()) { - DeleteOperation op = deleteOperationIterator.next(); + for (DeleteOperation op : deleteOperations) { op.poll(requestRegistrationCallback); - if (op.isOperationComplete()) { - deleteOperationIterator.remove(); + if (op.isOperationComplete() && deleteOperations.remove(op)) { + // In order to ensure that an operation is completed only once, call onComplete() only at the place where the + // operation actually gets removed from the set of operations. See comment within close(). onComplete(op); } } @@ -126,7 +122,7 @@ public void poll(List requestListToFill) { /** * Handles responses received for each of the {@link DeleteOperation} within this delete manager. - * @param responseInfo A response from {@link com.github.ambry.network.NetworkClient} + * @param responseInfo the {@link ResponseInfo} containing the response. */ void handleResponse(ResponseInfo responseInfo) { int correlationId = ((DeleteRequest) responseInfo.getRequest()).getCorrelationId(); @@ -134,8 +130,7 @@ void handleResponse(ResponseInfo responseInfo) { // If it is still an active operation, hand over the response. Otherwise, ignore. if (deleteOperations.contains(deleteOperation)) { deleteOperation.handleResponse(responseInfo); - if (deleteOperation.isOperationComplete()) { - deleteOperations.remove(deleteOperation); + if (deleteOperation.isOperationComplete() && deleteOperations.remove(deleteOperation)) { onComplete(deleteOperation); } } @@ -144,7 +139,7 @@ void handleResponse(ResponseInfo responseInfo) { /** * Called when the delete operation is completed. The {@code DeleteManager} also finishes the delete operation * by performing the callback and notification. - * @param op The {@lilnk DeleteOperation} that has completed. + * @param op The {@link DeleteOperation} that has completed. */ void onComplete(DeleteOperation op) { if (op.getOperationException() == null) { @@ -159,12 +154,12 @@ void onComplete(DeleteOperation op) { * will have no effect. */ void close() { - if (isOpen.compareAndSet(true, false)) { - Iterator iter = deleteOperations.iterator(); - while (iter.hasNext()) { - DeleteOperation deleteOperation = iter.next(); - iter.remove(); - NonBlockingRouter.completeOperation(deleteOperation.getFutureResult(), deleteOperation.getCallback(), null, + for (DeleteOperation op : deleteOperations) { + // There is a rare scenario where the operation gets removed from this set and gets completed concurrently by + // the RequestResponseHandler thread when it is in poll() or handleResponse(). In order to avoid the completion + // from happening twice, complete it here only if the remove was successful. + if (deleteOperations.remove(op)) { + NonBlockingRouter.completeOperation(op.getFutureResult(), op.getCallback(), null, new RouterException("Aborted operation because Router is closed.", RouterErrorCode.RouterClosed)); } } diff --git a/ambry-router/src/main/java/com.github.ambry.router/DeleteOperation.java b/ambry-router/src/main/java/com.github.ambry.router/DeleteOperation.java index ee510e8d39..d0e196311c 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/DeleteOperation.java +++ b/ambry-router/src/main/java/com.github.ambry.router/DeleteOperation.java @@ -95,15 +95,21 @@ class DeleteOperation { /** * Gets a list of {@link DeleteRequest} for sending to replicas. - * @param requestFillCallback the {@link RequestRegistrationCallback} to call for every request + * @param requestRegistrationCallback the {@link RequestRegistrationCallback} to call for every request * that gets created as part of this poll operation. */ - void poll(RequestRegistrationCallback requestFillCallback) { + void poll(RequestRegistrationCallback requestRegistrationCallback) { cleanupExpiredInflightRequests(); checkAndMaybeComplete(); - if (isOperationComplete()) { - return; + if (!isOperationComplete()) { + fetchRequests(requestRegistrationCallback); } + } + + /** + * Fetch {@link DeleteRequest}s to send for the operation. + */ + private void fetchRequests(RequestRegistrationCallback requestRegistrationCallback) { Iterator replicaIterator = operationTracker.getReplicaIterator(); while (replicaIterator.hasNext()) { ReplicaId replica = replicaIterator.next(); @@ -112,7 +118,7 @@ void poll(RequestRegistrationCallback requestFillCallback) { DeleteRequest deleteRequest = createDeleteRequest(); inflightRequestInfos.put(deleteRequest.getCorrelationId(), new InflightRequestInfo(time.milliseconds(), replica)); RequestInfo requestInfo = new RequestInfo(hostname, port, deleteRequest); - requestFillCallback.registerRequestToSend(this, requestInfo); + requestRegistrationCallback.registerRequestToSend(this, requestInfo); replicaIterator.remove(); } } diff --git a/ambry-router/src/main/java/com.github.ambry.router/GetBlobInfoOperation.java b/ambry-router/src/main/java/com.github.ambry.router/GetBlobInfoOperation.java index 43811d091c..9c0f906159 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/GetBlobInfoOperation.java +++ b/ambry-router/src/main/java/com.github.ambry.router/GetBlobInfoOperation.java @@ -81,10 +81,25 @@ MessageFormatFlags getOperationFlag() { return MessageFormatFlags.BlobInfo; } + /** + * For this operation, create and populate get requests (in the form of {@link RequestInfo}) to send out. + * @param requestRegistrationCallback the {@link RequestRegistrationCallback} to call for every request that gets + * created as part of this poll operation. + */ + @Override + void poll(RequestRegistrationCallback requestRegistrationCallback) { + //First, check if any of the existing requests have timed out. + cleanupExpiredInFlightRequests(); + checkAndMaybeComplete(); + if (!isOperationComplete()) { + fetchRequests(requestRegistrationCallback); + } + } + /** * Clean up requests sent out by this operation that have now timed out. */ - void cleanupExpiredInFlightRequests() { + private void cleanupExpiredInFlightRequests() { Iterator> inFlightRequestsIterator = correlationIdToGetRequestInfo.entrySet().iterator(); while (inFlightRequestsIterator.hasNext()) { @@ -102,20 +117,9 @@ void cleanupExpiredInFlightRequests() { } /** - * For this operation, create and populate get requests (in the form of {@link RequestInfo}) to send out. - * @param requestRegistrationCallback the {@link RequestRegistrationCallback} to call for every request that gets - * created as part of this poll operation. + * Fetch {@link GetRequest}s to send for the operation. */ - @Override - void poll(RequestRegistrationCallback requestRegistrationCallback) { - //First, check if any of the existing requests have timed out. - cleanupExpiredInFlightRequests(); - - checkAndMaybeComplete(); - if (isOperationComplete()) { - return; - } - + private void fetchRequests(RequestRegistrationCallback requestRegistrationCallback) { Iterator replicaIterator = operationTracker.getReplicaIterator(); while (replicaIterator.hasNext()) { ReplicaId replicaId = replicaIterator.next(); @@ -144,10 +148,6 @@ void handleResponse(ResponseInfo responseInfo) { int correlationId = ((GetRequest) responseInfo.getRequest()).getCorrelationId(); // Get the GetOperation that generated the request. GetRequestInfo getRequestInfo = correlationIdToGetRequestInfo.remove(correlationId); - if (getRequestInfo == null) { - // Ignore right away. This associated operation has completed. - return; - } if (responseInfo.getError() != null) { setOperationException(new RouterException("Operation timed out", RouterErrorCode.OperationTimedOut)); responseHandler.onRequestResponseException(getRequestInfo.replicaId, new IOException("NetworkClient error")); diff --git a/ambry-router/src/main/java/com.github.ambry.router/GetManager.java b/ambry-router/src/main/java/com.github.ambry.router/GetManager.java index 937a762352..a2ddf31c78 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/GetManager.java +++ b/ambry-router/src/main/java/com.github.ambry.router/GetManager.java @@ -24,7 +24,6 @@ import com.github.ambry.utils.Time; import java.util.Collections; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -63,6 +62,7 @@ public void registerRequestToSend(GetOperation getOperation, RequestInfo request correlationIdToGetOperation.put(((RequestOrResponse) requestInfo.getRequest()).getCorrelationId(), getOperation); } } + // A single callback as this will never get called concurrently. The list of request to fill will be set as // appropriate before the callback is passed on to GetOperations, every time. private final GetRequestRegistrationCallbackImpl requestRegistrationCallback = @@ -122,13 +122,12 @@ void submitGetBlobOperation(String blobId, FutureResult f * @param requestListToFill list to be filled with the requests created */ void poll(List requestListToFill) { - Iterator getOperationIterator = getOperations.iterator(); requestRegistrationCallback.requestListToFill = requestListToFill; - while (getOperationIterator.hasNext()) { - GetOperation op = getOperationIterator.next(); + for (GetOperation op : getOperations) { op.poll(requestRegistrationCallback); - if (op.isOperationComplete()) { - getOperationIterator.remove(); + if (op.isOperationComplete() && getOperations.remove(op)) { + // In order to ensure that an operation is completed only once, call onComplete() only at the place where the + // operation actually gets removed from the set of operations. See comment within close(). onComplete(op); } } @@ -154,8 +153,7 @@ void handleResponse(ResponseInfo responseInfo) { GetOperation getOperation = correlationIdToGetOperation.get(getRequest.getCorrelationId()); if (getOperations.contains(getOperation)) { getOperation.handleResponse(responseInfo); - if (getOperation.isOperationComplete()) { - getOperations.remove(getOperation); + if (getOperation.isOperationComplete() && getOperations.remove(getOperation)) { onComplete(getOperation); } } @@ -166,13 +164,14 @@ void handleResponse(ResponseInfo responseInfo) { * Complete all existing get operations. */ void close() { - Iterator iter = getOperations.iterator(); - while (iter.hasNext()) { - GetOperation op = iter.next(); - iter.remove(); - logger.trace("Aborting operation for blob id: ", op.getBlobIdStr()); - NonBlockingRouter.completeOperation(op.getFuture(), op.getCallback(), null, - new RouterException("Aborted operation because Router is closed", RouterErrorCode.RouterClosed)); + for (GetOperation op : getOperations) { + // There is a rare scenario where the operation gets removed from this set and gets completed concurrently by + // the RequestResponseHandler thread when it is in poll() or handleResponse(). In order to avoid the completion + // from happening twice, complete it here only if the remove was successful. + if (getOperations.remove(op)) { + NonBlockingRouter.completeOperation(op.getFuture(), op.getCallback(), null, + new RouterException("Aborted operation because Router is closed", RouterErrorCode.RouterClosed)); + } } } } diff --git a/ambry-router/src/main/java/com.github.ambry.router/GetOperation.java b/ambry-router/src/main/java/com.github.ambry.router/GetOperation.java index 48d3a9410e..61a7983f38 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/GetOperation.java +++ b/ambry-router/src/main/java/com.github.ambry.router/GetOperation.java @@ -187,9 +187,8 @@ void setOperationException(RouterException exception) { */ protected GetRequest createGetRequest(BlobId blobId, MessageFormatFlags flag) { List blobIds = Collections.singletonList(blobId); - List partitionRequestInfoList = new ArrayList(); - PartitionRequestInfo partitionRequestInfo = new PartitionRequestInfo(blobId.getPartition(), blobIds); - partitionRequestInfoList.add(partitionRequestInfo); + List partitionRequestInfoList = Collections.singletonList(new PartitionRequestInfo(blobId + .getPartition(), blobIds)); return new GetRequest(NonBlockingRouter.correlationIdGenerator.incrementAndGet(), routerConfig.routerHostname, flag, partitionRequestInfoList, GetOptions.None); } diff --git a/ambry-router/src/main/java/com.github.ambry.router/PutManager.java b/ambry-router/src/main/java/com.github.ambry.router/PutManager.java index b25211e8a2..55726557db 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/PutManager.java +++ b/ambry-router/src/main/java/com.github.ambry.router/PutManager.java @@ -69,6 +69,7 @@ public void registerRequestToSend(PutOperation putOperation, RequestInfo request correlationIdToPutOperation.put(((RequestOrResponse) requestInfo.getRequest()).getCorrelationId(), putOperation); } } + // A single callback as this will never get called concurrently. The list of request to fill will be set as // appropriate before the callback is passed on to the PutOperations, every time. private final PutRequestRegistrationCallbackImpl requestRegistrationCallback = @@ -126,14 +127,12 @@ void submitPutBlobOperation(BlobProperties blobProperties, byte[] userMetaData, * @param requestListToFill list to be filled with the requests created */ void poll(List requestListToFill) { - Iterator putOperationIterator = putOperations.iterator(); requestRegistrationCallback.requestListToFill = requestListToFill; - while (putOperationIterator.hasNext()) { - PutOperation op = putOperationIterator.next(); + for (PutOperation op : putOperations) { op.poll(requestRegistrationCallback); - if (op.isOperationComplete()) { - // Operation is done. - putOperationIterator.remove(); + if (op.isOperationComplete() && putOperations.remove(op)) { + // In order to ensure that an operation is completed only once, call onComplete() only at the place where the + // operation actually gets removed from the set of operations. See comment within closePendingOperations(). onComplete(op); } } @@ -150,8 +149,7 @@ void handleResponse(ResponseInfo responseInfo) { // If it is still an active operation, hand over the response. Otherwise, ignore. if (putOperations.contains(putOperation)) { putOperation.handleResponse(responseInfo); - if (putOperation.isOperationComplete()) { - putOperations.remove(putOperation); + if (putOperation.isOperationComplete() && putOperations.remove(putOperation)) { onComplete(putOperation); } } @@ -159,7 +157,6 @@ void handleResponse(ResponseInfo responseInfo) { /** * Returns a list of ids of successfully put chunks that were part of unsuccessful put operations. - * @return list of ids to delete. */ void getIdsToDelete(List idsToDelete) { // @todo save and return ids of failed puts. @@ -211,12 +208,14 @@ void close() { * 2. By the {@link ChunkFiller} thread when it exits abnormally. */ void completePendingOperations() { - Iterator iter = putOperations.iterator(); - while (iter.hasNext()) { - PutOperation op = iter.next(); - iter.remove(); - NonBlockingRouter.completeOperation(op.getFuture(), op.getCallback(), null, - new RouterException("Aborted operation because Router is closed", RouterErrorCode.RouterClosed)); + for (PutOperation op : putOperations) { + // There is a rare scenario where the operation gets removed from this set and gets completed concurrently by + // the RequestResponseHandler thread when it is in poll() or handleResponse(). In order to avoid the completion + // from happening twice, complete it here only if the remove was successful. + if (putOperations.remove(op)) { + NonBlockingRouter.completeOperation(op.getFuture(), op.getCallback(), null, + new RouterException("Aborted operation because Router is closed", RouterErrorCode.RouterClosed)); + } } } diff --git a/ambry-router/src/main/java/com.github.ambry.router/PutOperation.java b/ambry-router/src/main/java/com.github.ambry.router/PutOperation.java index be60384cc8..979634890c 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/PutOperation.java +++ b/ambry-router/src/main/java/com.github.ambry.router/PutOperation.java @@ -655,18 +655,27 @@ void checkAndMaybeComplete() { } /** - * Fetch put requests to send for the current data chunk. * This is one of two main entry points to this class, the other being {@link #handleResponse(ResponseInfo)}. * Apart from fetching requests to send out, this also checks for timeouts of issued requests, * status of the operation and anything else that needs to be done within this PutChunk. The callers guarantee * that this method is called on all the PutChunks of an operation until either the operation, * or the chunk operation is completed. - * @param requestFillCallback the {@link RequestRegistrationCallback} to call for every request that gets created as - * part of this poll operation. + * @param requestRegistrationCallback the {@link RequestRegistrationCallback} to call for every request that gets + * created as part of this poll operation. */ - void poll(RequestRegistrationCallback requestFillCallback) { + void poll(RequestRegistrationCallback requestRegistrationCallback) { maybeFreeDefunctBuffers(); - //First, check if any of the existing requests have timed out. + cleanupExpiredInFlightRequests(); + checkAndMaybeComplete(); + if (!isComplete()) { + fetchRequests(requestRegistrationCallback); + } + } + + /** + * Clean up requests sent out by this operation that have now timed out. + */ + private void cleanupExpiredInFlightRequests() { Iterator> inFlightRequestsIterator = correlationIdToChunkPutRequestInfo.entrySet().iterator(); while (inFlightRequestsIterator.hasNext()) { @@ -680,12 +689,12 @@ void poll(RequestRegistrationCallback requestFillCallback) { break; } } + } - checkAndMaybeComplete(); - if (isComplete()) { - return; - } - + /** + * Fetch {@link PutRequest}s to send for the current data chunk. + */ + private void fetchRequests(RequestRegistrationCallback requestRegistrationCallback) { Iterator replicaIterator = operationTracker.getReplicaIterator(); while (replicaIterator.hasNext()) { ReplicaId replicaId = replicaIterator.next(); @@ -697,7 +706,7 @@ void poll(RequestRegistrationCallback requestFillCallback) { correlationIdToChunkPutRequestInfo .put(correlationId, new ChunkPutRequestInfo(replicaId, putRequest, time.milliseconds())); correlationIdToPutChunk.put(correlationId, this); - requestFillCallback.registerRequestToSend(PutOperation.this, request); + requestRegistrationCallback.registerRequestToSend(PutOperation.this, request); replicaIterator.remove(); } } diff --git a/ambry-router/src/test/java/com.github.ambry.router/GetManagerTest.java b/ambry-router/src/test/java/com.github.ambry.router/GetManagerTest.java index 35e8b16027..c9e55a9673 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/GetManagerTest.java +++ b/ambry-router/src/test/java/com.github.ambry.router/GetManagerTest.java @@ -130,8 +130,8 @@ public void testFailureOnAllPollThatSends() Assert.fail("operation should have thrown"); } catch (ExecutionException e) { RouterException routerException = (RouterException) e.getCause(); - Assert.assertEquals("Exception received should be router closed error", routerException.getErrorCode(), - RouterErrorCode.RouterClosed); + Assert.assertEquals("Exception received should be router closed error", RouterErrorCode.RouterClosed, + routerException.getErrorCode()); } } diff --git a/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java b/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java index eb9f9a8fbe..550f0598b5 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java +++ b/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java @@ -84,8 +84,6 @@ public void registerRequestToSend(GetOperation getOperation, RequestInfo request } } - ; - public GetOperationTest() throws Exception { VerifiableProperties vprops = getNonBlockingRouterProperties(); diff --git a/ambry-router/src/test/java/com.github.ambry.router/PutOperationTest.java b/ambry-router/src/test/java/com.github.ambry.router/PutOperationTest.java index 8645f547a1..31e605ba60 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/PutOperationTest.java +++ b/ambry-router/src/test/java/com.github.ambry.router/PutOperationTest.java @@ -56,7 +56,8 @@ public void registerRequestToSend(PutOperation putOperation, RequestInfo request requestListToFill.add(requestInfo); correlationIdToPutOperation.put(((RequestOrResponse) requestInfo.getRequest()).getCorrelationId(), putOperation); } - }; + } + private final PutTestRequestRegistrationCallbackImpl requestRegistrationCallback = new PutTestRequestRegistrationCallbackImpl(); From 53f75903d299e35db81938da81515d38f724330a Mon Sep 17 00:00:00 2001 From: Priyesh Narayanan Date: Mon, 9 May 2016 09:40:08 -0700 Subject: [PATCH 4/6] Rename GetOperationTest to GetBlobInfoOperationTest --- ...GetOperationTest.java => GetBlobInfoOperationTest.java} | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) rename ambry-router/src/test/java/com.github.ambry.router/{GetOperationTest.java => GetBlobInfoOperationTest.java} (99%) diff --git a/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java b/ambry-router/src/test/java/com.github.ambry.router/GetBlobInfoOperationTest.java similarity index 99% rename from ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java rename to ambry-router/src/test/java/com.github.ambry.router/GetBlobInfoOperationTest.java index 550f0598b5..9678f42e59 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/GetOperationTest.java +++ b/ambry-router/src/test/java/com.github.ambry.router/GetBlobInfoOperationTest.java @@ -45,7 +45,10 @@ import org.junit.Test; -public class GetOperationTest { +/** + * Tests for {@link GetBlobInfoOperation} + */ +public class GetBlobInfoOperationTest { private static final int MAX_PORTS_PLAIN_TEXT = 3; private static final int MAX_PORTS_SSL = 3; private static final int CHECKOUT_TIMEOUT_MS = 1000; @@ -84,7 +87,7 @@ public void registerRequestToSend(GetOperation getOperation, RequestInfo request } } - public GetOperationTest() + public GetBlobInfoOperationTest() throws Exception { VerifiableProperties vprops = getNonBlockingRouterProperties(); routerConfig = new RouterConfig(vprops); From 03b4afde954280c03d7fc068c340624275548a8c Mon Sep 17 00:00:00 2001 From: Priyesh Narayanan Date: Mon, 9 May 2016 12:25:50 -0700 Subject: [PATCH 5/6] fix a bug in GetManager#handleResponse() --- .../src/main/java/com.github.ambry.router/GetManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ambry-router/src/main/java/com.github.ambry.router/GetManager.java b/ambry-router/src/main/java/com.github.ambry.router/GetManager.java index a2ddf31c78..1eddeac083 100644 --- a/ambry-router/src/main/java/com.github.ambry.router/GetManager.java +++ b/ambry-router/src/main/java/com.github.ambry.router/GetManager.java @@ -150,7 +150,7 @@ void onComplete(GetOperation op) { */ void handleResponse(ResponseInfo responseInfo) { GetRequest getRequest = (GetRequest) responseInfo.getRequest(); - GetOperation getOperation = correlationIdToGetOperation.get(getRequest.getCorrelationId()); + GetOperation getOperation = correlationIdToGetOperation.remove(getRequest.getCorrelationId()); if (getOperations.contains(getOperation)) { getOperation.handleResponse(responseInfo); if (getOperation.isOperationComplete() && getOperations.remove(getOperation)) { From e499475aa102afcda1f07568bb3b89734a8684a2 Mon Sep 17 00:00:00 2001 From: Priyesh Narayanan Date: Mon, 9 May 2016 22:49:30 -0700 Subject: [PATCH 6/6] Test improvements --- .../GetBlobInfoOperationTest.java | 35 ++++++++-- .../com.github.ambry.router/MockServer.java | 69 ++++++++++--------- 2 files changed, 64 insertions(+), 40 deletions(-) diff --git a/ambry-router/src/test/java/com.github.ambry.router/GetBlobInfoOperationTest.java b/ambry-router/src/test/java/com.github.ambry.router/GetBlobInfoOperationTest.java index 9678f42e59..0040f8b46d 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/GetBlobInfoOperationTest.java +++ b/ambry-router/src/test/java/com.github.ambry.router/GetBlobInfoOperationTest.java @@ -57,7 +57,7 @@ public class GetBlobInfoOperationTest { private int requestParallelism = 2; private int successTarget = 1; - private final RouterConfig routerConfig; + private RouterConfig routerConfig; private final MockClusterMap mockClusterMap; private final MockServerLayout mockServerLayout; private final int replicasCount; @@ -89,7 +89,7 @@ public void registerRequestToSend(GetOperation getOperation, RequestInfo request public GetBlobInfoOperationTest() throws Exception { - VerifiableProperties vprops = getNonBlockingRouterProperties(); + VerifiableProperties vprops = new VerifiableProperties(getNonBlockingRouterProperties()); routerConfig = new RouterConfig(vprops); mockClusterMap = new MockClusterMap(); mockServerLayout = new MockServerLayout(mockClusterMap); @@ -338,6 +338,28 @@ private void testErrorPrecedence(ServerErrorCode[] serverErrorCodesInOrder, Rout @Test public void testSuccessInThePresenceOfVariousErrors() throws Exception { + // The put for the blob being requested happened. + String dcWherePutHappened = routerConfig.routerDatacenterName; + + // test requests coming in from local dc as well as cross-colo. + Properties props = getNonBlockingRouterProperties(); + props.setProperty("router.datacenter.name", "DC1"); + routerConfig = new RouterConfig(new VerifiableProperties(props)); + testVariousErrors(dcWherePutHappened); + + props = getNonBlockingRouterProperties(); + props.setProperty("router.datacenter.name", "DC2"); + routerConfig = new RouterConfig(new VerifiableProperties(props)); + testVariousErrors(dcWherePutHappened); + + props = getNonBlockingRouterProperties(); + props.setProperty("router.datacenter.name", "DC3"); + routerConfig = new RouterConfig(new VerifiableProperties(props)); + testVariousErrors(dcWherePutHappened); + } + + private void testVariousErrors(String dcWherePutHappened) + throws Exception { GetBlobInfoOperation op = new GetBlobInfoOperation(routerConfig, mockClusterMap, responseHandler, blobIdStr, operationFuture, null, time); ArrayList requestListToFill = new ArrayList<>(); @@ -355,11 +377,10 @@ public void testSuccessInThePresenceOfVariousErrors() mockServers.get(7).setServerErrorForAllRequests(ServerErrorCode.Disk_Unavailable); mockServers.get(8).setServerErrorForAllRequests(ServerErrorCode.Unknown_Error); - // set the status of one of the servers in the local datacenter to success (we depend on an actual put of the blob - // in order to get back the BlobInfo, and the put would have gone only to the local datacenter). + // clear the hard error in one of the servers in the datacenter where the put happened. for (int i = 0; i < mockServers.size(); i++) { MockServer mockServer = mockServers.get(i); - if (mockServer.getDataCenter().equals(routerConfig.routerDatacenterName)) { + if (mockServer.getDataCenter().equals(dcWherePutHappened)) { mockServer.setServerErrorForAllRequests(ServerErrorCode.No_Error); break; } @@ -417,13 +438,13 @@ private void assertSuccess(GetBlobInfoOperation op) { * Get the properties for the {@link NonBlockingRouter}. * @return the constructed properties. */ - private VerifiableProperties getNonBlockingRouterProperties() { + private Properties getNonBlockingRouterProperties() { Properties properties = new Properties(); properties.setProperty("router.hostname", "localhost"); properties.setProperty("router.datacenter.name", "DC1"); properties.setProperty("router.get.request.parallelism", Integer.toString(requestParallelism)); properties.setProperty("router.get.success.target", Integer.toString(successTarget)); - return new VerifiableProperties(properties); + return properties; } } diff --git a/ambry-router/src/test/java/com.github.ambry.router/MockServer.java b/ambry-router/src/test/java/com.github.ambry.router/MockServer.java index 1ae44ac02a..f682db9891 100644 --- a/ambry-router/src/test/java/com.github.ambry.router/MockServer.java +++ b/ambry-router/src/test/java/com.github.ambry.router/MockServer.java @@ -39,6 +39,7 @@ import java.io.DataInputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.channels.Channels; import java.util.ArrayList; import java.util.HashMap; import java.util.LinkedList; @@ -66,12 +67,12 @@ class MockServer { /** * Take in a request in the form of {@link Send} and return a response in the form of a - * {@link MockBoundedByteBufferReceive}. + * {@link BoundedByteBufferReceive}. * @param send the request. * @return the response. * @throws IOException if there was an error in interpreting the request. */ - public MockBoundedByteBufferReceive send(Send send) + public BoundedByteBufferReceive send(Send send) throws IOException { if (!shouldRespond) { return null; @@ -97,12 +98,19 @@ public MockBoundedByteBufferReceive send(Send send) response.writeTo(channel); ByteBuffer payload = channel.getBuffer(); payload.flip(); - // read the size off. the size is used by the client to allocate the buffer and then interpret the response. - // MockServer abstracts it away at a level above that and returns the "allocated buffer". - payload.getLong(); - return new MockBoundedByteBufferReceive(payload); + BoundedByteBufferReceive boundedByteBufferReceive = new BoundedByteBufferReceive(); + boundedByteBufferReceive.readFrom(Channels.newChannel(new ByteBufferInputStream(payload))); + return boundedByteBufferReceive; } + /** + * Make a {@link PutResponse} for the given {@link PutRequest} for which the given {@link ServerErrorCode} was + * encountered. + * @param putRequest the {@link PutRequest} for which the response is being constructed. + * @param putError the {@link ServerErrorCode} that was encountered. + * @return the created {@link PutResponse} + * @throws IOException if there was an error constructing the response. + */ PutResponse makePutResponse(PutRequest putRequest, ServerErrorCode putError) throws IOException { if (putError == ServerErrorCode.No_Error) { @@ -111,6 +119,15 @@ PutResponse makePutResponse(PutRequest putRequest, ServerErrorCode putError) return new PutResponse(putRequest.getCorrelationId(), putRequest.getClientId(), putError); } + /** + * Make a {@link GetResponse} for the given {@link GetRequest} for which the given {@link ServerErrorCode} was + * encountered. The request could be for BlobInfo or for Blob (the only two options that the router would request + * for). + * @param getRequest the {@link GetRequest} for which the response is being constructed. + * @param getError the {@link ServerErrorCode} that was encountered. + * @return the constructed {@link GetResponse} + * @throws IOException if there was an error constructing the response. + */ GetResponse makeGetResponse(GetRequest getRequest, ServerErrorCode getError) throws IOException { GetResponse getResponse; @@ -194,11 +211,21 @@ GetResponse makeGetResponse(GetRequest getRequest, ServerErrorCode getError) getResponse = new GetResponse(getRequest.getCorrelationId(), getRequest.getClientId(), partitionResponseInfoList, responseSend, serverError); } else { - getResponse = new GetResponse(getRequest.getCorrelationId(), getRequest.getClientId(), serverError); + getResponse = new GetResponse(getRequest.getCorrelationId(), getRequest.getClientId(), + new ArrayList(), new ByteBufferSend(ByteBuffer.allocate(0)), serverError); } return getResponse; } + /** + * + * Make a {@link DeleteResponse} for the given {@link DeleteRequest} for which the given {@link ServerErrorCode} was + * encountered. + * @param deleteRequest the {@link DeleteRequest} for which the response is being constructed. + * @param deleteError the {@link ServerErrorCode} that was encountered. + * @return the constructed {@link DeleteResponse} + * @throws IOException if there was an error constructing the response. + */ DeleteResponse makeDeleteResponse(DeleteRequest deleteRequest, ServerErrorCode deleteError) throws IOException { String blobIdString = deleteRequest.getBlobId().getID(); @@ -289,7 +316,8 @@ public ServerErrorCode getErrorFromBlobIdStr(String blobIdString) { } /** - * Set the mapping relationship between a {@code blobIdString} and the {@link ServerErrorCode} this server should return. + * Set the mapping relationship between a {@code blobIdString} and the {@link ServerErrorCode} this server should + * return. * @param blobIdString The key in this mapping relation. * @param code The {@link ServerErrorCode} for the {@code blobIdString}. */ @@ -298,28 +326,3 @@ public void setBlobIdToServerErrorCode(String blobIdString, ServerErrorCode code } } -/** - * A mock implementation of {@link BoundedByteBufferReceive} that constructs a buffer with the passed in correlation - * id and returns that buffer as part of {@link #getPayload()}. - */ -class MockBoundedByteBufferReceive extends BoundedByteBufferReceive { - private final ByteBuffer buf; - - /** - * Construct a MockBoundedByteBufferReceive with the given correlation id. - * @param buf the ByteBuffer that is the payload of this object. - */ - public MockBoundedByteBufferReceive(ByteBuffer buf) { - this.buf = buf; - } - - /** - * Return the buffer associated with this object. - * @return the buffer associated with this object. - */ - @Override - public ByteBuffer getPayload() { - return buf; - } -} -