-
Notifications
You must be signed in to change notification settings - Fork 263
Add a zero-copy deserializer to gRPC Read #564
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
Merged
mprashanthsagar
merged 9 commits into
GoogleCloudDataproc:master
from
veblush:zero-copy
Jul 28, 2021
Merged
Changes from all commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
77f5859
gRPC=1.39.0
veblush 8d9bdcf
Added zero-copy deserializer
veblush 9baf515
Update by review
veblush f92b381
Reformat
veblush 4500a07
Run google-java-format
veblush 7ba7688
Update by review
veblush bf35aee
Update by review
veblush d19037d
Added tests
veblush 3d80ba1
Merge branch 'master' into zero-copy
mprashanthsagar File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
132 changes: 132 additions & 0 deletions
132
gcsio/src/main/java/com/google/cloud/hadoop/gcsio/ZeroCopyMessageMarshaller.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,132 @@ | ||
| /* | ||
| * Copyright 2021 Google Inc. 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. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package com.google.cloud.hadoop.gcsio; | ||
|
|
||
| import com.google.protobuf.ByteString; | ||
| import com.google.protobuf.CodedInputStream; | ||
| import com.google.protobuf.InvalidProtocolBufferException; | ||
| import com.google.protobuf.MessageLite; | ||
| import com.google.protobuf.Parser; | ||
| import com.google.protobuf.UnsafeByteOperations; | ||
| import io.grpc.Detachable; | ||
| import io.grpc.HasByteBuffer; | ||
| import io.grpc.KnownLength; | ||
| import io.grpc.MethodDescriptor.PrototypeMarshaller; | ||
| import io.grpc.Status; | ||
| import io.grpc.protobuf.lite.ProtoLiteUtils; | ||
| import java.io.IOException; | ||
| import java.io.InputStream; | ||
| import java.nio.ByteBuffer; | ||
| import java.util.ArrayList; | ||
| import java.util.Collections; | ||
| import java.util.IdentityHashMap; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
|
|
||
| /** | ||
| * Custom gRPC marshaller to use zero memory copy feature of gRPC when deserializing messages. This | ||
| * achieves zero-copy by deserializing proto messages pointing to the buffers in the input stream to | ||
| * avoid memory copy so stream should live as long as the message can be referenced. Hence, it | ||
| * exposes the input stream to applications (through popStream) and applications are responsible to | ||
| * close it when it's no longer needed. Otherwise, it'd cause memory leak. | ||
| */ | ||
| class ZeroCopyMessageMarshaller<T extends MessageLite> implements PrototypeMarshaller<T> { | ||
| private Map<T, InputStream> unclosedStreams = | ||
| Collections.synchronizedMap(new IdentityHashMap<>()); | ||
| private final Parser<T> parser; | ||
| private final PrototypeMarshaller<T> marshaller; | ||
|
|
||
| ZeroCopyMessageMarshaller(T defaultInstance) { | ||
|
mprashanthsagar marked this conversation as resolved.
|
||
| parser = (Parser<T>) defaultInstance.getParserForType(); | ||
| marshaller = (PrototypeMarshaller<T>) ProtoLiteUtils.marshaller(defaultInstance); | ||
| } | ||
|
|
||
| @Override | ||
| public Class<T> getMessageClass() { | ||
| return marshaller.getMessageClass(); | ||
| } | ||
|
|
||
| @Override | ||
| public T getMessagePrototype() { | ||
| return marshaller.getMessagePrototype(); | ||
| } | ||
|
|
||
| @Override | ||
| public InputStream stream(T value) { | ||
| return marshaller.stream(value); | ||
| } | ||
|
|
||
| @Override | ||
| public T parse(InputStream stream) { | ||
|
mprashanthsagar marked this conversation as resolved.
|
||
| try { | ||
| if (stream instanceof KnownLength | ||
| && stream instanceof Detachable | ||
| && stream instanceof HasByteBuffer | ||
| && ((HasByteBuffer) stream).byteBufferSupported()) { | ||
| int size = stream.available(); | ||
| // Stream is now detached here and should be closed later. | ||
| stream = ((Detachable) stream).detach(); | ||
| // This mark call is to keep buffer while traversing buffers using skip. | ||
| stream.mark(size); | ||
| List<ByteString> byteStrings = new ArrayList<>(); | ||
| while (stream.available() != 0) { | ||
| ByteBuffer buffer = ((HasByteBuffer) stream).getByteBuffer(); | ||
| byteStrings.add(UnsafeByteOperations.unsafeWrap(buffer)); | ||
| stream.skip(buffer.remaining()); | ||
| } | ||
| stream.reset(); | ||
| CodedInputStream codedInputStream = ByteString.copyFrom(byteStrings).newCodedInput(); | ||
| codedInputStream.enableAliasing(true); | ||
| codedInputStream.setSizeLimit(Integer.MAX_VALUE); | ||
| // fast path (no memory copy) | ||
| T message; | ||
| try { | ||
| message = parseFrom(codedInputStream); | ||
| } catch (InvalidProtocolBufferException ipbe) { | ||
| stream.close(); | ||
| throw Status.INTERNAL | ||
| .withDescription("Invalid protobuf byte sequence") | ||
| .withCause(ipbe) | ||
| .asRuntimeException(); | ||
| } | ||
| unclosedStreams.put(message, stream); | ||
| return message; | ||
| } | ||
| } catch (IOException e) { | ||
| throw new RuntimeException(e); | ||
| } | ||
| // slow path | ||
| return marshaller.parse(stream); | ||
| } | ||
|
|
||
| private T parseFrom(CodedInputStream stream) throws InvalidProtocolBufferException { | ||
| T message = parser.parseFrom(stream); | ||
| try { | ||
| stream.checkLastTagWas(0); | ||
| return message; | ||
| } catch (InvalidProtocolBufferException e) { | ||
| e.setUnfinishedMessage(message); | ||
| throw e; | ||
| } | ||
| } | ||
|
|
||
| // Application needs to call this function to get the stream for the message and | ||
| // call stream.close() function to return it to the pool. | ||
| public InputStream popStream(T message) { | ||
| return unclosedStreams.remove(message); | ||
| } | ||
| } | ||
67 changes: 67 additions & 0 deletions
67
gcsio/src/main/java/com/google/cloud/hadoop/gcsio/ZeroCopyReadinessChecker.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,67 @@ | ||
| /* | ||
| * Copyright 2021 Google Inc. 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. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package com.google.cloud.hadoop.gcsio; | ||
|
|
||
| import com.google.common.flogger.GoogleLogger; | ||
| import com.google.protobuf.MessageLite; | ||
| import io.grpc.KnownLength; | ||
|
|
||
| /** | ||
| * Checker to test whether a zero-copy masharller is available from the versions of gRPC and | ||
| * Protobuf. | ||
| */ | ||
| class ZeroCopyReadinessChecker { | ||
| private static final GoogleLogger logger = GoogleLogger.forEnclosingClass(); | ||
| private static final boolean isZeroCopyReady; | ||
|
|
||
| static { | ||
| // Check whether io.grpc.Detachable exists? | ||
| boolean detachableClassExists = false; | ||
| try { | ||
| // Try to load Detachable interface in the package where KnownLength is in. | ||
| // This can be done directly by looking up io.grpc.Detachable but rather | ||
| // done indirectly to handle the case where gRPC is being shaded in a | ||
| // different package. | ||
| String knownLengthClassName = KnownLength.class.getName(); | ||
| String detachableClassName = | ||
| knownLengthClassName.substring(0, knownLengthClassName.lastIndexOf('.') + 1) | ||
| + "Detachable"; | ||
| Class<?> detachableClass = Class.forName(detachableClassName); | ||
| detachableClassExists = (detachableClass != null); | ||
| } catch (ClassNotFoundException ex) { | ||
|
mprashanthsagar marked this conversation as resolved.
|
||
| logger.atFine().withCause(ex).log("io.grpc.Detachable not found"); | ||
| } | ||
| // Check whether com.google.protobuf.UnsafeByteOperations exists? | ||
| boolean unsafeByteOperationsClassExists = false; | ||
| try { | ||
| // Same above | ||
| String messageLiteClassName = MessageLite.class.getName(); | ||
| String unsafeByteOperationsClassName = | ||
| messageLiteClassName.substring(0, messageLiteClassName.lastIndexOf('.') + 1) | ||
| + "UnsafeByteOperations"; | ||
| Class<?> unsafeByteOperationsClass = Class.forName(unsafeByteOperationsClassName); | ||
| unsafeByteOperationsClassExists = (unsafeByteOperationsClass != null); | ||
| } catch (ClassNotFoundException ex) { | ||
| logger.atFine().withCause(ex).log("com.google.protobuf.UnsafeByteOperations not found"); | ||
| } | ||
| isZeroCopyReady = detachableClassExists && unsafeByteOperationsClassExists; | ||
| } | ||
|
|
||
| public static boolean isReady() { | ||
| return isZeroCopyReady; | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.