Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Demo zero copy grpc #966

Draft
wants to merge 4 commits into
base: release-2.5.1
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ public static ClientId valueOf(UUID uuid) {
}

private ClientId(ByteString data) {
super(data);
super(ByteString.copyFrom(data.asReadOnlyByteBuffer()));
}

private ClientId(UUID uuid) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,13 +29,15 @@
import org.apache.ratis.protocol.exceptions.ReadIndexException;
import org.apache.ratis.protocol.exceptions.StateMachineException;
import org.apache.ratis.protocol.exceptions.TransferLeadershipException;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.util.JavaUtils;
import org.apache.ratis.util.Preconditions;
import org.apache.ratis.util.ProtoUtils;
import org.apache.ratis.util.ReflectionUtils;

import java.util.Collection;
import java.util.Collections;
import java.util.stream.Collectors;

/**
* Reply from server to client
Expand Down Expand Up @@ -162,7 +164,16 @@ public static Builder newBuilder() {
this.message = message;
this.exception = exception;
this.logIndex = logIndex;
this.commitInfos = commitInfos != null? commitInfos: Collections.emptyList();
this.commitInfos = commitInfos != null?
commitInfos.stream().map(x -> {
try {
return CommitInfoProto.parseFrom(x.toByteArray());
} catch (
InvalidProtocolBufferException e) {
throw new RuntimeException(e);
}
}).collect(Collectors.toList())
: Collections.emptyList();

if (exception != null) {
Preconditions.assertTrue(!success,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ private RaftGroupId(UUID id) {
}

private RaftGroupId(ByteString data) {
super(data);
super(ByteString.copyFrom(data.asReadOnlyByteBuffer()));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ public final class RaftPeerId {
private static final Map<String, RaftPeerId> STRING_MAP = new ConcurrentHashMap<>();

public static RaftPeerId valueOf(ByteString id) {
return BYTE_STRING_MAP.computeIfAbsent(id, RaftPeerId::new);
return BYTE_STRING_MAP.computeIfAbsent(ByteString.copyFrom(id.asReadOnlyByteBuffer()), RaftPeerId::new);
}

public static RaftPeerId valueOf(String id) {
Expand Down
11 changes: 11 additions & 0 deletions ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java
Original file line number Diff line number Diff line change
Expand Up @@ -229,6 +229,17 @@ static void setAsyncRequestThreadPoolSize(RaftProperties properties, int port) {
setInt(properties::setInt, ASYNC_REQUEST_THREAD_POOL_SIZE_KEY, port);
}

String ZERO_COPY_ENABLED = PREFIX + ".zerocopy.enabled";
boolean ZERO_COPY_ENABLED_DEFAULT = false;
static boolean zeroCopyEnabled(RaftProperties properties) {
return getBoolean(properties::getBoolean, ZERO_COPY_ENABLED,
ZERO_COPY_ENABLED_DEFAULT, getDefaultLog());
}
static void setZeroCopyEnabled(RaftProperties properties, boolean enabled) {
setBoolean(properties::setBoolean, ZERO_COPY_ENABLED, enabled);
}


String TLS_CONF_PARAMETER = PREFIX + ".tls.conf";
Class<GrpcTlsConfig> TLS_CONF_CLASS = TLS.CONF_CLASS;
static GrpcTlsConfig tlsConf(Parameters parameters) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.ratis.grpc.metrics;

import com.codahale.metrics.Gauge;
import org.apache.ratis.metrics.MetricRegistryInfo;
import org.apache.ratis.metrics.RatisMetricRegistry;
import org.apache.ratis.metrics.RatisMetrics;

public class DirectMemoryMetrics extends RatisMetrics {
private static final String RATIS_GRPC_METRICS_APP_NAME = "ratis_grpc";
private static final String RATIS_GRPC_METRICS_COMP_NAME = "direct_memory_disposer";
private static final String RATIS_GRPC_METRICS_DESC = "Metrics for Ratis Grpc direct memory management";

public DirectMemoryMetrics() {
registry = getMetricRegistryForGrpcServer();
}

private RatisMetricRegistry getMetricRegistryForGrpcServer() {
return create(new MetricRegistryInfo("mem",
RATIS_GRPC_METRICS_APP_NAME,
RATIS_GRPC_METRICS_COMP_NAME, RATIS_GRPC_METRICS_DESC));
}


public void onBufferRegistered(long size) {
registry.counter("mem_registered_bytes").inc(size);
}

public void onBufferDisposed(long size) {
registry.counter("mem_disposed_bytes").inc(size);
}

public void watchPendingMemoryBytes(Gauge<Long> pendingMemBytes) {
registry.gauge("mem_pending_bytes", () -> pendingMemBytes);
}

public void watchPendingBuffes(Gauge<Integer> pendingBuffers) {
registry.gauge("pending_buffers", () -> pendingBuffers);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,18 @@

import org.apache.ratis.client.impl.ClientProtoUtils;
import org.apache.ratis.grpc.GrpcUtil;
import org.apache.ratis.grpc.util.ZeroCopyMessageMarshaller;
import org.apache.ratis.grpc.util.ZeroCopyReadinessChecker;
import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto.TypeCase;
import org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc;
import org.apache.ratis.protocol.*;
import org.apache.ratis.protocol.exceptions.AlreadyClosedException;
import org.apache.ratis.protocol.exceptions.GroupMismatchException;
import org.apache.ratis.protocol.exceptions.RaftException;
import org.apache.ratis.server.util.DirectBufferCleaner;
import org.apache.ratis.thirdparty.io.grpc.MethodDescriptor;
import org.apache.ratis.thirdparty.io.grpc.ServerCallHandler;
import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition;
import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto;
Expand All @@ -35,6 +43,7 @@
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
Expand All @@ -45,11 +54,18 @@
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import java.util.function.Supplier;

class GrpcClientProtocolService extends RaftClientProtocolServiceImplBase {
private static final Logger LOG = LoggerFactory.getLogger(GrpcClientProtocolService.class);
private ZeroCopyMessageMarshaller<RaftClientRequestProto> zeroCopyRequestMarshaller =
new ZeroCopyMessageMarshaller<>(RaftClientRequestProto.getDefaultInstance());
private final boolean zeroCopyEnabled;

// private OneCopyMessageMarshaller<RaftClientRequestProto> oneCopyRequestMarshaller =
// new OneCopyMessageMarshaller<>(RaftClientRequestProto.getDefaultInstance());

private static class PendingOrderedRequest implements SlidingWindow.ServerSideRequest<RaftClientReply> {
private final RaftClientRequest request;
Expand Down Expand Up @@ -137,10 +153,11 @@ void closeAllExisting(RaftGroupId groupId) {
private final OrderedStreamObservers orderedStreamObservers = new OrderedStreamObservers();

GrpcClientProtocolService(Supplier<RaftPeerId> idSupplier, RaftClientAsynchronousProtocol protocol,
ExecutorService executor) {
ExecutorService executor, boolean zeroCopyEnabled) {
this.idSupplier = idSupplier;
this.protocol = protocol;
this.executor = executor;
this.zeroCopyEnabled = zeroCopyEnabled;
}

RaftPeerId getId() {
Expand All @@ -164,6 +181,43 @@ public StreamObserver<RaftClientRequestProto> unordered(StreamObserver<RaftClien
return new UnorderedRequestStreamObserver(responseObserver);
}

/**
* Bind this grpc service with zero-copy marshaller for
* ordered and unordered methods if zero-copy is enabled.
*/
public ServerServiceDefinition bindServiceWithZeroCopy() {
ServerServiceDefinition orig = super.bindService();
if (!ZeroCopyReadinessChecker.isReady()) {
LOG.info("Zero copy is not ready.");
return orig;
}
if (!zeroCopyEnabled) {
LOG.info("Zero copy is disabled");
return orig;
}

ServerServiceDefinition.Builder builder =
ServerServiceDefinition.builder(orig.getServiceDescriptor().getName());

addZeroCopyMethod(orig, builder, RaftClientProtocolServiceGrpc.getOrderedMethod());
addZeroCopyMethod(orig, builder, RaftClientProtocolServiceGrpc.getUnorderedMethod());

return builder.build();
}

@SuppressWarnings("unchecked")
private void addZeroCopyMethod(ServerServiceDefinition orig,
ServerServiceDefinition.Builder builder,
MethodDescriptor<RaftClientRequestProto, RaftClientReplyProto> origMethod) {
MethodDescriptor<RaftClientRequestProto, RaftClientReplyProto> newMethod = origMethod.toBuilder()
.setRequestMarshaller(zeroCopyRequestMarshaller)
.build();
ServerCallHandler<RaftClientRequestProto, RaftClientReplyProto> serverCallHandler =
(ServerCallHandler<RaftClientRequestProto, RaftClientReplyProto>) orig.getMethod(newMethod.getFullMethodName())
.getServerCallHandler();
builder.addMethod(newMethod, serverCallHandler);
}

private final AtomicInteger streamCount = new AtomicInteger();

private abstract class RequestStreamObserver implements StreamObserver<RaftClientRequestProto> {
Expand Down Expand Up @@ -220,11 +274,11 @@ boolean isClosed() {
return isClosed.get();
}

CompletableFuture<Void> processClientRequest(RaftClientRequest request, Consumer<RaftClientReply> replyHandler) {
CompletableFuture<Void> processClientRequest(RaftClientRequest request, Consumer<RaftClientReply> replyHandler, BiConsumer<RaftClientReply, Throwable> cleaner) {
try {
final String errMsg = LOG.isDebugEnabled() ? "processClientRequest for " + request : "";
return protocol.submitClientRequestAsync(request
).thenAcceptAsync(replyHandler, executor
).whenCompleteAsync(cleaner).thenAcceptAsync(replyHandler, executor
).exceptionally(exception -> {
// TODO: the exception may be from either raft or state machine.
// Currently we skip all the following responses when getting an
Expand All @@ -237,14 +291,23 @@ CompletableFuture<Void> processClientRequest(RaftClientRequest request, Consumer
}
}

abstract void processClientRequest(RaftClientRequest request);
abstract void processClientRequest(RaftClientRequest request, BiConsumer<RaftClientReply, Throwable> cleaner);

@Override
public void onNext(RaftClientRequestProto request) {
InputStream handle = zeroCopyRequestMarshaller.popStream(request);
BiConsumer<RaftClientReply, Throwable> cleaner = (r, e) -> {
if (r != null && r.isSuccess() && request.getTypeCase().equals(TypeCase.WRITE)) {
DirectBufferCleaner.INSTANCE.watch(r, handle);
} else {
org.apache.ratis.thirdparty.io.grpc.internal.GrpcUtil.closeQuietly(handle);
}
};
try {
final RaftClientRequest r = ClientProtoUtils.toRaftClientRequest(request);
processClientRequest(r);
processClientRequest(r, cleaner);
} catch (Exception e) {
org.apache.ratis.thirdparty.io.grpc.internal.GrpcUtil.closeQuietly(handle);
responseError(e, () -> "onNext for " + ClientProtoUtils.toString(request) + " in " + name);
}
}
Expand Down Expand Up @@ -278,14 +341,14 @@ private class UnorderedRequestStreamObserver extends RequestStreamObserver {
}

@Override
void processClientRequest(RaftClientRequest request) {
void processClientRequest(RaftClientRequest request, BiConsumer<RaftClientReply, Throwable> cleaner) {
final CompletableFuture<Void> f = processClientRequest(request, reply -> {
if (!reply.isSuccess()) {
LOG.info("Failed " + request + ", reply=" + reply);
}
final RaftClientReplyProto proto = ClientProtoUtils.toRaftClientReplyProto(reply);
responseNext(proto);
});
}, cleaner);
final long callId = request.getCallId();
put(callId, f);
f.thenAccept(dummy -> remove(callId));
Expand Down Expand Up @@ -327,14 +390,14 @@ RaftGroupId getGroupId() {
return groupId.get();
}

void processClientRequest(PendingOrderedRequest pending) {
void processClientRequest(PendingOrderedRequest pending, BiConsumer<RaftClientReply, Throwable> cleaner) {
final long seq = pending.getSeqNum();
processClientRequest(pending.getRequest(),
reply -> slidingWindow.receiveReply(seq, reply, this::sendReply));
reply -> slidingWindow.receiveReply(seq, reply, this::sendReply), cleaner);
}

@Override
void processClientRequest(RaftClientRequest r) {
void processClientRequest(RaftClientRequest r, BiConsumer<RaftClientReply, Throwable> cleaner) {
if (isClosed()) {
final AlreadyClosedException exception = new AlreadyClosedException(getName() + ": the stream is closed");
responseError(exception, () -> "processClientRequest (stream already closed) for " + r);
Expand All @@ -350,10 +413,11 @@ void processClientRequest(RaftClientRequest r) {
+ ": The group (" + requestGroupId + ") of " + r.getClientId()
+ " does not match the group (" + updated + ") of the " + JavaUtils.getClassSimpleName(getClass()));
responseError(exception, () -> "processClientRequest (Group mismatched) for " + r);
cleaner.accept(null, exception);
return;
}

slidingWindow.receivedRequest(pending, this::processClientRequest);
slidingWindow.receivedRequest(pending, x -> this.processClientRequest(x, cleaner));
}

private void sendReply(PendingOrderedRequest ready) {
Expand Down
Loading
Loading