[
https://issues.apache.org/jira/browse/HADOOP-18146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17651160#comment-17651160
]
ASF GitHub Bot commented on HADOOP-18146:
-----------------------------------------
anmolanmol1234 commented on code in PR #4039:
URL: https://github.com/apache/hadoop/pull/4039#discussion_r1055218825
##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java:
##########
@@ -314,13 +327,46 @@ public void sendRequest(byte[] buffer, int offset, int
length) throws IOExceptio
if (this.isTraceEnabled) {
startTime = System.nanoTime();
}
- try (OutputStream outputStream = this.connection.getOutputStream()) {
- // update bytes sent before they are sent so we may observe
- // attempted sends as well as successful sends via the
- // accompanying statusCode
+ OutputStream outputStream = null;
+ // Updates the expected bytes to be sent based on length.
+ this.expectedBytesToBeSent = length;
+ try {
+ try {
+ /* Without expect header enabled, if getOutputStream() throws
+ an exception, it gets caught by the restOperation. But with
+ expect header enabled we return back without throwing an exception
+ for the correct response code processing.
+ */
+ outputStream = this.connection.getOutputStream();
+ } catch (IOException e) {
+ /* If getOutputStream fails with an exception and expect header
+ is enabled, we return back without throwing an exception to
+ the caller. The caller is responsible for setting the correct
status code.
+ If expect header is not enabled, we throw back the exception.
+ */
+ String expectHeader = this.connection.getRequestProperty(EXPECT);
+ if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)) {
+ LOG.debug("Getting output stream failed with expect header enabled,
returning back "
+ + ExceptionUtils.getStackTrace(e));
+ return;
Review Comment:
The request was sent to the server with expect header enabled but it didn't
respond back with 100-continue because client didn't meet with the
expectations. So the connection was successfully set and hence it can set all
these properties accordingly.
> ABFS: Add changes for expect hundred continue header with append requests
> -------------------------------------------------------------------------
>
> Key: HADOOP-18146
> URL: https://issues.apache.org/jira/browse/HADOOP-18146
> Project: Hadoop Common
> Issue Type: Sub-task
> Affects Versions: 3.3.1
> Reporter: Anmol Asrani
> Assignee: Anmol Asrani
> Priority: Major
> Labels: pull-request-available
> Time Spent: 2h 10m
> Remaining Estimate: 0h
>
> Heavy load from a Hadoop cluster lead to high resource utilization at FE
> nodes. Investigations from the server side indicate payload buffering at
> Http.Sys as the cause. Payload of requests that eventually fail due to
> throttling limits are also getting buffered, as its triggered before FE could
> start request processing.
> Approach: Client sends Append Http request with Expect header, but holds back
> on payload transmission until server replies back with HTTP 100. We add this
> header for all append requests so as to reduce.
> We made several workload runs with and without hundred continue enabled and
> the overall observation is that :-
> # The ratio of TCP SYN packet count with and without expect hundred continue
> enabled is 0.32 : 3 on average.
> # The ingress into the machine at TCP level is almost 3 times lesser with
> hundred continue enabled which implies a lot of bandwidth save.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]