Skip to content

Implement write path zero copy#8526

Merged
calvinjia merged 4 commits intoAlluxio:masterfrom
bf8086:implement-write-zero-copy
Mar 13, 2019
Merged

Implement write path zero copy#8526
calvinjia merged 4 commits intoAlluxio:masterfrom
bf8086:implement-write-zero-copy

Conversation

@bf8086
Copy link
Contributor

@bf8086 bf8086 commented Mar 6, 2019

Implemented write path zero copy to improve performance on remote and domain socket writes. Also made write handler write to disk asynchronously to avoid blocking network traffic.

@AmplabJenkins
Copy link

Merged build finished. Test PASSed.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Alluxio-Pull-Request-Builder/2331/
Test PASSed.

@bf8086
Copy link
Contributor Author

bf8086 commented Mar 6, 2019

@calvinjia @gpang PTAL.

Copy link
Contributor

@gpang gpang left a comment

Choose a reason for hiding this comment

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

@bf8086 Thanks! I left a few questions.


@Override
public void readBytes(OutputStream outputStream, int length) {
throw new UnsupportedOperationException("ByteArrayDataBuffer#readBytes is not implemented.");
Copy link
Contributor

Choose a reason for hiding this comment

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

Why are these not supported?

Can we distinguish the 2 calls from the exception messages?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This class is only used in unit tests and only one main function is implemented. It's not used in write path so I didn't spend any time completing this class.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Message updated.

Copy link
Contributor

Choose a reason for hiding this comment

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

If this is just a test class, then can we move it to a test directory? It is not obvious that this is only used for tests.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

/**
* Appends buffer.readableBytes() bytes to the end of this block writer from the given buffer.
*
* @param buffer the byte buffer to hold the data
Copy link
Contributor

Choose a reason for hiding this comment

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

is this the buffer that holds the data currently? to hold the data makes is sound like this is the buffer that is the output?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah I copied the javadoc from an overloaded method. Updated.

return null;
}
DataBuffer buffer = mMarshaller.pollBuffer(response);
DataBuffer buffer = mResponseMarshaller.pollBuffer(response);
Copy link
Contributor

Choose a reason for hiding this comment

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

mResponseMarshaller can be null?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

User should not call this method if they do not have a response marshaller. I will add a check.

new Builder(Name.WORKER_NETWORK_WRITER_BUFFER_SIZE_MESSAGES)
.setDefaultValue(8)
.setDescription("When a client writes to a remote worker, the maximum number of "
+ "data messages to buffer by the server.")
Copy link
Contributor

Choose a reason for hiding this comment

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

this is what the worker buffers, per client?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated the description.

* Gets a Netty buffer directly from a gRPC ReadableBuffer.
*
* @param buffer the input buffer
* @return the raw ByteBuf
Copy link
Contributor

Choose a reason for hiding this comment

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

when will this return null?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated javadoc.

DataMessageMarshaller<WriteRequest> marshaller =
((DataMessageMarshallerProvider<WriteRequest, WriteResponse>) responseObserver)
.getRequestMarshaller();
Preconditions.checkNotNull(marshaller);
Copy link
Contributor

Choose a reason for hiding this comment

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

Doesn't a DataMessageMarshallerProvider provide null sometimes?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

DataMessageMarshallerProvider is a generic holder of request marshaller and response marshaller. Depending on the situation, the caller most likely only provide one of the marshallers. In this writeBlock method, it is required that the user should provide the request marshaller for zero copy to work correctly, hence the check.

If a user does not want to use zero copy at all, they should not pass in a DataMessageMarshallerProvider.

((DataMessageMarshallerProvider<ReadResponse>) responseObserver).getMarshaller();
((DataMessageMarshallerProvider<ReadRequest, ReadResponse>) responseObserver)
.getResponseMarshaller();
Preconditions.checkNotNull(marshaller);
Copy link
Contributor

Choose a reason for hiding this comment

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

can't this be null sometimes?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Similar to above, if user does not want to use zero copy for reading blocks, they should not pass DataMessageMarshallerProvider instead of giving a null response marshaller.

chunkBuffer = new NettyDataBuffer(
Unpooled.wrappedBuffer(message.getChunk().getData().asReadOnlyByteBuffer()));
}
int size = message.getSerializedSize() - chunkBuffer.readableBytes();
Copy link
Contributor

Choose a reason for hiding this comment

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

NIT: is this header size?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Renamed.

private static final Logger LOG = LoggerFactory.getLogger(AbstractWriteHandler.class);

private final StreamObserver<WriteResponse> mResponseObserver;
private final SerializingExecutor mSerializingExecutor;
Copy link
Contributor

Choose a reason for hiding this comment

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

can you add comments on these fields? I don't fully understand them.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

@bf8086
Copy link
Contributor Author

bf8086 commented Mar 7, 2019

@gpang Thanks for the review. I updated the PR to address your comments. PTAL.

@AmplabJenkins
Copy link

Merged build finished. Test FAILed.

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Alluxio-Pull-Request-Builder/2367/
Test FAILed.

@gpang
Copy link
Contributor

gpang commented Mar 7, 2019

Jenkins, test this please

@AmplabJenkins
Copy link

Merged build finished. Test PASSed.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Alluxio-Pull-Request-Builder/2374/
Test PASSed.

Copy link
Contributor

@gpang gpang left a comment

Choose a reason for hiding this comment

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

@bf8086 I left a few comments.

*/
AbstractWriteHandler(StreamObserver<WriteResponse> responseObserver) {
mResponseObserver = responseObserver;
mSerializingExecutor = new SerializingExecutor(GrpcExecutors.BLOCK_WRITER_EXECUTOR);
Copy link
Contributor

Choose a reason for hiding this comment

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

the serializing executor only runs 1 task at a time, in order. Why does it need a large threadpool?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It only serialize tasks within the executor for a single request. There can be multiple concurrent requests which requires more threads for processing messages concurrently.

Preconditions.checkState(buffer.readableBytes() > 0,
"invalid data size from write request message");
try {
mSemaphore.acquire();
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe there should be a helper method that does this acquire?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated.

Thread.currentThread().interrupt();
return;
}
mSerializingExecutor.execute(() -> {
Copy link
Contributor

Choose a reason for hiding this comment

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

I feel like every serializing executor should follow the model:

try {
  ...
} finally {
  sem.release()
}

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good catch. Fixed.

@AmplabJenkins
Copy link

Merged build finished. Test PASSed.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Alluxio-Pull-Request-Builder/2375/
Test PASSed.

@bf8086
Copy link
Contributor Author

bf8086 commented Mar 7, 2019

Thanks @gpang! I updated the PR. PTAL.

@bf8086
Copy link
Contributor Author

bf8086 commented Mar 8, 2019

@gpang I updated the PR to address your comment. PTAL.

@AmplabJenkins
Copy link

Merged build finished. Test PASSed.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/Alluxio-Pull-Request-Builder/2385/
Test PASSed.

Copy link
Contributor

@gpang gpang left a comment

Choose a reason for hiding this comment

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

LGTM

if (!tryAcquireSemaphore()) {
return;
}
mSerializingExecutor.execute(() -> {
Copy link
Contributor

Choose a reason for hiding this comment

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

What will happen if client tries zero copy but the server does not support it?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The zero copy code path serialize and deserialize messages in the same format as the protobuf marshaller, therefore the client and the server can switch between zero copy or protobuf marshaller without breaking compatibility with the other side.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants