diff --git a/pom.xml b/pom.xml index b0d7809f7cffe..78a5921257f82 100644 --- a/pom.xml +++ b/pom.xml @@ -735,6 +735,12 @@ ${dep.drift.version} + + com.facebook.drift + drift-transport-spi + ${dep.drift.version} + + io.airlift.tpch tpch @@ -1311,6 +1317,12 @@ com.facebook.presto.cassandra cassandra-driver 3.1.4-1 + + + io.netty + * + + diff --git a/presto-elasticsearch/pom.xml b/presto-elasticsearch/pom.xml index 6ba3086415df8..86610f9e34832 100644 --- a/presto-elasticsearch/pom.xml +++ b/presto-elasticsearch/pom.xml @@ -87,6 +87,10 @@ org.elasticsearch.plugin transport-netty4-client + + io.netty + * + @@ -157,6 +161,12 @@ org.elasticsearch.plugin transport-netty4-client + + + io.netty + * + + ${dep.elasticsearch.version} diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveDistributedQueriesWithThriftRpc.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveDistributedQueriesWithThriftRpc.java new file mode 100644 index 0000000000000..be12c3ab356ad --- /dev/null +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveDistributedQueriesWithThriftRpc.java @@ -0,0 +1,51 @@ +/* + * 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.facebook.presto.hive; + +import com.facebook.presto.tests.AbstractTestDistributedQueries; +import com.google.common.collect.ImmutableMap; + +import java.util.Optional; + +import static com.facebook.presto.hive.HiveQueryRunner.createQueryRunner; +import static io.airlift.tpch.TpchTable.getTables; + +public class TestHiveDistributedQueriesWithThriftRpc + extends AbstractTestDistributedQueries +{ + public TestHiveDistributedQueriesWithThriftRpc() + { + super(() -> createQueryRunner( + getTables(), + ImmutableMap.of( + "internal-communication.task-communication-protocol", "THRIFT", + "internal-communication.server-info-communication-protocol", "THRIFT"), + ImmutableMap.of(), + Optional.empty())); + } + + @Override + protected boolean supportsNotNullColumns() + { + return false; + } + + @Override + public void testDelete() + { + // Hive connector currently does not support row-by-row delete + } + + // Hive specific tests should normally go in TestHiveIntegrationSmokeTest +} diff --git a/presto-main/pom.xml b/presto-main/pom.xml index f2314b2cad332..9aea0928a7376 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -196,6 +196,36 @@ joda-to-java-time-bridge + + com.facebook.drift + drift-server + + + + com.facebook.drift + drift-transport-netty + + + + com.facebook.drift + drift-transport-spi + + + + com.facebook.drift + drift-client + + + + com.facebook.drift + drift-codec + + + + com.facebook.drift + drift-api + + com.teradata re2j-td diff --git a/presto-main/src/main/java/com/facebook/presto/execution/LocationFactory.java b/presto-main/src/main/java/com/facebook/presto/execution/LocationFactory.java index e61f2b62f6374..2644e2cb58683 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/LocationFactory.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/LocationFactory.java @@ -26,6 +26,16 @@ public interface LocationFactory URI createLocalTaskLocation(TaskId taskId); + /** + * TODO: this method is required since not not all RPC call is supported by thrift. + * It should be merged into {@code createTaskLocation} once full thrift support is in-place for v1/task + */ + @Deprecated + URI createLegacyTaskLocation(InternalNode node, TaskId taskId); + + /** + * TODO: implement full thrift support for v1/task + */ URI createTaskLocation(InternalNode node, TaskId taskId); URI createMemoryInfoLocation(InternalNode node); diff --git a/presto-main/src/main/java/com/facebook/presto/execution/RemoteTask.java b/presto-main/src/main/java/com/facebook/presto/execution/RemoteTask.java index 32358eddf01f0..73ea659501e29 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/RemoteTask.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/RemoteTask.java @@ -20,6 +20,8 @@ import com.google.common.collect.Multimap; import com.google.common.util.concurrent.ListenableFuture; +import java.net.URI; + public interface RemoteTask { TaskId getTaskId(); @@ -30,6 +32,11 @@ public interface RemoteTask TaskStatus getTaskStatus(); + /** + * TODO: this should be merged into getTaskStatus once full thrift support is in-place for v1/task + */ + URI getRemoteTaskLocation(); + void start(); void addSplits(Multimap splits); diff --git a/presto-main/src/main/java/com/facebook/presto/execution/SqlStageExecution.java b/presto-main/src/main/java/com/facebook/presto/execution/SqlStageExecution.java index 8e480a61ed424..a6e7e12b99af2 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/SqlStageExecution.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/SqlStageExecution.java @@ -365,7 +365,7 @@ public synchronized void addExchangeLocations(PlanFragmentId fragmentId, Set newSplits = ImmutableMultimap.builder(); for (RemoteTask sourceTask : sourceTasks) { TaskStatus sourceTaskStatus = sourceTask.getTaskStatus(); - newSplits.put(remoteSource.getId(), createRemoteSplitFor(task.getTaskId(), sourceTaskStatus.getSelf(), sourceTaskStatus.getTaskId())); + newSplits.put(remoteSource.getId(), createRemoteSplitFor(task.getTaskId(), sourceTask.getRemoteTaskLocation(), sourceTaskStatus.getTaskId())); } task.addSplits(newSplits.build()); } @@ -491,7 +491,7 @@ private synchronized RemoteTask scheduleTask(InternalNode node, TaskId taskId, M sourceTasks.forEach((planNodeId, task) -> { TaskStatus status = task.getTaskStatus(); if (status.getState() != TaskState.FINISHED) { - initialSplits.put(planNodeId, createRemoteSplitFor(taskId, status.getSelf(), status.getTaskId())); + initialSplits.put(planNodeId, createRemoteSplitFor(taskId, task.getRemoteTaskLocation(), status.getTaskId())); } }); diff --git a/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionId.java b/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionId.java index 2f309c18deefa..ff9920c40855f 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionId.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/StageExecutionId.java @@ -13,6 +13,9 @@ */ package com.facebook.presto.execution; +import com.facebook.drift.annotations.ThriftConstructor; +import com.facebook.drift.annotations.ThriftField; +import com.facebook.drift.annotations.ThriftStruct; import com.facebook.presto.spi.QueryId; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; @@ -25,6 +28,7 @@ import static java.lang.Integer.parseInt; import static java.util.Objects.requireNonNull; +@ThriftStruct public class StageExecutionId { private final StageId stageId; @@ -42,17 +46,20 @@ public static StageExecutionId valueOf(List ids) return new StageExecutionId(new StageId(new QueryId(ids.get(0)), parseInt(ids.get(1))), parseInt(ids.get(2))); } + @ThriftConstructor public StageExecutionId(StageId stageId, int id) { this.stageId = requireNonNull(stageId, "stageId is null"); this.id = id; } + @ThriftField(1) public StageId getStageId() { return stageId; } + @ThriftField(2) public int getId() { return id; diff --git a/presto-main/src/main/java/com/facebook/presto/execution/StageId.java b/presto-main/src/main/java/com/facebook/presto/execution/StageId.java index 2be58256568fa..bf01c0143718c 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/StageId.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/StageId.java @@ -13,6 +13,9 @@ */ package com.facebook.presto.execution; +import com.facebook.drift.annotations.ThriftConstructor; +import com.facebook.drift.annotations.ThriftField; +import com.facebook.drift.annotations.ThriftStruct; import com.facebook.presto.spi.QueryId; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; @@ -23,6 +26,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; +@ThriftStruct public class StageId { @JsonCreator @@ -47,11 +51,24 @@ public StageId(QueryId queryId, int id) this.id = id; } + @ThriftConstructor + public StageId(String queryId, int id) + { + this(QueryId.valueOf(queryId), id); + } + public QueryId getQueryId() { return queryId; } + @ThriftField(value = 1, name = "queryId") + public String getQueryIdString() + { + return queryId.toString(); + } + + @ThriftField(2) public int getId() { return id; diff --git a/presto-main/src/main/java/com/facebook/presto/execution/TaskId.java b/presto-main/src/main/java/com/facebook/presto/execution/TaskId.java index 805bbc054d039..968c43cd801a4 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/TaskId.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/TaskId.java @@ -13,6 +13,9 @@ */ package com.facebook.presto.execution; +import com.facebook.drift.annotations.ThriftConstructor; +import com.facebook.drift.annotations.ThriftField; +import com.facebook.drift.annotations.ThriftStruct; import com.facebook.presto.spi.QueryId; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; @@ -25,6 +28,7 @@ import static java.lang.Integer.parseInt; import static java.util.Objects.requireNonNull; +@ThriftStruct public class TaskId { private final StageExecutionId stageExecutionId; @@ -42,6 +46,7 @@ public TaskId(String queryId, int stageId, int stageExecutionId, int id) this(new StageExecutionId(new StageId(new QueryId(queryId), stageId), stageExecutionId), id); } + @ThriftConstructor public TaskId(StageExecutionId stageExecutionId, int id) { this.stageExecutionId = requireNonNull(stageExecutionId, "stageExecutionId"); @@ -49,11 +54,13 @@ public TaskId(StageExecutionId stageExecutionId, int id) this.id = id; } + @ThriftField(1) public StageExecutionId getStageExecutionId() { return stageExecutionId; } + @ThriftField(2) public int getId() { return id; diff --git a/presto-main/src/main/java/com/facebook/presto/execution/buffer/BufferResult.java b/presto-main/src/main/java/com/facebook/presto/execution/buffer/BufferResult.java index 2ca0ae62fd107..1730de1b02129 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/buffer/BufferResult.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/buffer/BufferResult.java @@ -13,6 +13,9 @@ */ package com.facebook.presto.execution.buffer; +import com.facebook.drift.annotations.ThriftConstructor; +import com.facebook.drift.annotations.ThriftField; +import com.facebook.drift.annotations.ThriftStruct; import com.google.common.collect.ImmutableList; import java.util.List; @@ -23,6 +26,7 @@ import static com.google.common.base.Strings.isNullOrEmpty; import static java.util.Objects.requireNonNull; +@ThriftStruct public class BufferResult { public static BufferResult emptyResults(String taskInstanceId, long token, boolean bufferComplete) @@ -36,6 +40,7 @@ public static BufferResult emptyResults(String taskInstanceId, long token, boole private final boolean bufferComplete; private final List serializedPages; + @ThriftConstructor public BufferResult(String taskInstanceId, long token, long nextToken, boolean bufferComplete, List serializedPages) { checkArgument(!isNullOrEmpty(taskInstanceId), "taskInstanceId is null"); @@ -47,21 +52,31 @@ public BufferResult(String taskInstanceId, long token, long nextToken, boolean b this.serializedPages = ImmutableList.copyOf(requireNonNull(serializedPages, "serializedPages is null")); } + @ThriftField(1) + public String getTaskInstanceId() + { + return taskInstanceId; + } + + @ThriftField(2) public long getToken() { return token; } + @ThriftField(3) public long getNextToken() { return nextToken; } + @ThriftField(4) public boolean isBufferComplete() { return bufferComplete; } + @ThriftField(5) public List getSerializedPages() { return serializedPages; @@ -77,11 +92,6 @@ public boolean isEmpty() return serializedPages.isEmpty(); } - public String getTaskInstanceId() - { - return taskInstanceId; - } - @Override public boolean equals(Object o) { diff --git a/presto-main/src/main/java/com/facebook/presto/execution/buffer/OutputBuffers.java b/presto-main/src/main/java/com/facebook/presto/execution/buffer/OutputBuffers.java index 7a7b73472197f..9b29fae7dc661 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/buffer/OutputBuffers.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/buffer/OutputBuffers.java @@ -13,6 +13,9 @@ */ package com.facebook.presto.execution.buffer; +import com.facebook.drift.annotations.ThriftConstructor; +import com.facebook.drift.annotations.ThriftField; +import com.facebook.drift.annotations.ThriftStruct; import com.facebook.presto.sql.planner.PartitioningHandle; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -248,6 +251,7 @@ private void checkHasBuffer(OutputBufferId bufferId, int partition) partition); } + @ThriftStruct public static class OutputBufferId { // this is needed by JAX-RS @@ -258,6 +262,7 @@ public static OutputBufferId fromString(String id) private final int id; + @ThriftConstructor @JsonCreator public OutputBufferId(int id) { @@ -278,6 +283,7 @@ public boolean equals(Object o) return id == that.id; } + @ThriftField(1) @JsonValue public int getId() { diff --git a/presto-main/src/main/java/com/facebook/presto/execution/buffer/SerializedPage.java b/presto-main/src/main/java/com/facebook/presto/execution/buffer/SerializedPage.java index c1c1e8297bd87..a3db57486f45b 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/buffer/SerializedPage.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/buffer/SerializedPage.java @@ -13,15 +13,21 @@ */ package com.facebook.presto.execution.buffer; +import com.facebook.drift.annotations.ThriftConstructor; +import com.facebook.drift.annotations.ThriftField; +import com.facebook.drift.annotations.ThriftStruct; import io.airlift.slice.Slice; +import io.airlift.slice.Slices; import org.openjdk.jol.info.ClassLayout; import static com.facebook.presto.execution.buffer.PageCodecMarker.COMPRESSED; import static com.facebook.presto.execution.buffer.PageCodecMarker.ENCRYPTED; import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; +@ThriftStruct public class SerializedPage { private static final int INSTANCE_SIZE = ClassLayout.parseClass(SerializedPage.class).instanceSize(); @@ -34,6 +40,7 @@ public class SerializedPage public SerializedPage(Slice slice, byte pageCodecMarkers, int positionCount, int uncompressedSizeInBytes) { this.slice = requireNonNull(slice, "slice is null"); + checkArgument(slice.getBase() == null || slice.getBase() instanceof byte[], "serialization type only supports byte[]"); this.positionCount = positionCount; checkArgument(uncompressedSizeInBytes >= 0, "uncompressedSizeInBytes is negative"); this.uncompressedSizeInBytes = uncompressedSizeInBytes; @@ -49,34 +56,56 @@ public SerializedPage(Slice slice, byte pageCodecMarkers, int positionCount, int } } - public int getSizeInBytes() + @ThriftConstructor + public SerializedPage(byte[] data, byte pageCodecMarkers, int positionCount, int uncompressedSizeInBytes) { - return slice.length(); + this(Slices.wrappedBuffer(data), pageCodecMarkers, positionCount, uncompressedSizeInBytes); } - public int getUncompressedSizeInBytes() + @ThriftField(1) + public byte[] getData() { - return uncompressedSizeInBytes; + if (slice.isCompact()) { + Object base = slice.getBase(); + checkState(base instanceof byte[], "unexpected serialization type %s", base.getClass()); + return (byte[]) base; + } + + // do a copy + return slice.getBytes(); } - public long getRetainedSizeInBytes() + @ThriftField(2) + public byte getPageCodecMarkers() { - return INSTANCE_SIZE + slice.getRetainedSize(); + return pageCodecMarkers; } + @ThriftField(3) public int getPositionCount() { return positionCount; } - public Slice getSlice() + @ThriftField(4) + public int getUncompressedSizeInBytes() { - return slice; + return uncompressedSizeInBytes; } - public byte getPageCodecMarkers() + public int getSizeInBytes() { - return pageCodecMarkers; + return slice.length(); + } + + public long getRetainedSizeInBytes() + { + return INSTANCE_SIZE + slice.getRetainedSize(); + } + + public Slice getSlice() + { + return slice; } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/DiscoveryNodeManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/DiscoveryNodeManager.java index 272445019580f..ef7897cb90005 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/DiscoveryNodeManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/DiscoveryNodeManager.java @@ -19,10 +19,13 @@ import com.facebook.airlift.http.client.HttpClient; import com.facebook.airlift.log.Logger; import com.facebook.airlift.node.NodeInfo; +import com.facebook.drift.client.DriftClient; import com.facebook.presto.client.NodeVersion; import com.facebook.presto.connector.system.GlobalSystemConnector; import com.facebook.presto.failureDetector.FailureDetector; import com.facebook.presto.server.InternalCommunicationConfig; +import com.facebook.presto.server.InternalCommunicationConfig.CommunicationProtocol; +import com.facebook.presto.server.thrift.ThriftServerInfoClient; import com.facebook.presto.spi.ConnectorId; import com.facebook.presto.spi.NodeState; import com.google.common.base.Splitter; @@ -45,6 +48,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; +import java.util.OptionalInt; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; @@ -77,10 +81,12 @@ public final class DiscoveryNodeManager private final NodeVersion expectedNodeVersion; private final ConcurrentHashMap nodeStates = new ConcurrentHashMap<>(); private final HttpClient httpClient; + private final DriftClient driftClient; private final ScheduledExecutorService nodeStateUpdateExecutor; private final ExecutorService nodeStateEventExecutor; private final boolean httpsRequired; private final InternalNode currentNode; + private final CommunicationProtocol protocol; @GuardedBy("this") private SetMultimap activeNodesByConnectorId; @@ -101,12 +107,14 @@ public DiscoveryNodeManager( FailureDetector failureDetector, NodeVersion expectedNodeVersion, @ForNodeManager HttpClient httpClient, + @ForNodeManager DriftClient driftClient, InternalCommunicationConfig internalCommunicationConfig) { this.serviceSelector = requireNonNull(serviceSelector, "serviceSelector is null"); this.failureDetector = requireNonNull(failureDetector, "failureDetector is null"); this.expectedNodeVersion = requireNonNull(expectedNodeVersion, "expectedNodeVersion is null"); this.httpClient = requireNonNull(httpClient, "httpClient is null"); + this.driftClient = requireNonNull(driftClient, "driftClient is null"); this.nodeStateUpdateExecutor = newSingleThreadScheduledExecutor(threadsNamed("node-state-poller-%s")); this.nodeStateEventExecutor = newCachedThreadPool(threadsNamed("node-state-events-%s")); this.httpsRequired = internalCommunicationConfig.isHttpsRequired(); @@ -116,6 +124,7 @@ public DiscoveryNodeManager( requireNonNull(nodeInfo, "nodeInfo is null").getNodeId(), expectedNodeVersion, httpsRequired); + this.protocol = internalCommunicationConfig.getServerInfoCommunicationProtocol(); refreshNodesInternal(); } @@ -124,9 +133,10 @@ private static InternalNode findCurrentNode(List allServices, { for (ServiceDescriptor service : allServices) { URI uri = getHttpUri(service, httpsRequired); + OptionalInt thriftPort = getThriftServerPort(service); NodeVersion nodeVersion = getNodeVersion(service); if (uri != null && nodeVersion != null) { - InternalNode node = new InternalNode(service.getNodeId(), uri, nodeVersion, isCoordinator(service)); + InternalNode node = new InternalNode(service.getNodeId(), uri, thriftPort, nodeVersion, isCoordinator(service)); if (node.getNodeIdentifier().equals(currentNodeId)) { checkState( @@ -174,8 +184,21 @@ private void pollWorkers() // Add new nodes for (InternalNode node : aliveNodes) { - nodeStates.putIfAbsent(node.getNodeIdentifier(), - new RemoteNodeState(httpClient, uriBuilderFrom(node.getInternalUri()).appendPath("/v1/info/state").build())); + switch (protocol) { + case HTTP: + nodeStates.putIfAbsent(node.getNodeIdentifier(), + new HttpRemoteNodeState(httpClient, uriBuilderFrom(node.getInternalUri()).appendPath("/v1/info/state").build())); + break; + case THRIFT: + if (node.getThriftPort().isPresent()) { + nodeStates.put(node.getNodeIdentifier(), + new ThriftRemoteNodeState(driftClient, uriBuilderFrom(node.getInternalUri()).scheme("thrift").port(node.getThriftPort().getAsInt()).build())); + } + else { + // thrift port has not yet been populated; ignore the node for now + } + break; + } } // Schedule refresh @@ -213,10 +236,11 @@ private synchronized void refreshNodesInternal() for (ServiceDescriptor service : services) { URI uri = getHttpUri(service, httpsRequired); + OptionalInt thriftPort = getThriftServerPort(service); NodeVersion nodeVersion = getNodeVersion(service); boolean coordinator = isCoordinator(service); if (uri != null && nodeVersion != null) { - InternalNode node = new InternalNode(service.getNodeId(), uri, nodeVersion, coordinator); + InternalNode node = new InternalNode(service.getNodeId(), uri, thriftPort, nodeVersion, coordinator); NodeState nodeState = getNodeState(node); switch (nodeState) { @@ -381,6 +405,19 @@ private static URI getHttpUri(ServiceDescriptor descriptor, boolean httpsRequire return null; } + private static OptionalInt getThriftServerPort(ServiceDescriptor descriptor) + { + String port = descriptor.getProperties().get("thriftServerPort"); + if (port != null) { + try { + return OptionalInt.of(Integer.parseInt(port)); + } + catch (IllegalArgumentException ignored) { + } + } + return OptionalInt.empty(); + } + private static NodeVersion getNodeVersion(ServiceDescriptor descriptor) { String nodeVersion = descriptor.getProperties().get("node_version"); diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/HttpRemoteNodeState.java b/presto-main/src/main/java/com/facebook/presto/metadata/HttpRemoteNodeState.java new file mode 100644 index 0000000000000..cdbad264471d4 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/metadata/HttpRemoteNodeState.java @@ -0,0 +1,117 @@ +/* + * 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.facebook.presto.metadata; + +import com.facebook.airlift.http.client.FullJsonResponseHandler.JsonResponse; +import com.facebook.airlift.http.client.HttpClient; +import com.facebook.airlift.http.client.HttpClient.HttpResponseFuture; +import com.facebook.airlift.http.client.Request; +import com.facebook.airlift.log.Logger; +import com.facebook.presto.spi.NodeState; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import io.airlift.units.Duration; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.ThreadSafe; + +import java.net.URI; +import java.util.Optional; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import static com.facebook.airlift.http.client.FullJsonResponseHandler.createFullJsonResponseHandler; +import static com.facebook.airlift.http.client.HttpStatus.OK; +import static com.facebook.airlift.http.client.Request.Builder.prepareGet; +import static com.facebook.airlift.json.JsonCodec.jsonCodec; +import static com.google.common.net.MediaType.JSON_UTF_8; +import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static io.airlift.units.Duration.nanosSince; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.SECONDS; +import static javax.ws.rs.core.HttpHeaders.CONTENT_TYPE; + +@ThreadSafe +public class HttpRemoteNodeState + implements RemoteNodeState +{ + private static final Logger log = Logger.get(HttpRemoteNodeState.class); + + private final HttpClient httpClient; + private final URI stateInfoUri; + private final AtomicReference> nodeState = new AtomicReference<>(Optional.empty()); + private final AtomicReference> future = new AtomicReference<>(); + private final AtomicLong lastUpdateNanos = new AtomicLong(); + private final AtomicLong lastWarningLogged = new AtomicLong(); + + public HttpRemoteNodeState(HttpClient httpClient, URI stateInfoUri) + { + this.httpClient = requireNonNull(httpClient, "httpClient is null"); + this.stateInfoUri = requireNonNull(stateInfoUri, "stateInfoUri is null"); + } + + @Override + public Optional getNodeState() + { + return nodeState.get(); + } + + @Override + public synchronized void asyncRefresh() + { + Duration sinceUpdate = nanosSince(lastUpdateNanos.get()); + if (nanosSince(lastWarningLogged.get()).toMillis() > 1_000 && + sinceUpdate.toMillis() > 10_000 && + future.get() != null) { + log.warn("Node state update request to %s has not returned in %s", stateInfoUri, sinceUpdate.toString(SECONDS)); + lastWarningLogged.set(System.nanoTime()); + } + if (sinceUpdate.toMillis() > 1_000 && future.get() == null) { + Request request = prepareGet() + .setUri(stateInfoUri) + .setHeader(CONTENT_TYPE, JSON_UTF_8.toString()) + .build(); + HttpResponseFuture> responseFuture = httpClient.executeAsync(request, createFullJsonResponseHandler(jsonCodec(NodeState.class))); + future.compareAndSet(null, responseFuture); + + Futures.addCallback(responseFuture, new FutureCallback>() + { + @Override + public void onSuccess(@Nullable JsonResponse result) + { + lastUpdateNanos.set(System.nanoTime()); + future.compareAndSet(responseFuture, null); + if (result != null) { + if (result.hasValue()) { + nodeState.set(Optional.ofNullable(result.getValue())); + } + if (result.getStatusCode() != OK.code()) { + log.warn("Error fetching node state from %s returned status %d: %s", stateInfoUri, result.getStatusCode(), result.getStatusMessage()); + return; + } + } + } + + @Override + public void onFailure(Throwable t) + { + log.warn("Error fetching node state from %s: %s", stateInfoUri, t.getMessage()); + lastUpdateNanos.set(System.nanoTime()); + future.compareAndSet(responseFuture, null); + } + }, directExecutor()); + } + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/InternalNode.java b/presto-main/src/main/java/com/facebook/presto/metadata/InternalNode.java index b869e4375b7bb..873340125f853 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/InternalNode.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/InternalNode.java @@ -18,6 +18,7 @@ import com.facebook.presto.spi.Node; import java.net.URI; +import java.util.OptionalInt; import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Strings.emptyToNull; @@ -32,14 +33,21 @@ public class InternalNode { private final String nodeIdentifier; private final URI internalUri; + private final OptionalInt thriftPort; private final NodeVersion nodeVersion; private final boolean coordinator; public InternalNode(String nodeIdentifier, URI internalUri, NodeVersion nodeVersion, boolean coordinator) + { + this(nodeIdentifier, internalUri, OptionalInt.empty(), nodeVersion, coordinator); + } + + public InternalNode(String nodeIdentifier, URI internalUri, OptionalInt thriftPort, NodeVersion nodeVersion, boolean coordinator) { nodeIdentifier = emptyToNull(nullToEmpty(nodeIdentifier).trim()); this.nodeIdentifier = requireNonNull(nodeIdentifier, "nodeIdentifier is null or empty"); this.internalUri = requireNonNull(internalUri, "internalUri is null"); + this.thriftPort = requireNonNull(thriftPort, "thriftPort is null"); this.nodeVersion = requireNonNull(nodeVersion, "nodeVersion is null"); this.coordinator = coordinator; } @@ -63,6 +71,11 @@ public URI getHttpUri() return getInternalUri(); } + public OptionalInt getThriftPort() + { + return thriftPort; + } + public URI getInternalUri() { return internalUri; @@ -116,6 +129,7 @@ public String toString() return toStringHelper(this) .add("nodeIdentifier", nodeIdentifier) .add("internalUri", internalUri) + .add("thriftPort", thriftPort) .add("nodeVersion", nodeVersion) .toString(); } diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/RemoteNodeState.java b/presto-main/src/main/java/com/facebook/presto/metadata/RemoteNodeState.java index 6bb16384c8cd4..56e824ba92ce1 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/RemoteNodeState.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/RemoteNodeState.java @@ -13,102 +13,13 @@ */ package com.facebook.presto.metadata; -import com.facebook.airlift.http.client.FullJsonResponseHandler.JsonResponse; -import com.facebook.airlift.http.client.HttpClient; -import com.facebook.airlift.http.client.HttpClient.HttpResponseFuture; -import com.facebook.airlift.http.client.Request; -import com.facebook.airlift.log.Logger; import com.facebook.presto.spi.NodeState; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import io.airlift.units.Duration; -import javax.annotation.Nullable; -import javax.annotation.concurrent.ThreadSafe; - -import java.net.URI; import java.util.Optional; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; - -import static com.facebook.airlift.http.client.FullJsonResponseHandler.createFullJsonResponseHandler; -import static com.facebook.airlift.http.client.HttpStatus.OK; -import static com.facebook.airlift.http.client.Request.Builder.prepareGet; -import static com.facebook.airlift.json.JsonCodec.jsonCodec; -import static com.google.common.net.MediaType.JSON_UTF_8; -import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.airlift.units.Duration.nanosSince; -import static java.util.Objects.requireNonNull; -import static java.util.concurrent.TimeUnit.SECONDS; -import static javax.ws.rs.core.HttpHeaders.CONTENT_TYPE; -@ThreadSafe -public class RemoteNodeState +public interface RemoteNodeState { - private static final Logger log = Logger.get(RemoteNodeState.class); - - private final HttpClient httpClient; - private final URI stateInfoUri; - private final AtomicReference> nodeState = new AtomicReference<>(Optional.empty()); - private final AtomicReference> future = new AtomicReference<>(); - private final AtomicLong lastUpdateNanos = new AtomicLong(); - private final AtomicLong lastWarningLogged = new AtomicLong(); - - public RemoteNodeState(HttpClient httpClient, URI stateInfoUri) - { - this.httpClient = requireNonNull(httpClient, "httpClient is null"); - this.stateInfoUri = requireNonNull(stateInfoUri, "stateInfoUri is null"); - } - - public Optional getNodeState() - { - return nodeState.get(); - } - - public synchronized void asyncRefresh() - { - Duration sinceUpdate = nanosSince(lastUpdateNanos.get()); - if (nanosSince(lastWarningLogged.get()).toMillis() > 1_000 && - sinceUpdate.toMillis() > 10_000 && - future.get() != null) { - log.warn("Node state update request to %s has not returned in %s", stateInfoUri, sinceUpdate.toString(SECONDS)); - lastWarningLogged.set(System.nanoTime()); - } - if (sinceUpdate.toMillis() > 1_000 && future.get() == null) { - Request request = prepareGet() - .setUri(stateInfoUri) - .setHeader(CONTENT_TYPE, JSON_UTF_8.toString()) - .build(); - HttpResponseFuture> responseFuture = httpClient.executeAsync(request, createFullJsonResponseHandler(jsonCodec(NodeState.class))); - future.compareAndSet(null, responseFuture); - - Futures.addCallback(responseFuture, new FutureCallback>() - { - @Override - public void onSuccess(@Nullable JsonResponse result) - { - lastUpdateNanos.set(System.nanoTime()); - future.compareAndSet(responseFuture, null); - if (result != null) { - if (result.hasValue()) { - nodeState.set(Optional.ofNullable(result.getValue())); - } - if (result.getStatusCode() != OK.code()) { - log.warn("Error fetching node state from %s returned status %d: %s", stateInfoUri, result.getStatusCode(), result.getStatusMessage()); - return; - } - } - } + Optional getNodeState(); - @Override - public void onFailure(Throwable t) - { - log.warn("Error fetching node state from %s: %s", stateInfoUri, t.getMessage()); - lastUpdateNanos.set(System.nanoTime()); - future.compareAndSet(responseFuture, null); - } - }, directExecutor()); - } - } + void asyncRefresh(); } diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/ThriftRemoteNodeState.java b/presto-main/src/main/java/com/facebook/presto/metadata/ThriftRemoteNodeState.java new file mode 100644 index 0000000000000..011ed41bbde49 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/metadata/ThriftRemoteNodeState.java @@ -0,0 +1,90 @@ +/* + * 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.facebook.presto.metadata; + +import com.facebook.drift.client.DriftClient; +import com.facebook.presto.server.thrift.ThriftServerInfoClient; +import com.facebook.presto.spi.NodeState; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.units.Duration; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.ThreadSafe; + +import java.net.URI; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static io.airlift.units.Duration.nanosSince; +import static java.util.Objects.requireNonNull; + +@ThreadSafe +public class ThriftRemoteNodeState + implements RemoteNodeState +{ + private final ThriftServerInfoClient thriftClient; + private final AtomicReference> nodeState = new AtomicReference<>(Optional.empty()); + private final AtomicBoolean requestInflight = new AtomicBoolean(); + private final AtomicLong lastUpdateNanos = new AtomicLong(); + + public ThriftRemoteNodeState(DriftClient thriftClient, URI stateInfoUri) + { + requireNonNull(stateInfoUri, "stateInfoUri is null"); + checkState(stateInfoUri.getScheme().equals("thrift"), "unexpected scheme %s", stateInfoUri.getScheme()); + + this.thriftClient = requireNonNull(thriftClient, "thriftClient is null").get(Optional.of(stateInfoUri.getAuthority())); + } + + @Override + public Optional getNodeState() + { + return nodeState.get(); + } + + @Override + public void asyncRefresh() + { + Duration sinceUpdate = nanosSince(lastUpdateNanos.get()); + + if (sinceUpdate.toMillis() > 1_000 && requestInflight.compareAndSet(false, true)) { + ListenableFuture responseFuture = thriftClient.getServerState(); + + Futures.addCallback(responseFuture, new FutureCallback() + { + @Override + public void onSuccess(@Nullable Integer result) + { + lastUpdateNanos.set(System.nanoTime()); + requestInflight.compareAndSet(true, false); + if (result != null) { + nodeState.set(Optional.of(NodeState.valueOf(result))); + } + } + + @Override + public void onFailure(Throwable t) + { + lastUpdateNanos.set(System.nanoTime()); + requestInflight.compareAndSet(true, false); + } + }, directExecutor()); + } + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/ExchangeClient.java b/presto-main/src/main/java/com/facebook/presto/operator/ExchangeClient.java index 516cb9f437794..134aa6ba6fcbd 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/ExchangeClient.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/ExchangeClient.java @@ -14,12 +14,15 @@ package com.facebook.presto.operator; import com.facebook.airlift.http.client.HttpClient; +import com.facebook.drift.client.DriftClient; import com.facebook.presto.execution.TaskId; import com.facebook.presto.execution.buffer.PageCodecMarker; import com.facebook.presto.execution.buffer.SerializedPage; import com.facebook.presto.memory.context.LocalMemoryContext; -import com.facebook.presto.operator.HttpPageBufferClient.ClientCallback; +import com.facebook.presto.operator.PageBufferClient.ClientCallback; import com.facebook.presto.operator.WorkProcessor.ProcessState; +import com.facebook.presto.server.thrift.ThriftTaskClient; +import com.facebook.presto.spi.PrestoException; import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -37,6 +40,7 @@ import java.util.Deque; import java.util.LinkedList; import java.util.List; +import java.util.Locale; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -46,6 +50,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static com.facebook.presto.spi.block.PageBuilderStatus.DEFAULT_MAX_PAGE_SIZE_IN_BYTES; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -60,7 +65,7 @@ /** * {@link ExchangeClient} is the client on receiver side, used in operators requiring data exchange from other tasks, * such as {@link ExchangeOperator} and {@link MergeOperator}. - * For each sender that ExchangeClient receives data from, a {@link HttpPageBufferClient} is used in ExchangeClient to communicate with the sender, i.e. + * For each sender that ExchangeClient receives data from, a {@link PageBufferClient} is used in ExchangeClient to communicate with the sender, i.e. * *
  *                    /   HttpPageBufferClient_1  - - - Remote Source 1
@@ -82,20 +87,21 @@ public class ExchangeClient
     private final Duration maxErrorDuration;
     private final boolean acknowledgePages;
     private final HttpClient httpClient;
+    private final DriftClient driftClient;
     private final ScheduledExecutorService scheduler;
 
     @GuardedBy("this")
     private boolean noMoreLocations;
 
-    private final ConcurrentMap allClients = new ConcurrentHashMap<>();
+    private final ConcurrentMap allClients = new ConcurrentHashMap<>();
     private final ConcurrentMap taskIdToLocationMap = new ConcurrentHashMap<>();
     private final Set removedRemoteSourceTaskIds = ConcurrentHashMap.newKeySet();
 
     @GuardedBy("this")
-    private final Deque queuedClients = new LinkedList<>();
+    private final Deque queuedClients = new LinkedList<>();
 
-    private final Set completedClients = newConcurrentHashSet();
-    private final Set removedClients = newConcurrentHashSet();
+    private final Set completedClients = newConcurrentHashSet();
+    private final Set removedClients = newConcurrentHashSet();
     private final LinkedBlockingDeque pageBuffer = new LinkedBlockingDeque<>();
 
     @GuardedBy("this")
@@ -126,6 +132,7 @@ public ExchangeClient(
             boolean acknowledgePages,
             double responseSizeExponentialMovingAverageDecayingAlpha,
             HttpClient httpClient,
+            DriftClient driftClient,
             ScheduledExecutorService scheduler,
             LocalMemoryContext systemMemoryContext,
             Executor pageBufferClientCallbackExecutor)
@@ -137,6 +144,7 @@ public ExchangeClient(
         this.maxErrorDuration = maxErrorDuration;
         this.acknowledgePages = acknowledgePages;
         this.httpClient = httpClient;
+        this.driftClient = driftClient;
         this.scheduler = scheduler;
         this.systemMemoryContext = systemMemoryContext;
         this.maxBufferRetainedSizeInBytes = Long.MIN_VALUE;
@@ -150,7 +158,7 @@ public ExchangeClientStatus getStatus()
         // It does not guarantee a consistent view between different exchange clients.
         // Guaranteeing a consistent view introduces significant lock contention.
         ImmutableList.Builder pageBufferClientStatusBuilder = ImmutableList.builder();
-        for (HttpPageBufferClient client : allClients.values()) {
+        for (PageBufferClient client : allClients.values()) {
             pageBufferClientStatusBuilder.add(client.getStatus());
         }
         List pageBufferClientStatus = pageBufferClientStatusBuilder.build();
@@ -185,8 +193,21 @@ public synchronized void addLocation(URI location, TaskId remoteSourceTaskId)
 
         checkState(!noMoreLocations, "No more locations already set");
 
-        HttpPageBufferClient client = new HttpPageBufferClient(
-                httpClient,
+        RpcShuffleClient resultClient;
+        switch (location.getScheme().toLowerCase(Locale.ENGLISH)) {
+            case "http":
+            case "https":
+                resultClient = new HttpRpcShuffleClient(httpClient, location);
+                break;
+            case "thrift":
+                resultClient = new ThriftRpcShuffleClient(driftClient, location);
+                break;
+            default:
+                throw new PrestoException(GENERIC_INTERNAL_ERROR, "unsupported task result client scheme " + location.getScheme());
+        }
+
+        PageBufferClient client = new PageBufferClient(
+                resultClient,
                 maxErrorDuration,
                 acknowledgePages,
                 location,
@@ -216,7 +237,7 @@ public synchronized void removeRemoteSource(TaskId sourceTaskId)
             return;
         }
 
-        HttpPageBufferClient client = allClients.get(location);
+        PageBufferClient client = allClients.get(location);
         if (client == null) {
             return;
         }
@@ -318,7 +339,7 @@ public synchronized void close()
             return;
         }
 
-        for (HttpPageBufferClient client : allClients.values()) {
+        for (PageBufferClient client : allClients.values()) {
             closeQuietly(client);
         }
         pageBuffer.clear();
@@ -360,7 +381,7 @@ public synchronized void scheduleRequestIfNecessary()
         clientCount -= pendingClients;
 
         for (int i = 0; i < clientCount; ) {
-            HttpPageBufferClient client = queuedClients.poll();
+            PageBufferClient client = queuedClients.poll();
             if (client == null) {
                 // no more clients available
                 return;
@@ -426,7 +447,7 @@ private synchronized void notifyBlockedCallers()
         }
     }
 
-    private synchronized void requestComplete(HttpPageBufferClient client)
+    private synchronized void requestComplete(PageBufferClient client)
     {
         if (!queuedClients.contains(client)) {
             queuedClients.add(client);
@@ -434,14 +455,14 @@ private synchronized void requestComplete(HttpPageBufferClient client)
         scheduleRequestIfNecessary();
     }
 
-    private synchronized void clientFinished(HttpPageBufferClient client)
+    private synchronized void clientFinished(PageBufferClient client)
     {
         requireNonNull(client, "client is null");
         completedClients.add(client);
         scheduleRequestIfNecessary();
     }
 
-    private synchronized void clientFailed(HttpPageBufferClient client, Throwable cause)
+    private synchronized void clientFailed(PageBufferClient client, Throwable cause)
     {
         // ignore failure for removed clients
         if (removedClients.contains(client)) {
@@ -474,7 +495,7 @@ private class ExchangeClientCallback
             implements ClientCallback
     {
         @Override
-        public boolean addPages(HttpPageBufferClient client, List pages)
+        public boolean addPages(PageBufferClient client, List pages)
         {
             requireNonNull(client, "client is null");
             requireNonNull(pages, "pages is null");
@@ -482,20 +503,20 @@ public boolean addPages(HttpPageBufferClient client, List pages)
         }
 
         @Override
-        public void requestComplete(HttpPageBufferClient client)
+        public void requestComplete(PageBufferClient client)
         {
             requireNonNull(client, "client is null");
             ExchangeClient.this.requestComplete(client);
         }
 
         @Override
-        public void clientFinished(HttpPageBufferClient client)
+        public void clientFinished(PageBufferClient client)
         {
             ExchangeClient.this.clientFinished(client);
         }
 
         @Override
-        public void clientFailed(HttpPageBufferClient client, Throwable cause)
+        public void clientFailed(PageBufferClient client, Throwable cause)
         {
             requireNonNull(client, "client is null");
             requireNonNull(cause, "cause is null");
@@ -504,7 +525,7 @@ public void clientFailed(HttpPageBufferClient client, Throwable cause)
         }
     }
 
-    private static void closeQuietly(HttpPageBufferClient client)
+    private static void closeQuietly(PageBufferClient client)
     {
         try {
             client.close();
diff --git a/presto-main/src/main/java/com/facebook/presto/operator/ExchangeClientFactory.java b/presto-main/src/main/java/com/facebook/presto/operator/ExchangeClientFactory.java
index d3950865d0280..e4dc7032c010f 100644
--- a/presto-main/src/main/java/com/facebook/presto/operator/ExchangeClientFactory.java
+++ b/presto-main/src/main/java/com/facebook/presto/operator/ExchangeClientFactory.java
@@ -15,7 +15,9 @@
 
 import com.facebook.airlift.concurrent.ThreadPoolExecutorMBean;
 import com.facebook.airlift.http.client.HttpClient;
+import com.facebook.drift.client.DriftClient;
 import com.facebook.presto.memory.context.LocalMemoryContext;
+import com.facebook.presto.server.thrift.ThriftTaskClient;
 import io.airlift.units.DataSize;
 import io.airlift.units.Duration;
 import org.weakref.jmx.Managed;
@@ -41,6 +43,7 @@ public class ExchangeClientFactory
     private final int concurrentRequestMultiplier;
     private final Duration maxErrorDuration;
     private final HttpClient httpClient;
+    private final DriftClient driftClient;
     private final DataSize maxResponseSize;
     private final boolean acknowledgePages;
     private final double responseSizeExponentialMovingAverageDecayingAlpha;
@@ -52,6 +55,7 @@ public class ExchangeClientFactory
     public ExchangeClientFactory(
             ExchangeClientConfig config,
             @ForExchange HttpClient httpClient,
+            @ForExchange DriftClient driftClient,
             @ForExchange ScheduledExecutorService scheduler)
     {
         this(
@@ -63,6 +67,7 @@ public ExchangeClientFactory(
                 config.getPageBufferClientMaxCallbackThreads(),
                 config.getResponseSizeExponentialMovingAverageDecayingAlpha(),
                 httpClient,
+                driftClient,
                 scheduler);
     }
 
@@ -75,6 +80,7 @@ public ExchangeClientFactory(
             int pageBufferClientMaxCallbackThreads,
             double responseSizeExponentialMovingAverageDecayingAlpha,
             HttpClient httpClient,
+            DriftClient driftClient,
             ScheduledExecutorService scheduler)
     {
         this.maxBufferedBytes = requireNonNull(maxBufferedBytes, "maxBufferedBytes is null");
@@ -82,6 +88,7 @@ public ExchangeClientFactory(
         this.maxErrorDuration = requireNonNull(maxErrorDuration, "maxErrorDuration is null");
         this.acknowledgePages = acknowledgePages;
         this.httpClient = requireNonNull(httpClient, "httpClient is null");
+        this.driftClient = requireNonNull(driftClient, "driftClient is null");
 
         // Use only 0.75 of the maxResponseSize to leave room for additional bytes from the encoding
         // TODO figure out a better way to compute the size of data that will be transferred over the network
@@ -126,6 +133,7 @@ public ExchangeClient get(LocalMemoryContext systemMemoryContext)
                 acknowledgePages,
                 responseSizeExponentialMovingAverageDecayingAlpha,
                 httpClient,
+                driftClient,
                 scheduler,
                 systemMemoryContext,
                 pageBufferClientCallbackExecutor);
diff --git a/presto-main/src/main/java/com/facebook/presto/operator/HttpRpcShuffleClient.java b/presto-main/src/main/java/com/facebook/presto/operator/HttpRpcShuffleClient.java
new file mode 100644
index 0000000000000..788afc5d71921
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/operator/HttpRpcShuffleClient.java
@@ -0,0 +1,237 @@
+/*
+ * 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.facebook.presto.operator;
+
+import com.facebook.airlift.http.client.HttpClient;
+import com.facebook.airlift.http.client.HttpStatus;
+import com.facebook.airlift.http.client.Request;
+import com.facebook.airlift.http.client.Response;
+import com.facebook.airlift.http.client.ResponseHandler;
+import com.facebook.airlift.http.client.ResponseTooLargeException;
+import com.facebook.airlift.log.Logger;
+import com.facebook.presto.execution.buffer.SerializedPage;
+import com.facebook.presto.operator.PageBufferClient.PagesResponse;
+import com.google.common.collect.ImmutableList;
+import com.google.common.net.MediaType;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.airlift.slice.InputStreamSliceInput;
+import io.airlift.slice.SliceInput;
+import io.airlift.units.DataSize;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URI;
+import java.util.List;
+
+import static com.facebook.airlift.http.client.HttpStatus.familyForStatusCode;
+import static com.facebook.airlift.http.client.HttpUriBuilder.uriBuilderFrom;
+import static com.facebook.airlift.http.client.Request.Builder.prepareDelete;
+import static com.facebook.airlift.http.client.Request.Builder.prepareGet;
+import static com.facebook.airlift.http.client.ResponseHandlerUtils.propagate;
+import static com.facebook.airlift.http.client.StatusResponseHandler.createStatusResponseHandler;
+import static com.facebook.presto.PrestoMediaTypes.PRESTO_PAGES_TYPE;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_BUFFER_COMPLETE;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_MAX_SIZE;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_PAGE_NEXT_TOKEN;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_PAGE_TOKEN;
+import static com.facebook.presto.client.PrestoHeaders.PRESTO_TASK_INSTANCE_ID;
+import static com.facebook.presto.execution.buffer.PagesSerdeUtil.readSerializedPages;
+import static com.facebook.presto.operator.PageBufferClient.PagesResponse.createEmptyPagesResponse;
+import static com.facebook.presto.operator.PageBufferClient.PagesResponse.createPagesResponse;
+import static com.google.common.net.HttpHeaders.CONTENT_TYPE;
+import static java.lang.String.format;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.Objects.requireNonNull;
+
+@ThreadSafe
+public final class HttpRpcShuffleClient
+        implements RpcShuffleClient
+{
+    private static final Logger log = Logger.get(HttpRpcShuffleClient.class);
+
+    private final HttpClient httpClient;
+    private final URI location;
+
+    public HttpRpcShuffleClient(HttpClient httpClient, URI location)
+    {
+        this.httpClient = requireNonNull(httpClient, "httpClient is null");
+        this.location = requireNonNull(location, "location is null");
+    }
+
+    @Override
+    public ListenableFuture getResults(long token, DataSize maxResponseSize)
+    {
+        URI uri = uriBuilderFrom(location).appendPath(String.valueOf(token)).build();
+        return httpClient.executeAsync(
+                prepareGet()
+                        .setHeader(PRESTO_MAX_SIZE, maxResponseSize.toString())
+                        .setUri(uri).build(),
+                new PageResponseHandler());
+    }
+
+    @Override
+    public void acknowledgeResultsAsync(long nextToken)
+    {
+        URI uri = uriBuilderFrom(location).appendPath(String.valueOf(nextToken)).appendPath("acknowledge").build();
+        httpClient.executeAsync(prepareGet().setUri(uri).build(), new ResponseHandler()
+        {
+            @Override
+            public Void handleException(Request request, Exception exception)
+            {
+                log.debug(exception, "Acknowledge request failed: %s", uri);
+                return null;
+            }
+
+            @Override
+            public Void handle(Request request, Response response)
+            {
+                if (familyForStatusCode(response.getStatusCode()) != HttpStatus.Family.SUCCESSFUL) {
+                    log.debug("Unexpected acknowledge response code: %s", response.getStatusCode());
+                }
+                return null;
+            }
+        });
+    }
+
+    @Override
+    public ListenableFuture abortResults()
+    {
+        return httpClient.executeAsync(prepareDelete().setUri(location).build(), createStatusResponseHandler());
+    }
+
+    @Override
+    public Throwable rewriteException(Throwable throwable)
+    {
+        if (throwable instanceof ResponseTooLargeException) {
+            return new PageTooLargeException(throwable);
+        }
+        return throwable;
+    }
+
+    public static class PageResponseHandler
+            implements ResponseHandler
+    {
+        @Override
+        public PagesResponse handleException(Request request, Exception exception)
+        {
+            throw propagate(request, exception);
+        }
+
+        @Override
+        public PagesResponse handle(Request request, Response response)
+        {
+            try {
+                // no content means no content was created within the wait period, but query is still ok
+                // if job is finished, complete is set in the response
+                if (response.getStatusCode() == HttpStatus.NO_CONTENT.code()) {
+                    return createEmptyPagesResponse(getTaskInstanceId(response), getToken(response), getNextToken(response), getComplete(response));
+                }
+
+                // otherwise we must have gotten an OK response, everything else is considered fatal
+                if (response.getStatusCode() != HttpStatus.OK.code()) {
+                    StringBuilder body = new StringBuilder();
+                    try (BufferedReader reader = new BufferedReader(new InputStreamReader(response.getInputStream(), UTF_8))) {
+                        // Get up to 1000 lines for debugging
+                        for (int i = 0; i < 1000; i++) {
+                            String line = reader.readLine();
+                            // Don't output more than 100KB
+                            if (line == null || body.length() + line.length() > 100 * 1024) {
+                                break;
+                            }
+                            body.append(line + "\n");
+                        }
+                    }
+                    catch (RuntimeException | IOException e) {
+                        // Ignored. Just return whatever message we were able to decode
+                    }
+                    throw new PageTransportErrorException(format("Expected response code to be 200, but was %s %s:%n%s", response.getStatusCode(), response.getStatusMessage(), body.toString()));
+                }
+
+                // invalid content type can happen when an error page is returned, but is unlikely given the above 200
+                String contentType = response.getHeader(CONTENT_TYPE);
+                if (contentType == null) {
+                    throw new PageTransportErrorException(format("%s header is not set: %s", CONTENT_TYPE, response));
+                }
+                if (!mediaTypeMatches(contentType, PRESTO_PAGES_TYPE)) {
+                    throw new PageTransportErrorException(format("Expected %s response from server but got %s", PRESTO_PAGES_TYPE, contentType));
+                }
+
+                String taskInstanceId = getTaskInstanceId(response);
+                long token = getToken(response);
+                long nextToken = getNextToken(response);
+                boolean complete = getComplete(response);
+
+                try (SliceInput input = new InputStreamSliceInput(response.getInputStream())) {
+                    List pages = ImmutableList.copyOf(readSerializedPages(input));
+                    return createPagesResponse(taskInstanceId, token, nextToken, pages, complete);
+                }
+                catch (IOException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+            catch (PageTransportErrorException e) {
+                throw new PageTransportErrorException("Error fetching " + request.getUri().toASCIIString(), e);
+            }
+        }
+
+        private static String getTaskInstanceId(Response response)
+        {
+            String taskInstanceId = response.getHeader(PRESTO_TASK_INSTANCE_ID);
+            if (taskInstanceId == null) {
+                throw new PageTransportErrorException(format("Expected %s header", PRESTO_TASK_INSTANCE_ID));
+            }
+            return taskInstanceId;
+        }
+
+        private static long getToken(Response response)
+        {
+            String tokenHeader = response.getHeader(PRESTO_PAGE_TOKEN);
+            if (tokenHeader == null) {
+                throw new PageTransportErrorException(format("Expected %s header", PRESTO_PAGE_TOKEN));
+            }
+            return Long.parseLong(tokenHeader);
+        }
+
+        private static long getNextToken(Response response)
+        {
+            String nextTokenHeader = response.getHeader(PRESTO_PAGE_NEXT_TOKEN);
+            if (nextTokenHeader == null) {
+                throw new PageTransportErrorException(format("Expected %s header", PRESTO_PAGE_NEXT_TOKEN));
+            }
+            return Long.parseLong(nextTokenHeader);
+        }
+
+        private static boolean getComplete(Response response)
+        {
+            String bufferComplete = response.getHeader(PRESTO_BUFFER_COMPLETE);
+            if (bufferComplete == null) {
+                throw new PageTransportErrorException(format("Expected %s header", PRESTO_BUFFER_COMPLETE));
+            }
+            return Boolean.parseBoolean(bufferComplete);
+        }
+
+        private static boolean mediaTypeMatches(String value, MediaType range)
+        {
+            try {
+                return MediaType.parse(value).is(range);
+            }
+            catch (IllegalArgumentException | IllegalStateException e) {
+                return false;
+            }
+        }
+    }
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/operator/HttpPageBufferClient.java b/presto-main/src/main/java/com/facebook/presto/operator/PageBufferClient.java
similarity index 60%
rename from presto-main/src/main/java/com/facebook/presto/operator/HttpPageBufferClient.java
rename to presto-main/src/main/java/com/facebook/presto/operator/PageBufferClient.java
index a5446adf15629..b3d1ee9cf148c 100644
--- a/presto-main/src/main/java/com/facebook/presto/operator/HttpPageBufferClient.java
+++ b/presto-main/src/main/java/com/facebook/presto/operator/PageBufferClient.java
@@ -13,25 +13,16 @@
  */
 package com.facebook.presto.operator;
 
-import com.facebook.airlift.http.client.HttpClient;
-import com.facebook.airlift.http.client.HttpClient.HttpResponseFuture;
-import com.facebook.airlift.http.client.HttpStatus;
 import com.facebook.airlift.http.client.HttpUriBuilder;
-import com.facebook.airlift.http.client.Request;
-import com.facebook.airlift.http.client.Response;
-import com.facebook.airlift.http.client.ResponseHandler;
-import com.facebook.airlift.http.client.ResponseTooLargeException;
 import com.facebook.airlift.log.Logger;
 import com.facebook.presto.execution.buffer.SerializedPage;
 import com.facebook.presto.server.remotetask.Backoff;
 import com.facebook.presto.spi.PrestoException;
 import com.google.common.base.Ticker;
 import com.google.common.collect.ImmutableList;
-import com.google.common.net.MediaType;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
-import io.airlift.slice.InputStreamSliceInput;
-import io.airlift.slice.SliceInput;
+import com.google.common.util.concurrent.ListenableFuture;
 import io.airlift.units.DataSize;
 import io.airlift.units.Duration;
 import org.joda.time.DateTime;
@@ -40,10 +31,7 @@
 import javax.annotation.concurrent.GuardedBy;
 import javax.annotation.concurrent.ThreadSafe;
 
-import java.io.BufferedReader;
 import java.io.Closeable;
-import java.io.IOException;
-import java.io.InputStreamReader;
 import java.net.URI;
 import java.util.List;
 import java.util.OptionalInt;
@@ -54,21 +42,6 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
-import static com.facebook.airlift.http.client.HttpStatus.familyForStatusCode;
-import static com.facebook.airlift.http.client.Request.Builder.prepareDelete;
-import static com.facebook.airlift.http.client.Request.Builder.prepareGet;
-import static com.facebook.airlift.http.client.ResponseHandlerUtils.propagate;
-import static com.facebook.airlift.http.client.StatusResponseHandler.StatusResponse;
-import static com.facebook.airlift.http.client.StatusResponseHandler.createStatusResponseHandler;
-import static com.facebook.presto.PrestoMediaTypes.PRESTO_PAGES_TYPE;
-import static com.facebook.presto.client.PrestoHeaders.PRESTO_BUFFER_COMPLETE;
-import static com.facebook.presto.client.PrestoHeaders.PRESTO_MAX_SIZE;
-import static com.facebook.presto.client.PrestoHeaders.PRESTO_PAGE_NEXT_TOKEN;
-import static com.facebook.presto.client.PrestoHeaders.PRESTO_PAGE_TOKEN;
-import static com.facebook.presto.client.PrestoHeaders.PRESTO_TASK_INSTANCE_ID;
-import static com.facebook.presto.execution.buffer.PagesSerdeUtil.readSerializedPages;
-import static com.facebook.presto.operator.HttpPageBufferClient.PagesResponse.createEmptyPagesResponse;
-import static com.facebook.presto.operator.HttpPageBufferClient.PagesResponse.createPagesResponse;
 import static com.facebook.presto.spi.HostAddress.fromUri;
 import static com.facebook.presto.spi.StandardErrorCode.REMOTE_BUFFER_CLOSE_FAILED;
 import static com.facebook.presto.spi.StandardErrorCode.REMOTE_TASK_MISMATCH;
@@ -77,18 +50,16 @@
 import static com.google.common.base.MoreObjects.toStringHelper;
 import static com.google.common.base.Preconditions.checkState;
 import static com.google.common.base.Strings.isNullOrEmpty;
-import static com.google.common.net.HttpHeaders.CONTENT_TYPE;
 import static java.lang.String.format;
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Objects.requireNonNull;
 import static java.util.concurrent.TimeUnit.NANOSECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
 
 @ThreadSafe
-public final class HttpPageBufferClient
+public final class PageBufferClient
         implements Closeable
 {
-    private static final Logger log = Logger.get(HttpPageBufferClient.class);
+    private static final Logger log = Logger.get(PageBufferClient.class);
 
     /**
      * For each request, the addPage method will be called zero or more times,
@@ -100,16 +71,16 @@ public final class HttpPageBufferClient
      */
     public interface ClientCallback
     {
-        boolean addPages(HttpPageBufferClient client, List pages);
+        boolean addPages(PageBufferClient client, List pages);
 
-        void requestComplete(HttpPageBufferClient client);
+        void requestComplete(PageBufferClient client);
 
-        void clientFinished(HttpPageBufferClient client);
+        void clientFinished(PageBufferClient client);
 
-        void clientFailed(HttpPageBufferClient client, Throwable cause);
+        void clientFailed(PageBufferClient client, Throwable cause);
     }
 
-    private final HttpClient httpClient;
+    private final RpcShuffleClient resultClient;
     private final boolean acknowledgePages;
     private final URI location;
     private final ClientCallback clientCallback;
@@ -119,7 +90,7 @@ public interface ClientCallback
     @GuardedBy("this")
     private boolean closed;
     @GuardedBy("this")
-    private HttpResponseFuture future;
+    private ListenableFuture future;
     @GuardedBy("this")
     private DateTime lastUpdate = DateTime.now();
     @GuardedBy("this")
@@ -143,8 +114,8 @@ public interface ClientCallback
 
     private final Executor pageBufferClientCallbackExecutor;
 
-    public HttpPageBufferClient(
-            HttpClient httpClient,
+    public PageBufferClient(
+            RpcShuffleClient resultClient,
             Duration maxErrorDuration,
             boolean acknowledgePages,
             URI location,
@@ -152,11 +123,11 @@ public HttpPageBufferClient(
             ScheduledExecutorService scheduler,
             Executor pageBufferClientCallbackExecutor)
     {
-        this(httpClient, maxErrorDuration, acknowledgePages, location, clientCallback, scheduler, Ticker.systemTicker(), pageBufferClientCallbackExecutor);
+        this(resultClient, maxErrorDuration, acknowledgePages, location, clientCallback, scheduler, Ticker.systemTicker(), pageBufferClientCallbackExecutor);
     }
 
-    public HttpPageBufferClient(
-            HttpClient httpClient,
+    public PageBufferClient(
+            RpcShuffleClient resultClient,
             Duration maxErrorDuration,
             boolean acknowledgePages,
             URI location,
@@ -165,7 +136,7 @@ public HttpPageBufferClient(
             Ticker ticker,
             Executor pageBufferClientCallbackExecutor)
     {
-        this.httpClient = requireNonNull(httpClient, "httpClient is null");
+        this.resultClient = requireNonNull(resultClient, "resultClient is null");
         this.acknowledgePages = acknowledgePages;
         this.location = requireNonNull(location, "location is null");
         this.clientCallback = requireNonNull(clientCallback, "clientCallback is null");
@@ -194,10 +165,6 @@ else if (completed) {
         else {
             state = "queued";
         }
-        String httpRequestState = "not scheduled";
-        if (future != null) {
-            httpRequestState = future.getState();
-        }
 
         long rejectedRows = rowsRejected.get();
         int rejectedPages = pagesRejected.get();
@@ -213,7 +180,7 @@ else if (completed) {
                 requestsScheduled.get(),
                 requestsCompleted.get(),
                 requestsFailed.get(),
-                httpRequestState);
+                future == null ? "not scheduled" : "processing request");
     }
 
     public synchronized boolean isRunning()
@@ -265,7 +232,7 @@ public synchronized void scheduleRequest(DataSize maxResponseSize)
             }
             catch (Throwable t) {
                 // should not happen, but be safe and fail the operator
-                clientCallback.clientFailed(HttpPageBufferClient.this, t);
+                clientCallback.clientFailed(PageBufferClient.this, t);
             }
         }, delayNanos, NANOSECONDS);
 
@@ -293,11 +260,7 @@ private synchronized void initiateRequest(DataSize maxResponseSize)
     private synchronized void sendGetResults(DataSize maxResponseSize)
     {
         URI uri = HttpUriBuilder.uriBuilderFrom(location).appendPath(String.valueOf(token)).build();
-        HttpResponseFuture resultFuture = httpClient.executeAsync(
-                prepareGet()
-                        .setHeader(PRESTO_MAX_SIZE, maxResponseSize.toString())
-                        .setUri(uri).build(),
-                new PageResponseHandler());
+        ListenableFuture resultFuture = resultClient.getResults(token, maxResponseSize);
 
         future = resultFuture;
         Futures.addCallback(resultFuture, new FutureCallback()
@@ -312,7 +275,7 @@ public void onSuccess(PagesResponse result)
                 List pages;
                 try {
                     boolean shouldAcknowledge = false;
-                    synchronized (HttpPageBufferClient.this) {
+                    synchronized (PageBufferClient.this) {
                         if (taskInstanceId == null) {
                             taskInstanceId = result.getTaskInstanceId();
                         }
@@ -336,25 +299,7 @@ public void onSuccess(PagesResponse result)
                         // Acknowledge token without handling the response.
                         // The next request will also make sure the token is acknowledged.
                         // This is to fast release the pages on the buffer side.
-                        URI uri = HttpUriBuilder.uriBuilderFrom(location).appendPath(String.valueOf(result.getNextToken())).appendPath("acknowledge").build();
-                        httpClient.executeAsync(prepareGet().setUri(uri).build(), new ResponseHandler()
-                        {
-                            @Override
-                            public Void handleException(Request request, Exception exception)
-                            {
-                                log.debug(exception, "Acknowledge request failed: %s", uri);
-                                return null;
-                            }
-
-                            @Override
-                            public Void handle(Request request, Response response)
-                            {
-                                if (familyForStatusCode(response.getStatusCode()) != HttpStatus.Family.SUCCESSFUL) {
-                                    log.debug("Unexpected acknowledge response code: %s", response.getStatusCode());
-                                }
-                                return null;
-                            }
-                        });
+                        resultClient.acknowledgeResultsAsync(result.getNextToken());
                     }
                 }
                 catch (PrestoException e) {
@@ -367,7 +312,7 @@ public Void handle(Request request, Response response)
                 // clientCallback can keep stats of requests and responses. For example, it may
                 // keep track of how often a client returns empty response and adjust request
                 // frequency or buffer size.
-                if (clientCallback.addPages(HttpPageBufferClient.this, pages)) {
+                if (clientCallback.addPages(PageBufferClient.this, pages)) {
                     pagesReceived.addAndGet(pages.size());
                     rowsReceived.addAndGet(pages.stream().mapToLong(SerializedPage::getPositionCount).sum());
                 }
@@ -376,7 +321,7 @@ public Void handle(Request request, Response response)
                     rowsRejected.addAndGet(pages.stream().mapToLong(SerializedPage::getPositionCount).sum());
                 }
 
-                synchronized (HttpPageBufferClient.this) {
+                synchronized (PageBufferClient.this) {
                     // client is complete, acknowledge it by sending it a delete in the next request
                     if (result.isClientComplete()) {
                         completed = true;
@@ -387,7 +332,7 @@ public Void handle(Request request, Response response)
                     lastUpdate = DateTime.now();
                 }
                 requestsCompleted.incrementAndGet();
-                clientCallback.requestComplete(HttpPageBufferClient.this);
+                clientCallback.requestComplete(PageBufferClient.this);
             }
 
             @Override
@@ -396,7 +341,7 @@ public void onFailure(Throwable t)
                 log.debug("Request to %s failed %s", uri, t);
                 checkNotHoldsLock(this);
 
-                t = rewriteException(t);
+                t = resultClient.rewriteException(t);
                 if (!(t instanceof PrestoException) && backoff.failure()) {
                     String message = format("%s (%s - %s failures, failure duration %s, total failed request time %s)",
                             WORKER_NODE_ERROR,
@@ -413,16 +358,16 @@ public void onFailure(Throwable t)
 
     private synchronized void sendDelete()
     {
-        HttpResponseFuture resultFuture = httpClient.executeAsync(prepareDelete().setUri(location).build(), createStatusResponseHandler());
+        ListenableFuture resultFuture = resultClient.abortResults();
         future = resultFuture;
-        Futures.addCallback(resultFuture, new FutureCallback()
+        Futures.addCallback(resultFuture, new FutureCallback()
         {
             @Override
-            public void onSuccess(@Nullable StatusResponse result)
+            public void onSuccess(@Nullable Object result)
             {
                 checkNotHoldsLock(this);
                 backoff.success();
-                synchronized (HttpPageBufferClient.this) {
+                synchronized (PageBufferClient.this) {
                     closed = true;
                     if (future == resultFuture) {
                         future = null;
@@ -430,7 +375,7 @@ public void onSuccess(@Nullable StatusResponse result)
                     lastUpdate = DateTime.now();
                 }
                 requestsCompleted.incrementAndGet();
-                clientCallback.clientFinished(HttpPageBufferClient.this);
+                clientCallback.clientFinished(PageBufferClient.this);
             }
 
             @Override
@@ -457,7 +402,7 @@ private static void checkNotHoldsLock(Object lock)
         checkState(!Thread.holdsLock(lock), "Cannot execute this method while holding a lock");
     }
 
-    private void handleFailure(Throwable t, HttpResponseFuture expectedFuture)
+    private void handleFailure(Throwable t, ListenableFuture expectedFuture)
     {
         // Can not delegate to other callback while holding a lock on this
         checkNotHoldsLock(this);
@@ -466,16 +411,16 @@ private void handleFailure(Throwable t, HttpResponseFuture expectedFuture)
         requestsCompleted.incrementAndGet();
 
         if (t instanceof PrestoException) {
-            clientCallback.clientFailed(HttpPageBufferClient.this, t);
+            clientCallback.clientFailed(PageBufferClient.this, t);
         }
 
-        synchronized (HttpPageBufferClient.this) {
+        synchronized (PageBufferClient.this) {
             if (future == expectedFuture) {
                 future = null;
             }
             lastUpdate = DateTime.now();
         }
-        clientCallback.requestComplete(HttpPageBufferClient.this);
+        clientCallback.requestComplete(PageBufferClient.this);
     }
 
     @Override
@@ -488,7 +433,7 @@ public boolean equals(Object o)
             return false;
         }
 
-        HttpPageBufferClient that = (HttpPageBufferClient) o;
+        PageBufferClient that = (PageBufferClient) o;
 
         if (!location.equals(that.location)) {
             return false;
@@ -524,127 +469,6 @@ else if (future != null) {
                 .toString();
     }
 
-    private static Throwable rewriteException(Throwable t)
-    {
-        if (t instanceof ResponseTooLargeException) {
-            return new PageTooLargeException();
-        }
-        return t;
-    }
-
-    public static class PageResponseHandler
-            implements ResponseHandler
-    {
-        @Override
-        public PagesResponse handleException(Request request, Exception exception)
-        {
-            throw propagate(request, exception);
-        }
-
-        @Override
-        public PagesResponse handle(Request request, Response response)
-        {
-            try {
-                // no content means no content was created within the wait period, but query is still ok
-                // if job is finished, complete is set in the response
-                if (response.getStatusCode() == HttpStatus.NO_CONTENT.code()) {
-                    return createEmptyPagesResponse(getTaskInstanceId(response), getToken(response), getNextToken(response), getComplete(response));
-                }
-
-                // otherwise we must have gotten an OK response, everything else is considered fatal
-                if (response.getStatusCode() != HttpStatus.OK.code()) {
-                    StringBuilder body = new StringBuilder();
-                    try (BufferedReader reader = new BufferedReader(new InputStreamReader(response.getInputStream(), UTF_8))) {
-                        // Get up to 1000 lines for debugging
-                        for (int i = 0; i < 1000; i++) {
-                            String line = reader.readLine();
-                            // Don't output more than 100KB
-                            if (line == null || body.length() + line.length() > 100 * 1024) {
-                                break;
-                            }
-                            body.append(line + "\n");
-                        }
-                    }
-                    catch (RuntimeException | IOException e) {
-                        // Ignored. Just return whatever message we were able to decode
-                    }
-                    throw new PageTransportErrorException(format("Expected response code to be 200, but was %s %s:%n%s", response.getStatusCode(), response.getStatusMessage(), body.toString()));
-                }
-
-                // invalid content type can happen when an error page is returned, but is unlikely given the above 200
-                String contentType = response.getHeader(CONTENT_TYPE);
-                if (contentType == null) {
-                    throw new PageTransportErrorException(format("%s header is not set: %s", CONTENT_TYPE, response));
-                }
-                if (!mediaTypeMatches(contentType, PRESTO_PAGES_TYPE)) {
-                    throw new PageTransportErrorException(format("Expected %s response from server but got %s", PRESTO_PAGES_TYPE, contentType));
-                }
-
-                String taskInstanceId = getTaskInstanceId(response);
-                long token = getToken(response);
-                long nextToken = getNextToken(response);
-                boolean complete = getComplete(response);
-
-                try (SliceInput input = new InputStreamSliceInput(response.getInputStream())) {
-                    List pages = ImmutableList.copyOf(readSerializedPages(input));
-                    return createPagesResponse(taskInstanceId, token, nextToken, pages, complete);
-                }
-                catch (IOException e) {
-                    throw new RuntimeException(e);
-                }
-            }
-            catch (PageTransportErrorException e) {
-                throw new PageTransportErrorException("Error fetching " + request.getUri().toASCIIString(), e);
-            }
-        }
-
-        private static String getTaskInstanceId(Response response)
-        {
-            String taskInstanceId = response.getHeader(PRESTO_TASK_INSTANCE_ID);
-            if (taskInstanceId == null) {
-                throw new PageTransportErrorException(format("Expected %s header", PRESTO_TASK_INSTANCE_ID));
-            }
-            return taskInstanceId;
-        }
-
-        private static long getToken(Response response)
-        {
-            String tokenHeader = response.getHeader(PRESTO_PAGE_TOKEN);
-            if (tokenHeader == null) {
-                throw new PageTransportErrorException(format("Expected %s header", PRESTO_PAGE_TOKEN));
-            }
-            return Long.parseLong(tokenHeader);
-        }
-
-        private static long getNextToken(Response response)
-        {
-            String nextTokenHeader = response.getHeader(PRESTO_PAGE_NEXT_TOKEN);
-            if (nextTokenHeader == null) {
-                throw new PageTransportErrorException(format("Expected %s header", PRESTO_PAGE_NEXT_TOKEN));
-            }
-            return Long.parseLong(nextTokenHeader);
-        }
-
-        private static boolean getComplete(Response response)
-        {
-            String bufferComplete = response.getHeader(PRESTO_BUFFER_COMPLETE);
-            if (bufferComplete == null) {
-                throw new PageTransportErrorException(format("Expected %s header", PRESTO_BUFFER_COMPLETE));
-            }
-            return Boolean.parseBoolean(bufferComplete);
-        }
-
-        private static boolean mediaTypeMatches(String value, MediaType range)
-        {
-            try {
-                return MediaType.parse(value).is(range);
-            }
-            catch (IllegalArgumentException | IllegalStateException e) {
-                return false;
-            }
-        }
-    }
-
     public static class PagesResponse
     {
         public static PagesResponse createPagesResponse(String taskInstanceId, long token, long nextToken, Iterable pages, boolean complete)
diff --git a/presto-main/src/main/java/com/facebook/presto/operator/PageTooLargeException.java b/presto-main/src/main/java/com/facebook/presto/operator/PageTooLargeException.java
index 31b3885fe0d71..98b847a8972a2 100644
--- a/presto-main/src/main/java/com/facebook/presto/operator/PageTooLargeException.java
+++ b/presto-main/src/main/java/com/facebook/presto/operator/PageTooLargeException.java
@@ -20,8 +20,8 @@
 public class PageTooLargeException
         extends PrestoException
 {
-    public PageTooLargeException()
+    public PageTooLargeException(Throwable e)
     {
-        super(PAGE_TOO_LARGE, "Remote page is too large");
+        super(PAGE_TOO_LARGE, "Remote page is too large", e);
     }
 }
diff --git a/presto-main/src/main/java/com/facebook/presto/operator/RpcShuffleClient.java b/presto-main/src/main/java/com/facebook/presto/operator/RpcShuffleClient.java
new file mode 100644
index 0000000000000..b03ea89e042bf
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/operator/RpcShuffleClient.java
@@ -0,0 +1,44 @@
+/*
+ * 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.facebook.presto.operator;
+
+import com.facebook.presto.operator.PageBufferClient.PagesResponse;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.airlift.units.DataSize;
+
+/**
+ * All methods in this class should be async
+ */
+public interface RpcShuffleClient
+{
+    ListenableFuture getResults(long token, DataSize maxResponseSize);
+
+    /**
+     * A fire and forget call to issue the ack to the buffer.
+     * No need to handle the response; it is ok for a server to miss the ack.
+     * {@param nextToken} N represents token N - 1 is to be acknowledged.
+     * The implementation needs to guarantee the function is non-blocking and result or failure is not important.
+     */
+    void acknowledgeResultsAsync(long nextToken);
+
+    /**
+     * Close remote buffer
+     */
+    ListenableFuture abortResults();
+
+    /**
+     * Rewrite network related exception to Presto exception
+     */
+    Throwable rewriteException(Throwable throwable);
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/operator/ThriftRpcShuffleClient.java b/presto-main/src/main/java/com/facebook/presto/operator/ThriftRpcShuffleClient.java
new file mode 100644
index 0000000000000..a4086b9dd36ab
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/operator/ThriftRpcShuffleClient.java
@@ -0,0 +1,93 @@
+/*
+ * 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.facebook.presto.operator;
+
+import com.facebook.drift.client.DriftClient;
+import com.facebook.drift.transport.client.MessageTooLargeException;
+import com.facebook.presto.execution.TaskId;
+import com.facebook.presto.execution.buffer.BufferResult;
+import com.facebook.presto.execution.buffer.OutputBuffers.OutputBufferId;
+import com.facebook.presto.operator.PageBufferClient.PagesResponse;
+import com.facebook.presto.server.thrift.ThriftTaskClient;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.airlift.units.DataSize;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.net.URI;
+import java.util.Optional;
+
+import static com.facebook.presto.operator.PageBufferClient.PagesResponse.createPagesResponse;
+import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
+import static java.util.Objects.requireNonNull;
+
+@ThreadSafe
+public final class ThriftRpcShuffleClient
+        implements RpcShuffleClient
+{
+    private final ThriftTaskClient thriftClient;
+    private final TaskId taskId;
+    private final OutputBufferId outputBufferId;
+
+    public ThriftRpcShuffleClient(DriftClient driftClient, URI location)
+    {
+        requireNonNull(location, "location is null");
+
+        this.thriftClient = requireNonNull(driftClient, "thriftClient is null").get(Optional.of(location.getAuthority()));
+
+        // TODO: refactor the entire LocationFactory interfaces to completely replace URI with more efficient/expressive data structures
+        // location format: thrift://{ipAddress}:{thriftPort}/v1/task/{taskId}/results/{bufferId}/
+        String[] paths = location.getPath().split("/");
+        this.taskId = TaskId.valueOf(paths[3]);
+        this.outputBufferId = OutputBufferId.fromString(paths[5]);
+    }
+
+    @Override
+    public ListenableFuture getResults(long token, DataSize maxResponseSize)
+    {
+        ListenableFuture future = thriftClient.getResults(taskId, outputBufferId, token, maxResponseSize.toBytes());
+        return Futures.transform(
+                future,
+                result -> createPagesResponse(
+                        result.getTaskInstanceId(),
+                        result.getToken(),
+                        result.getNextToken(),
+                        result.getSerializedPages(),
+                        result.isBufferComplete()),
+                directExecutor());
+    }
+
+    @Override
+    public void acknowledgeResultsAsync(long nextToken)
+    {
+        // no need to handle the future
+        thriftClient.acknowledgeResults(taskId, outputBufferId, nextToken);
+    }
+
+    @Override
+    public ListenableFuture abortResults()
+    {
+        return thriftClient.abortResults(taskId, outputBufferId);
+    }
+
+    @Override
+    public Throwable rewriteException(Throwable throwable)
+    {
+        if (throwable instanceof MessageTooLargeException) {
+            return new PageTooLargeException(throwable);
+        }
+        return throwable;
+    }
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/server/AsyncHttpExecutionMBean.java b/presto-main/src/main/java/com/facebook/presto/server/AsyncHttpExecutionMBean.java
index 811b8fcfd34be..d05b0e0ae4599 100644
--- a/presto-main/src/main/java/com/facebook/presto/server/AsyncHttpExecutionMBean.java
+++ b/presto-main/src/main/java/com/facebook/presto/server/AsyncHttpExecutionMBean.java
@@ -31,7 +31,7 @@ public class AsyncHttpExecutionMBean
     private final ThreadPoolExecutorMBean timeoutExecutor;
 
     @Inject
-    public AsyncHttpExecutionMBean(@ForAsyncHttp ExecutorService responseExecutor, @ForAsyncHttp ScheduledExecutorService timeoutExecutor)
+    public AsyncHttpExecutionMBean(@ForAsyncRpc ExecutorService responseExecutor, @ForAsyncRpc ScheduledExecutorService timeoutExecutor)
     {
         requireNonNull(responseExecutor, "responseExecutor is null");
         requireNonNull(timeoutExecutor, "timeoutExecutor is null");
diff --git a/presto-main/src/main/java/com/facebook/presto/server/ForAsyncHttp.java b/presto-main/src/main/java/com/facebook/presto/server/ForAsyncRpc.java
similarity index 97%
rename from presto-main/src/main/java/com/facebook/presto/server/ForAsyncHttp.java
rename to presto-main/src/main/java/com/facebook/presto/server/ForAsyncRpc.java
index f2f0b2d811d55..ccc896f05ed46 100644
--- a/presto-main/src/main/java/com/facebook/presto/server/ForAsyncHttp.java
+++ b/presto-main/src/main/java/com/facebook/presto/server/ForAsyncRpc.java
@@ -26,6 +26,6 @@
 @Retention(RUNTIME)
 @Target({FIELD, PARAMETER, METHOD})
 @Qualifier
-public @interface ForAsyncHttp
+public @interface ForAsyncRpc
 {
 }
diff --git a/presto-main/src/main/java/com/facebook/presto/server/InternalCommunicationConfig.java b/presto-main/src/main/java/com/facebook/presto/server/InternalCommunicationConfig.java
index 1c50dcafb913a..261167c65e26f 100644
--- a/presto-main/src/main/java/com/facebook/presto/server/InternalCommunicationConfig.java
+++ b/presto-main/src/main/java/com/facebook/presto/server/InternalCommunicationConfig.java
@@ -36,6 +36,8 @@ public class InternalCommunicationConfig
     private boolean kerberosUseCanonicalHostname = true;
     private boolean binaryTransportEnabled;
     private DataSize maxTaskUpdateSize = new DataSize(16, MEGABYTE);
+    private CommunicationProtocol taskCommunicationProtocol = CommunicationProtocol.HTTP;
+    private CommunicationProtocol serverInfoCommunicationProtocol = CommunicationProtocol.HTTP;
 
     public boolean isHttpsRequired()
     {
@@ -159,4 +161,36 @@ public InternalCommunicationConfig setMaxTaskUpdateSize(DataSize maxTaskUpdateSi
         this.maxTaskUpdateSize = maxTaskUpdateSize;
         return this;
     }
+
+    public enum CommunicationProtocol
+    {
+        HTTP,
+        THRIFT
+    }
+
+    public CommunicationProtocol getTaskCommunicationProtocol()
+    {
+        return taskCommunicationProtocol;
+    }
+
+    @Config("internal-communication.task-communication-protocol")
+    @ConfigDescription("Set task communication protocol")
+    public InternalCommunicationConfig setTaskCommunicationProtocol(CommunicationProtocol taskCommunicationProtocol)
+    {
+        this.taskCommunicationProtocol = taskCommunicationProtocol;
+        return this;
+    }
+
+    public CommunicationProtocol getServerInfoCommunicationProtocol()
+    {
+        return serverInfoCommunicationProtocol;
+    }
+
+    @Config("internal-communication.server-info-communication-protocol")
+    @ConfigDescription("Set server info communication protocol to broadcast state info")
+    public InternalCommunicationConfig setServerInfoCommunicationProtocol(CommunicationProtocol serverInfoCommunicationProtocol)
+    {
+        this.serverInfoCommunicationProtocol = serverInfoCommunicationProtocol;
+        return this;
+    }
 }
diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java
index 51790449ae797..6b1f92b62c1ed 100644
--- a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java
+++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java
@@ -28,6 +28,8 @@
 import com.facebook.airlift.log.Logger;
 import com.facebook.airlift.node.NodeModule;
 import com.facebook.airlift.tracetoken.TraceTokenModule;
+import com.facebook.drift.server.DriftServer;
+import com.facebook.drift.transport.netty.server.DriftNettyServerTransport;
 import com.facebook.presto.eventlistener.EventListenerManager;
 import com.facebook.presto.eventlistener.EventListenerModule;
 import com.facebook.presto.execution.resourceGroups.ResourceGroupManager;
@@ -52,6 +54,7 @@
 import com.google.inject.Module;
 import org.weakref.jmx.guice.MBeanModule;
 
+import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -138,6 +141,11 @@ public void run()
                     injector.getInstance(ServerConfig.class),
                     injector.getInstance(NodeSchedulerConfig.class));
 
+            // TODO: thrift server port should be announced by discovery server similar to http/https ports
+            updateThriftServerPort(
+                    injector.getInstance(Announcer.class),
+                    injector.getInstance(DriftServer.class));
+
             injector.getInstance(StaticFunctionNamespaceStore.class).loadFunctionNamespaceManagers();
             injector.getInstance(SessionPropertyDefaults.class).loadConfigurationManager();
             injector.getInstance(ResourceGroupManager.class).loadConfigurationManager();
@@ -203,6 +211,24 @@ private static void updateConnectorIds(Announcer announcer, CatalogManager metad
         announcer.addServiceAnnouncement(builder.build());
     }
 
+    private static void updateThriftServerPort(Announcer announcer, DriftServer driftServer)
+    {
+        // get existing announcement
+        ServiceAnnouncement announcement = getPrestoAnnouncement(announcer.getServiceAnnouncements());
+
+        // drift server::start can be called multiple times
+        driftServer.start();
+
+        // update announcement and thrift port property
+        int thriftPort = ((DriftNettyServerTransport) driftServer.getServerTransport()).getPort();
+        Map properties = new LinkedHashMap<>(announcement.getProperties());
+        properties.put("thriftServerPort", String.valueOf(thriftPort));
+        announcer.removeServiceAnnouncement(announcement.getId());
+        announcer.addServiceAnnouncement(serviceAnnouncement(announcement.getType()).addProperties(properties).build());
+
+        announcer.forceAnnounce();
+    }
+
     private static ServiceAnnouncement getPrestoAnnouncement(Set announcements)
     {
         for (ServiceAnnouncement announcement : announcements) {
diff --git a/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java b/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java
index e44cd282006ce..f35247e4dc9e8 100644
--- a/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java
+++ b/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java
@@ -18,6 +18,9 @@
 import com.facebook.airlift.stats.GcMonitor;
 import com.facebook.airlift.stats.JmxGcMonitor;
 import com.facebook.airlift.stats.PauseMeter;
+import com.facebook.drift.client.address.AddressSelector;
+import com.facebook.drift.transport.netty.client.DriftNettyClientModule;
+import com.facebook.drift.transport.netty.server.DriftNettyServerModule;
 import com.facebook.presto.GroupByHashPageIndexerFactory;
 import com.facebook.presto.PagesIndexPageSorter;
 import com.facebook.presto.SystemSessionProperties;
@@ -89,6 +92,11 @@
 import com.facebook.presto.operator.TableCommitContext;
 import com.facebook.presto.operator.index.IndexJoinLookupStats;
 import com.facebook.presto.server.remotetask.HttpLocationFactory;
+import com.facebook.presto.server.thrift.FixedAddressSelector;
+import com.facebook.presto.server.thrift.ThriftServerInfoClient;
+import com.facebook.presto.server.thrift.ThriftServerInfoService;
+import com.facebook.presto.server.thrift.ThriftTaskClient;
+import com.facebook.presto.server.thrift.ThriftTaskService;
 import com.facebook.presto.spi.ConnectorSplit;
 import com.facebook.presto.spi.PageIndexerFactory;
 import com.facebook.presto.spi.PageSorter;
@@ -170,6 +178,8 @@
 import static com.facebook.airlift.jaxrs.JaxrsBinder.jaxrsBinder;
 import static com.facebook.airlift.json.JsonBinder.jsonBinder;
 import static com.facebook.airlift.json.JsonCodecBinder.jsonCodecBinder;
+import static com.facebook.drift.client.guice.DriftClientBinder.driftClientBinder;
+import static com.facebook.drift.server.guice.DriftServerBinder.driftServerBinder;
 import static com.facebook.presto.execution.scheduler.NodeSchedulerConfig.NetworkTopologyType.FLAT;
 import static com.facebook.presto.execution.scheduler.NodeSchedulerConfig.NetworkTopologyType.LEGACY;
 import static com.facebook.presto.server.smile.SmileCodecBinder.smileCodecBinder;
@@ -255,6 +265,9 @@ protected void setup(Binder binder)
                     config.setIdleTimeout(new Duration(30, SECONDS));
                     config.setRequestTimeout(new Duration(10, SECONDS));
                 });
+        driftClientBinder(binder).bindDriftClient(ThriftServerInfoClient.class, ForNodeManager.class)
+                .withAddressSelector(((addressSelectorBinder, annotation, prefix) ->
+                        addressSelectorBinder.bind(AddressSelector.class).annotatedWith(annotation).to(FixedAddressSelector.class)));
 
         // node scheduler
         // TODO: remove from NodePartitioningManager and move to CoordinatorModule
@@ -344,6 +357,10 @@ protected void setup(Binder binder)
                     config.setMaxConnectionsPerServer(250);
                     config.setMaxContentLength(new DataSize(32, MEGABYTE));
                 });
+        binder.install(new DriftNettyClientModule());
+        driftClientBinder(binder).bindDriftClient(ThriftTaskClient.class, ForExchange.class)
+                .withAddressSelector(((addressSelectorBinder, annotation, prefix) ->
+                        addressSelectorBinder.bind(AddressSelector.class).annotatedWith(annotation).to(FixedAddressSelector.class)));
 
         configBinder(binder).bindConfig(ExchangeClientConfig.class);
         binder.bind(ExchangeExecutionMBean.class).in(Scopes.SINGLETON);
@@ -488,6 +505,11 @@ protected void setup(Binder binder)
         binder.bind(LocalSpillManager.class).in(Scopes.SINGLETON);
         configBinder(binder).bindConfig(NodeSpillConfig.class);
 
+        // Thrift RPC
+        binder.install(new DriftNettyServerModule());
+        driftServerBinder(binder).bindService(ThriftTaskService.class);
+        driftServerBinder(binder).bindService(ThriftServerInfoService.class);
+
         // cleanup
         binder.bind(ExecutorCleanup.class).in(Scopes.SINGLETON);
     }
@@ -502,7 +524,7 @@ public static ScheduledExecutorService createExchangeExecutor(ExchangeClientConf
 
     @Provides
     @Singleton
-    @ForAsyncHttp
+    @ForAsyncRpc
     public static ExecutorService createAsyncHttpResponseCoreExecutor()
     {
         return newCachedThreadPool(daemonThreadsNamed("async-http-response-%s"));
@@ -510,15 +532,15 @@ public static ExecutorService createAsyncHttpResponseCoreExecutor()
 
     @Provides
     @Singleton
-    @ForAsyncHttp
-    public static BoundedExecutor createAsyncHttpResponseExecutor(@ForAsyncHttp ExecutorService coreExecutor, TaskManagerConfig config)
+    @ForAsyncRpc
+    public static BoundedExecutor createAsyncHttpResponseExecutor(@ForAsyncRpc ExecutorService coreExecutor, TaskManagerConfig config)
     {
         return new BoundedExecutor(coreExecutor, config.getHttpResponseThreads());
     }
 
     @Provides
     @Singleton
-    @ForAsyncHttp
+    @ForAsyncRpc
     public static ScheduledExecutorService createAsyncHttpTimeoutExecutor(TaskManagerConfig config)
     {
         return newScheduledThreadPool(config.getHttpTimeoutThreads(), daemonThreadsNamed("async-http-timeout-%s"));
@@ -531,8 +553,8 @@ public static class ExecutorCleanup
         @Inject
         public ExecutorCleanup(
                 @ForExchange ScheduledExecutorService exchangeExecutor,
-                @ForAsyncHttp ExecutorService httpResponseExecutor,
-                @ForAsyncHttp ScheduledExecutorService httpTimeoutExecutor)
+                @ForAsyncRpc ExecutorService httpResponseExecutor,
+                @ForAsyncRpc ScheduledExecutorService httpTimeoutExecutor)
         {
             executors = ImmutableList.of(
                     exchangeExecutor,
diff --git a/presto-main/src/main/java/com/facebook/presto/server/TaskResource.java b/presto-main/src/main/java/com/facebook/presto/server/TaskResource.java
index 70a5951eec494..54fa7672873ca 100644
--- a/presto-main/src/main/java/com/facebook/presto/server/TaskResource.java
+++ b/presto-main/src/main/java/com/facebook/presto/server/TaskResource.java
@@ -58,7 +58,6 @@
 import java.util.List;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ThreadLocalRandom;
 
 import static com.facebook.airlift.concurrent.MoreFutures.addTimeout;
 import static com.facebook.airlift.http.server.AsyncResponseHandler.bindAsyncResponse;
@@ -71,6 +70,8 @@
 import static com.facebook.presto.client.PrestoHeaders.PRESTO_PAGE_NEXT_TOKEN;
 import static com.facebook.presto.client.PrestoHeaders.PRESTO_PAGE_TOKEN;
 import static com.facebook.presto.client.PrestoHeaders.PRESTO_TASK_INSTANCE_ID;
+import static com.facebook.presto.util.TaskUtils.DEFAULT_MAX_WAIT_TIME;
+import static com.facebook.presto.util.TaskUtils.randomizeWaitTime;
 import static com.google.common.collect.Iterables.transform;
 import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
 import static java.util.Objects.requireNonNull;
@@ -85,7 +86,6 @@
 public class TaskResource
 {
     private static final Duration ADDITIONAL_WAIT_TIME = new Duration(5, SECONDS);
-    private static final Duration DEFAULT_MAX_WAIT_TIME = new Duration(2, SECONDS);
 
     private final TaskManager taskManager;
     private final SessionPropertyManager sessionPropertyManager;
@@ -98,8 +98,8 @@ public class TaskResource
     public TaskResource(
             TaskManager taskManager,
             SessionPropertyManager sessionPropertyManager,
-            @ForAsyncHttp BoundedExecutor responseExecutor,
-            @ForAsyncHttp ScheduledExecutorService timeoutExecutor)
+            @ForAsyncRpc BoundedExecutor responseExecutor,
+            @ForAsyncRpc ScheduledExecutorService timeoutExecutor)
     {
         this.taskManager = requireNonNull(taskManager, "taskManager is null");
         this.sessionPropertyManager = requireNonNull(sessionPropertyManager, "sessionPropertyManager is null");
@@ -352,11 +352,4 @@ private static boolean shouldSummarize(UriInfo uriInfo)
     {
         return uriInfo.getQueryParameters().containsKey("summarize");
     }
-
-    private static Duration randomizeWaitTime(Duration waitTime)
-    {
-        // Randomize in [T/2, T], so wait is not near zero and the client-supplied max wait time is respected
-        long halfWaitMillis = waitTime.toMillis() / 2;
-        return new Duration(halfWaitMillis + ThreadLocalRandom.current().nextLong(halfWaitMillis), MILLISECONDS);
-    }
 }
diff --git a/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpLocationFactory.java b/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpLocationFactory.java
index db6df492df45b..ff80ac7a260b2 100644
--- a/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpLocationFactory.java
+++ b/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpLocationFactory.java
@@ -13,6 +13,7 @@
  */
 package com.facebook.presto.server.remotetask;
 
+import com.facebook.airlift.http.client.HttpUriBuilder;
 import com.facebook.airlift.http.server.HttpServerInfo;
 import com.facebook.presto.execution.LocationFactory;
 import com.facebook.presto.execution.StageId;
@@ -20,11 +21,13 @@
 import com.facebook.presto.metadata.InternalNode;
 import com.facebook.presto.metadata.InternalNodeManager;
 import com.facebook.presto.server.InternalCommunicationConfig;
+import com.facebook.presto.server.InternalCommunicationConfig.CommunicationProtocol;
 import com.facebook.presto.spi.QueryId;
 
 import javax.inject.Inject;
 
 import java.net.URI;
+import java.util.OptionalInt;
 
 import static com.facebook.airlift.http.client.HttpUriBuilder.uriBuilderFrom;
 import static java.util.Objects.requireNonNull;
@@ -34,17 +37,19 @@ public class HttpLocationFactory
 {
     private final InternalNodeManager nodeManager;
     private final URI baseUri;
+    private final CommunicationProtocol taskCommunicationProtocol;
 
     @Inject
     public HttpLocationFactory(InternalNodeManager nodeManager, HttpServerInfo httpServerInfo, InternalCommunicationConfig config)
     {
-        this(nodeManager, config.isHttpsRequired() ? httpServerInfo.getHttpsUri() : httpServerInfo.getHttpUri());
+        this(nodeManager, config.isHttpsRequired() ? httpServerInfo.getHttpsUri() : httpServerInfo.getHttpUri(), config.getTaskCommunicationProtocol());
     }
 
-    public HttpLocationFactory(InternalNodeManager nodeManager, URI baseUri)
+    public HttpLocationFactory(InternalNodeManager nodeManager, URI baseUri, CommunicationProtocol taskCommunicationProtocol)
     {
         this.nodeManager = requireNonNull(nodeManager, "nodeManager is null");
         this.baseUri = requireNonNull(baseUri, "baseUri is null");
+        this.taskCommunicationProtocol = requireNonNull(taskCommunicationProtocol, "taskCommunicationProtocol is null");
     }
 
     @Override
@@ -70,7 +75,13 @@ public URI createStageLocation(StageId stageId)
     @Override
     public URI createLocalTaskLocation(TaskId taskId)
     {
-        return createTaskLocation(nodeManager.getCurrentNode(), taskId);
+        return createHttpTaskLocation(nodeManager.getCurrentNode(), taskId);
+    }
+
+    @Override
+    public URI createLegacyTaskLocation(InternalNode node, TaskId taskId)
+    {
+        return createHttpTaskLocation(node, taskId);
     }
 
     @Override
@@ -78,10 +89,23 @@ public URI createTaskLocation(InternalNode node, TaskId taskId)
     {
         requireNonNull(node, "node is null");
         requireNonNull(taskId, "taskId is null");
-        return uriBuilderFrom(node.getInternalUri())
-                .appendPath("/v1/task")
-                .appendPath(taskId.toString())
-                .build();
+
+        if (taskCommunicationProtocol.equals(CommunicationProtocol.HTTP)) {
+            return createLegacyTaskLocation(node, taskId);
+        }
+
+        OptionalInt thriftPort = node.getThriftPort();
+
+        HttpUriBuilder builder = uriBuilderFrom(node.getInternalUri());
+        if (taskCommunicationProtocol.equals(CommunicationProtocol.THRIFT) && thriftPort.isPresent()) {
+            builder.scheme("thrift");
+            builder.port(thriftPort.getAsInt());
+        }
+        else {
+            // fall back to http case
+        }
+
+        return builder.appendPath("/v1/task").appendPath(taskId.toString()).build();
     }
 
     @Override
@@ -91,4 +115,14 @@ public URI createMemoryInfoLocation(InternalNode node)
         return uriBuilderFrom(node.getInternalUri())
                 .appendPath("/v1/memory").build();
     }
+
+    private URI createHttpTaskLocation(InternalNode node, TaskId taskId)
+    {
+        requireNonNull(node, "node is null");
+        requireNonNull(taskId, "taskId is null");
+        return uriBuilderFrom(node.getInternalUri())
+                .appendPath("/v1/task")
+                .appendPath(taskId.toString())
+                .build();
+    }
 }
diff --git a/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpRemoteTask.java b/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpRemoteTask.java
index 3984ccf64244b..5a5cfaa387430 100644
--- a/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpRemoteTask.java
+++ b/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpRemoteTask.java
@@ -124,6 +124,7 @@ public final class HttpRemoteTask
 
     private final TaskId taskId;
     private final URI taskLocation;
+    private final URI remoteTaskLocation;
 
     private final Session session;
     private final String nodeId;
@@ -190,6 +191,7 @@ public HttpRemoteTask(
             TaskId taskId,
             String nodeId,
             URI location,
+            URI remoteLocation,
             PlanFragment planFragment,
             Multimap initialSplits,
             OutputBuffers outputBuffers,
@@ -215,6 +217,7 @@ public HttpRemoteTask(
         requireNonNull(taskId, "taskId is null");
         requireNonNull(nodeId, "nodeId is null");
         requireNonNull(location, "location is null");
+        requireNonNull(remoteLocation, "remoteLocation is null");
         requireNonNull(planFragment, "planFragment is null");
         requireNonNull(outputBuffers, "outputBuffers is null");
         requireNonNull(httpClient, "httpClient is null");
@@ -231,6 +234,7 @@ public HttpRemoteTask(
         try (SetThreadName ignored = new SetThreadName("HttpRemoteTask-%s", taskId)) {
             this.taskId = taskId;
             this.taskLocation = location;
+            this.remoteTaskLocation = remoteLocation;
             this.session = session;
             this.nodeId = nodeId;
             this.planFragment = planFragment;
@@ -337,6 +341,12 @@ public TaskStatus getTaskStatus()
         return taskStatusFetcher.getTaskStatus();
     }
 
+    @Override
+    public URI getRemoteTaskLocation()
+    {
+        return remoteTaskLocation;
+    }
+
     @Override
     public void start()
     {
diff --git a/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpRemoteTaskFactory.java b/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpRemoteTaskFactory.java
index 264d3ed9ea097..77ae401cb6190 100644
--- a/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpRemoteTaskFactory.java
+++ b/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpRemoteTaskFactory.java
@@ -83,7 +83,8 @@ public class HttpRemoteTaskFactory
     private final int maxTaskUpdateSizeInBytes;
 
     @Inject
-    public HttpRemoteTaskFactory(QueryManagerConfig config,
+    public HttpRemoteTaskFactory(
+            QueryManagerConfig config,
             TaskManagerConfig taskConfig,
             @ForScheduler HttpClient httpClient,
             LocationFactory locationFactory,
@@ -140,7 +141,8 @@ public void stop()
     }
 
     @Override
-    public RemoteTask createRemoteTask(Session session,
+    public RemoteTask createRemoteTask(
+            Session session,
             TaskId taskId,
             InternalNode node,
             PlanFragment fragment,
@@ -150,9 +152,11 @@ public RemoteTask createRemoteTask(Session session,
             boolean summarizeTaskInfo,
             TableWriteInfo tableWriteInfo)
     {
-        return new HttpRemoteTask(session,
+        return new HttpRemoteTask(
+                session,
                 taskId,
                 node.getNodeIdentifier(),
+                locationFactory.createLegacyTaskLocation(node, taskId),
                 locationFactory.createTaskLocation(node, taskId),
                 fragment,
                 initialSplits,
diff --git a/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java
index d83fc24b00ec4..3e14a39670fa4 100644
--- a/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java
+++ b/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java
@@ -29,6 +29,8 @@
 import com.facebook.airlift.json.JsonModule;
 import com.facebook.airlift.node.testing.TestingNodeModule;
 import com.facebook.airlift.tracetoken.TraceTokenModule;
+import com.facebook.drift.server.DriftServer;
+import com.facebook.drift.transport.netty.server.DriftNettyServerTransport;
 import com.facebook.presto.connector.ConnectorManager;
 import com.facebook.presto.cost.StatsCalculator;
 import com.facebook.presto.eventlistener.EventListenerManager;
@@ -338,6 +340,11 @@ public TestingPrestoServer(
         announcer = injector.getInstance(Announcer.class);
         requestBlocker = injector.getInstance(RequestBlocker.class);
 
+        // Announce Thrift server address
+        DriftServer driftServer = injector.getInstance(DriftServer.class);
+        driftServer.start();
+        updateThriftServerAddressAnnouncement(announcer, driftServerPort(driftServer), nodeManager);
+
         announcer.forceAnnounce();
 
         refreshNodes();
@@ -559,6 +566,22 @@ private static void updateConnectorIdAnnouncement(Announcer announcer, Connector
         nodeManager.refreshNodes();
     }
 
+    // TODO: announcement does not work for coordinator
+    private static void updateThriftServerAddressAnnouncement(Announcer announcer, int thriftPort, InternalNodeManager nodeManager)
+    {
+        // get existing announcement
+        ServiceAnnouncement announcement = getPrestoAnnouncement(announcer.getServiceAnnouncements());
+
+        // update announcement and thrift port property
+        Map properties = new LinkedHashMap<>(announcement.getProperties());
+        properties.put("thriftServerPort", String.valueOf(thriftPort));
+        announcer.removeServiceAnnouncement(announcement.getId());
+        announcer.addServiceAnnouncement(serviceAnnouncement(announcement.getType()).addProperties(properties).build());
+        announcer.forceAnnounce();
+
+        nodeManager.refreshNodes();
+    }
+
     private static ServiceAnnouncement getPrestoAnnouncement(Set announcements)
     {
         for (ServiceAnnouncement announcement : announcements) {
@@ -625,4 +648,9 @@ public void init(FilterConfig filterConfig) {}
         @Override
         public void destroy() {}
     }
+
+    private static int driftServerPort(DriftServer server)
+    {
+        return ((DriftNettyServerTransport) server.getServerTransport()).getPort();
+    }
 }
diff --git a/presto-main/src/main/java/com/facebook/presto/server/thrift/FixedAddressSelector.java b/presto-main/src/main/java/com/facebook/presto/server/thrift/FixedAddressSelector.java
new file mode 100644
index 0000000000000..c9fd32174f07c
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/server/thrift/FixedAddressSelector.java
@@ -0,0 +1,45 @@
+/*
+ * 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.facebook.presto.server.thrift;
+
+import com.facebook.drift.client.address.AddressSelector;
+import com.facebook.drift.client.address.SimpleAddressSelector.SimpleAddress;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.net.HostAndPort;
+
+import java.util.Optional;
+import java.util.Set;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+public class FixedAddressSelector
+        implements AddressSelector
+
+{
+    @Override
+    public Optional selectAddress(Optional addressSelectionContext)
+    {
+        return selectAddress(addressSelectionContext, ImmutableSet.of());
+    }
+
+    @Override
+    public Optional selectAddress(Optional addressSelectionContext, Set attempted)
+    {
+        checkArgument(addressSelectionContext.isPresent());
+
+        // TODO: We should make context generic type in Drift library to avoid parsing and create address every time
+        HostAndPort address = HostAndPort.fromString(addressSelectionContext.get());
+        return Optional.of(new SimpleAddress(address));
+    }
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/server/thrift/ThriftServerInfoClient.java b/presto-main/src/main/java/com/facebook/presto/server/thrift/ThriftServerInfoClient.java
new file mode 100644
index 0000000000000..e30baccbe00e2
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/server/thrift/ThriftServerInfoClient.java
@@ -0,0 +1,29 @@
+/*
+ * 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.facebook.presto.server.thrift;
+
+import com.facebook.drift.annotations.ThriftMethod;
+import com.facebook.drift.annotations.ThriftService;
+import com.google.common.util.concurrent.ListenableFuture;
+
+@ThriftService("ThriftServerInfoService")
+public interface ThriftServerInfoClient
+{
+    /**
+     * Use integers to represent the ordinal of the enum.
+     * Use NodeState::valueOf to recover the enum from an integer
+     */
+    @ThriftMethod
+    ListenableFuture getServerState();
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/server/thrift/ThriftServerInfoService.java b/presto-main/src/main/java/com/facebook/presto/server/thrift/ThriftServerInfoService.java
new file mode 100644
index 0000000000000..a38de7a576dd7
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/server/thrift/ThriftServerInfoService.java
@@ -0,0 +1,62 @@
+/*
+ * 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.facebook.presto.server.thrift;
+
+import com.facebook.drift.annotations.ThriftMethod;
+import com.facebook.drift.annotations.ThriftService;
+import com.facebook.presto.server.GracefulShutdownHandler;
+import com.facebook.presto.server.ServerInfoResource;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+
+import javax.inject.Inject;
+
+import static com.facebook.airlift.concurrent.Threads.daemonThreadsNamed;
+import static com.facebook.presto.spi.NodeState.ACTIVE;
+import static com.facebook.presto.spi.NodeState.SHUTTING_DOWN;
+import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator;
+import static java.util.Objects.requireNonNull;
+import static java.util.concurrent.Executors.newSingleThreadExecutor;
+
+/**
+ * The corresponding Thrift implementation for {@link ServerInfoResource}.
+ * Only /state endpoint has been implemented.
+ */
+@ThriftService(value = "presto-info", idlName = "ThriftServerInfoService")
+public class ThriftServerInfoService
+{
+    private final GracefulShutdownHandler shutdownHandler;
+    private final ListeningExecutorService executor = listeningDecorator(newSingleThreadExecutor(daemonThreadsNamed("server-info-executor")));
+
+    @Inject
+    public ThriftServerInfoService(GracefulShutdownHandler shutdownHandler)
+    {
+        this.shutdownHandler = requireNonNull(shutdownHandler, "shutdownHandler is null");
+    }
+
+    /**
+     * Use integers to represent the ordinal of the enum.
+     * Use NodeState::valueOf to recover the enum from an integer
+     */
+    @ThriftMethod
+    public ListenableFuture getServerState()
+    {
+        return executor.submit(() -> {
+            if (shutdownHandler.isShutdownRequested()) {
+                return SHUTTING_DOWN.getValue();
+            }
+            return ACTIVE.getValue();
+        });
+    }
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/server/thrift/ThriftTaskClient.java b/presto-main/src/main/java/com/facebook/presto/server/thrift/ThriftTaskClient.java
new file mode 100644
index 0000000000000..14ffc4c46a10d
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/server/thrift/ThriftTaskClient.java
@@ -0,0 +1,35 @@
+/*
+ * 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.facebook.presto.server.thrift;
+
+import com.facebook.drift.annotations.ThriftMethod;
+import com.facebook.drift.annotations.ThriftService;
+import com.facebook.presto.execution.TaskId;
+import com.facebook.presto.execution.buffer.BufferResult;
+import com.facebook.presto.execution.buffer.OutputBuffers.OutputBufferId;
+import com.google.common.util.concurrent.ListenableFuture;
+
+// TODO: the client currently only supports exchange; more methods (for /v1/task) should be supported
+@ThriftService("ThriftTaskService")
+public interface ThriftTaskClient
+{
+    @ThriftMethod
+    ListenableFuture getResults(TaskId taskId, OutputBufferId bufferId, long token, long maxSizeInBytes);
+
+    @ThriftMethod
+    ListenableFuture acknowledgeResults(TaskId taskId, OutputBufferId bufferId, long token);
+
+    @ThriftMethod
+    ListenableFuture abortResults(TaskId taskId, OutputBufferId bufferId);
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/server/thrift/ThriftTaskService.java b/presto-main/src/main/java/com/facebook/presto/server/thrift/ThriftTaskService.java
new file mode 100644
index 0000000000000..870223127a36e
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/server/thrift/ThriftTaskService.java
@@ -0,0 +1,91 @@
+/*
+ * 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.facebook.presto.server.thrift;
+
+import com.facebook.drift.annotations.ThriftMethod;
+import com.facebook.drift.annotations.ThriftService;
+import com.facebook.presto.execution.TaskId;
+import com.facebook.presto.execution.TaskManager;
+import com.facebook.presto.execution.buffer.BufferResult;
+import com.facebook.presto.execution.buffer.OutputBuffers.OutputBufferId;
+import com.facebook.presto.server.ForAsyncRpc;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.airlift.units.DataSize;
+import io.airlift.units.Duration;
+
+import javax.inject.Inject;
+
+import java.util.concurrent.ScheduledExecutorService;
+
+import static com.facebook.airlift.concurrent.MoreFutures.addTimeout;
+import static com.facebook.presto.util.TaskUtils.DEFAULT_MAX_WAIT_TIME;
+import static com.facebook.presto.util.TaskUtils.randomizeWaitTime;
+import static io.airlift.units.DataSize.Unit.BYTE;
+import static java.util.Objects.requireNonNull;
+
+// TODO: the server currently only supports exchange; more end points (for /v1/task) should be supported
+@ThriftService(value = "presto-task", idlName = "ThriftTaskService")
+public class ThriftTaskService
+{
+    private final TaskManager taskManager;
+    private final ScheduledExecutorService timeoutExecutor;
+
+    @Inject
+    public ThriftTaskService(TaskManager taskManager, @ForAsyncRpc ScheduledExecutorService timeoutExecutor)
+    {
+        this.taskManager = requireNonNull(taskManager, "taskManager is null");
+        this.timeoutExecutor = requireNonNull(timeoutExecutor, "timeoutExecutor is null");
+    }
+
+    @ThriftMethod
+    public ListenableFuture getResults(TaskId taskId, OutputBufferId bufferId, long token, long maxSizeInBytes)
+    {
+        requireNonNull(taskId, "taskId is null");
+        requireNonNull(bufferId, "bufferId is null");
+
+        ListenableFuture bufferResultFuture = taskManager.getTaskResults(taskId, bufferId, token, new DataSize(maxSizeInBytes, BYTE));
+        Duration waitTime = randomizeWaitTime(DEFAULT_MAX_WAIT_TIME);
+        return addTimeout(
+                bufferResultFuture,
+                () -> BufferResult.emptyResults(taskManager.getTaskInstanceId(taskId), token, false),
+                waitTime,
+                timeoutExecutor);
+    }
+
+    @ThriftMethod
+    public ListenableFuture acknowledgeResults(TaskId taskId, OutputBufferId bufferId, long token)
+    {
+        requireNonNull(taskId, "taskId is null");
+        requireNonNull(bufferId, "bufferId is null");
+
+        taskManager.acknowledgeTaskResults(taskId, bufferId, token);
+        return Futures.immediateFuture(null);
+    }
+
+    @ThriftMethod
+    public ListenableFuture abortResults(TaskId taskId, OutputBufferId bufferId)
+    {
+        requireNonNull(taskId, "taskId is null");
+        requireNonNull(bufferId, "bufferId is null");
+
+        // Use thrift server pool to abort tasks; it is dangerous to use a fixed thread pool to abort tasks.
+        // When having a surge of aborting results, a fixed thread pool may not be able to handle requests fast enough causing query to hang.
+        // TaskManager does not support async calls with a thread pool.
+        // Even getTaskResults is a fake async call with an immediate future wrapping around ClientBuffer::processRead.
+        // It might worth exploring true async RPC for /v1/task endpoint
+        taskManager.abortTaskResults(taskId, bufferId);
+        return Futures.immediateFuture(null);
+    }
+}
diff --git a/presto-main/src/main/java/com/facebook/presto/util/TaskUtils.java b/presto-main/src/main/java/com/facebook/presto/util/TaskUtils.java
new file mode 100644
index 0000000000000..cef3997f6e653
--- /dev/null
+++ b/presto-main/src/main/java/com/facebook/presto/util/TaskUtils.java
@@ -0,0 +1,35 @@
+/*
+ * 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.facebook.presto.util;
+
+import io.airlift.units.Duration;
+
+import java.util.concurrent.ThreadLocalRandom;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+public class TaskUtils
+{
+    public static final Duration DEFAULT_MAX_WAIT_TIME = new Duration(2, SECONDS);
+
+    private TaskUtils() {}
+
+    public static Duration randomizeWaitTime(Duration waitTime)
+    {
+        // Randomize in [T/2, T], so wait is not near zero and the client-supplied max wait time is respected
+        long halfWaitMillis = waitTime.toMillis() / 2;
+        return new Duration(halfWaitMillis + ThreadLocalRandom.current().nextLong(halfWaitMillis), MILLISECONDS);
+    }
+}
diff --git a/presto-main/src/test/java/com/facebook/presto/execution/MockRemoteTaskFactory.java b/presto-main/src/test/java/com/facebook/presto/execution/MockRemoteTaskFactory.java
index a1cffffb35b49..9907cdc67ac87 100644
--- a/presto-main/src/test/java/com/facebook/presto/execution/MockRemoteTaskFactory.java
+++ b/presto-main/src/test/java/com/facebook/presto/execution/MockRemoteTaskFactory.java
@@ -270,6 +270,12 @@ public TaskInfo getTaskInfo()
                     true);
         }
 
+        @Override
+        public URI getRemoteTaskLocation()
+        {
+            return location;
+        }
+
         @Override
         public TaskStatus getTaskStatus()
         {
diff --git a/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java b/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java
index cbaa439ad8d5c..6069ccbb78fa8 100644
--- a/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java
+++ b/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java
@@ -314,11 +314,17 @@ public URI createLocalTaskLocation(TaskId taskId)
         }
 
         @Override
-        public URI createTaskLocation(InternalNode node, TaskId taskId)
+        public URI createLegacyTaskLocation(InternalNode node, TaskId taskId)
         {
             return URI.create("http://fake.invalid/task/" + node.getNodeIdentifier() + "/" + taskId);
         }
 
+        @Override
+        public URI createTaskLocation(InternalNode node, TaskId taskId)
+        {
+            return createLegacyTaskLocation(node, taskId);
+        }
+
         @Override
         public URI createMemoryInfoLocation(InternalNode node)
         {
diff --git a/presto-main/src/test/java/com/facebook/presto/metadata/TestDiscoveryNodeManager.java b/presto-main/src/test/java/com/facebook/presto/metadata/TestDiscoveryNodeManager.java
index afd3a10bc793a..d437bf8ff4668 100644
--- a/presto-main/src/test/java/com/facebook/presto/metadata/TestDiscoveryNodeManager.java
+++ b/presto-main/src/test/java/com/facebook/presto/metadata/TestDiscoveryNodeManager.java
@@ -22,6 +22,7 @@
 import com.facebook.airlift.node.NodeInfo;
 import com.facebook.presto.client.NodeVersion;
 import com.facebook.presto.failureDetector.NoOpFailureDetector;
+import com.facebook.presto.operator.TestingDriftClient;
 import com.facebook.presto.server.InternalCommunicationConfig;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ImmutableList;
@@ -86,7 +87,7 @@ public void setup()
     @Test
     public void testGetAllNodes()
     {
-        DiscoveryNodeManager manager = new DiscoveryNodeManager(selector, nodeInfo, new NoOpFailureDetector(), expectedVersion, testHttpClient, internalCommunicationConfig);
+        DiscoveryNodeManager manager = new DiscoveryNodeManager(selector, nodeInfo, new NoOpFailureDetector(), expectedVersion, testHttpClient, new TestingDriftClient<>(), internalCommunicationConfig);
         try {
             AllNodes allNodes = manager.getAllNodes();
 
@@ -124,7 +125,7 @@ public void testGetCurrentNode()
                 .setEnvironment("test")
                 .setNodeId(currentNode.getNodeIdentifier()));
 
-        DiscoveryNodeManager manager = new DiscoveryNodeManager(selector, nodeInfo, new NoOpFailureDetector(), expectedVersion, testHttpClient, internalCommunicationConfig);
+        DiscoveryNodeManager manager = new DiscoveryNodeManager(selector, nodeInfo, new NoOpFailureDetector(), expectedVersion, testHttpClient, new TestingDriftClient<>(), internalCommunicationConfig);
         try {
             assertEquals(manager.getCurrentNode(), currentNode);
         }
@@ -136,7 +137,7 @@ public void testGetCurrentNode()
     @Test
     public void testGetCoordinators()
     {
-        DiscoveryNodeManager manager = new DiscoveryNodeManager(selector, nodeInfo, new NoOpFailureDetector(), expectedVersion, testHttpClient, internalCommunicationConfig);
+        DiscoveryNodeManager manager = new DiscoveryNodeManager(selector, nodeInfo, new NoOpFailureDetector(), expectedVersion, testHttpClient, new TestingDriftClient<>(), internalCommunicationConfig);
         try {
             assertEquals(manager.getCoordinators(), ImmutableSet.of(coordinator));
         }
@@ -149,14 +150,14 @@ public void testGetCoordinators()
     @Test(expectedExceptions = IllegalStateException.class, expectedExceptionsMessageRegExp = ".* current node not returned .*")
     public void testGetCurrentNodeRequired()
     {
-        new DiscoveryNodeManager(selector, new NodeInfo("test"), new NoOpFailureDetector(), expectedVersion, testHttpClient, internalCommunicationConfig);
+        new DiscoveryNodeManager(selector, new NodeInfo("test"), new NoOpFailureDetector(), expectedVersion, testHttpClient, new TestingDriftClient<>(), internalCommunicationConfig);
     }
 
     @Test(timeOut = 60000)
     public void testNodeChangeListener()
             throws Exception
     {
-        DiscoveryNodeManager manager = new DiscoveryNodeManager(selector, nodeInfo, new NoOpFailureDetector(), expectedVersion, testHttpClient, internalCommunicationConfig);
+        DiscoveryNodeManager manager = new DiscoveryNodeManager(selector, nodeInfo, new NoOpFailureDetector(), expectedVersion, testHttpClient, new TestingDriftClient<>(), internalCommunicationConfig);
         try {
             manager.startPollingNodeStates();
 
diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestExchangeClient.java b/presto-main/src/test/java/com/facebook/presto/operator/TestExchangeClient.java
index 8cef09fe22ac3..e034eb0e9cb95 100644
--- a/presto-main/src/test/java/com/facebook/presto/operator/TestExchangeClient.java
+++ b/presto-main/src/test/java/com/facebook/presto/operator/TestExchangeClient.java
@@ -122,6 +122,7 @@ public void testHappyPath()
                 true,
                 0.2,
                 new TestingHttpClient(processor, scheduler),
+                new TestingDriftClient<>(),
                 scheduler,
                 new SimpleLocalMemoryContext(newSimpleAggregatedMemoryContext(), "test"),
                 pageBufferClientCallbackExecutor);
@@ -162,6 +163,7 @@ public void testAddLocation()
                 true,
                 0.2,
                 new TestingHttpClient(processor, testingHttpClientExecutor),
+                new TestingDriftClient<>(),
                 scheduler,
                 new SimpleLocalMemoryContext(newSimpleAggregatedMemoryContext(), "test"),
                 pageBufferClientCallbackExecutor);
@@ -235,6 +237,7 @@ public void testBufferLimit()
                 true,
                 0.2,
                 new TestingHttpClient(processor, testingHttpClientExecutor),
+                new TestingDriftClient<>(),
                 scheduler,
                 new SimpleLocalMemoryContext(newSimpleAggregatedMemoryContext(), "test"),
                 pageBufferClientCallbackExecutor);
@@ -318,6 +321,7 @@ public void testClose()
                 true,
                 0.2,
                 new TestingHttpClient(processor, testingHttpClientExecutor),
+                new TestingDriftClient<>(),
                 scheduler,
                 new SimpleLocalMemoryContext(newSimpleAggregatedMemoryContext(), "test"),
                 pageBufferClientCallbackExecutor);
@@ -383,6 +387,7 @@ public Response handle(Request request)
                 true,
                 0.2,
                 new TestingHttpClient(processor, testingHttpClientExecutor),
+                new TestingDriftClient<>(),
                 scheduler,
                 new SimpleLocalMemoryContext(newSimpleAggregatedMemoryContext(), "test"),
                 pageBufferClientCallbackExecutor)) {
@@ -459,6 +464,7 @@ public void testRemoveRemoteSource()
                 true,
                 0.2,
                 new TestingHttpClient(processor, testingHttpClientExecutor),
+                new TestingDriftClient<>(),
                 scheduler,
                 new SimpleLocalMemoryContext(newSimpleAggregatedMemoryContext(), "test"),
                 pageBufferClientCallbackExecutor);
diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestExchangeOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestExchangeOperator.java
index caa7a4a7f0682..3df89c53a27cf 100644
--- a/presto-main/src/test/java/com/facebook/presto/operator/TestExchangeOperator.java
+++ b/presto-main/src/test/java/com/facebook/presto/operator/TestExchangeOperator.java
@@ -92,6 +92,7 @@ public void setUp()
                 true,
                 0.2,
                 httpClient,
+                new TestingDriftClient<>(),
                 scheduler,
                 systemMemoryUsageListener,
                 pageBufferClientCallbackExecutor);
diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestMergeOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestMergeOperator.java
index 4dddbeebadd0e..3b45b34916f5d 100644
--- a/presto-main/src/test/java/com/facebook/presto/operator/TestMergeOperator.java
+++ b/presto-main/src/test/java/com/facebook/presto/operator/TestMergeOperator.java
@@ -83,7 +83,7 @@ public void setUp()
 
         taskBuffers = CacheBuilder.newBuilder().build(CacheLoader.from(TestingTaskBuffer::new));
         httpClient = new TestingHttpClient(new TestingExchangeHttpClientHandler(taskBuffers), executor);
-        exchangeClientFactory = new ExchangeClientFactory(new ExchangeClientConfig(), httpClient, executor);
+        exchangeClientFactory = new ExchangeClientFactory(new ExchangeClientConfig(), httpClient, new TestingDriftClient<>(), executor);
         orderingCompiler = new OrderingCompiler();
     }
 
diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestHttpPageBufferClient.java b/presto-main/src/test/java/com/facebook/presto/operator/TestPageBufferClient.java
similarity index 93%
rename from presto-main/src/test/java/com/facebook/presto/operator/TestHttpPageBufferClient.java
rename to presto-main/src/test/java/com/facebook/presto/operator/TestPageBufferClient.java
index 0746e1dd6b116..2a8a72e378dce 100644
--- a/presto-main/src/test/java/com/facebook/presto/operator/TestHttpPageBufferClient.java
+++ b/presto-main/src/test/java/com/facebook/presto/operator/TestPageBufferClient.java
@@ -21,7 +21,7 @@
 import com.facebook.airlift.testing.TestingTicker;
 import com.facebook.presto.execution.buffer.PagesSerde;
 import com.facebook.presto.execution.buffer.SerializedPage;
-import com.facebook.presto.operator.HttpPageBufferClient.ClientCallback;
+import com.facebook.presto.operator.PageBufferClient.ClientCallback;
 import com.facebook.presto.spi.HostAddress;
 import com.facebook.presto.spi.Page;
 import com.google.common.collect.ImmutableListMultimap;
@@ -60,7 +60,7 @@
 import static java.util.concurrent.Executors.newScheduledThreadPool;
 import static org.testng.Assert.assertEquals;
 
-public class TestHttpPageBufferClient
+public class TestPageBufferClient
 {
     private ScheduledExecutorService scheduler;
     private ExecutorService pageBufferClientCallbackExecutor;
@@ -101,7 +101,8 @@ public void testHappyPath()
         TestingClientCallback callback = new TestingClientCallback(requestComplete);
 
         URI location = URI.create("http://localhost:8080");
-        HttpPageBufferClient client = new HttpPageBufferClient(new TestingHttpClient(processor, scheduler),
+        PageBufferClient client = new PageBufferClient(
+                new HttpRpcShuffleClient(new TestingHttpClient(processor, scheduler), location),
                 new Duration(1, TimeUnit.MINUTES),
                 true,
                 location,
@@ -186,7 +187,8 @@ public void testLifecycle()
         TestingClientCallback callback = new TestingClientCallback(requestComplete);
 
         URI location = URI.create("http://localhost:8080");
-        HttpPageBufferClient client = new HttpPageBufferClient(new TestingHttpClient(processor, scheduler),
+        PageBufferClient client = new PageBufferClient(
+                new HttpRpcShuffleClient(new TestingHttpClient(processor, scheduler), location),
                 new Duration(1, TimeUnit.MINUTES),
                 true,
                 location,
@@ -198,7 +200,7 @@ public void testLifecycle()
 
         client.scheduleRequest(expectedMaxSize);
         beforeRequest.await(10, TimeUnit.SECONDS);
-        assertStatus(client, location, "running", 0, 1, 0, 0, "PROCESSING_REQUEST");
+        assertStatus(client, location, "running", 0, 1, 0, 0, "processing request");
         assertEquals(client.isRunning(), true);
         afterRequest.await(10, TimeUnit.SECONDS);
 
@@ -207,7 +209,7 @@ public void testLifecycle()
 
         client.close();
         beforeRequest.await(10, TimeUnit.SECONDS);
-        assertStatus(client, location, "closed", 0, 1, 1, 1, "PROCESSING_REQUEST");
+        assertStatus(client, location, "closed", 0, 1, 1, 1, "processing request");
         afterRequest.await(10, TimeUnit.SECONDS);
         requestComplete.await(10, TimeUnit.SECONDS);
         assertStatus(client, location, "closed", 0, 1, 2, 1, "not scheduled");
@@ -226,7 +228,8 @@ public void testInvalidResponses()
         TestingClientCallback callback = new TestingClientCallback(requestComplete);
 
         URI location = URI.create("http://localhost:8080");
-        HttpPageBufferClient client = new HttpPageBufferClient(new TestingHttpClient(processor, scheduler),
+        PageBufferClient client = new PageBufferClient(
+                new HttpRpcShuffleClient(new TestingHttpClient(processor, scheduler), location),
                 new Duration(1, TimeUnit.MINUTES),
                 true,
                 location,
@@ -294,7 +297,8 @@ public void testCloseDuringPendingRequest()
         TestingClientCallback callback = new TestingClientCallback(requestComplete);
 
         URI location = URI.create("http://localhost:8080");
-        HttpPageBufferClient client = new HttpPageBufferClient(new TestingHttpClient(processor, scheduler),
+        PageBufferClient client = new PageBufferClient(
+                new HttpRpcShuffleClient(new TestingHttpClient(processor, scheduler), location),
                 new Duration(1, TimeUnit.MINUTES),
                 true,
                 location,
@@ -307,7 +311,7 @@ public void testCloseDuringPendingRequest()
         // send request
         client.scheduleRequest(expectedMaxSize);
         beforeRequest.await(10, TimeUnit.SECONDS);
-        assertStatus(client, location, "running", 0, 1, 0, 0, "PROCESSING_REQUEST");
+        assertStatus(client, location, "running", 0, 1, 0, 0, "processing request");
         assertEquals(client.isRunning(), true);
         // request is pending, now close it
         client.close();
@@ -348,7 +352,8 @@ public void testExceptionFromResponseHandler()
         TestingClientCallback callback = new TestingClientCallback(requestComplete);
 
         URI location = URI.create("http://localhost:8080");
-        HttpPageBufferClient client = new HttpPageBufferClient(new TestingHttpClient(processor, scheduler),
+        PageBufferClient client = new PageBufferClient(
+                new HttpRpcShuffleClient(new TestingHttpClient(processor, scheduler), location),
                 new Duration(30, TimeUnit.SECONDS),
                 true,
                 location,
@@ -399,13 +404,13 @@ public void testExceptionFromResponseHandler()
     @Test
     public void testErrorCodes()
     {
-        assertEquals(new PageTooLargeException().getErrorCode(), PAGE_TOO_LARGE.toErrorCode());
+        assertEquals(new PageTooLargeException(null).getErrorCode(), PAGE_TOO_LARGE.toErrorCode());
         assertEquals(new PageTransportErrorException("").getErrorCode(), PAGE_TRANSPORT_ERROR.toErrorCode());
         assertEquals(new PageTransportTimeoutException(HostAddress.fromParts("127.0.0.1", 8080), "", null).getErrorCode(), PAGE_TRANSPORT_TIMEOUT.toErrorCode());
     }
 
     private static void assertStatus(
-            HttpPageBufferClient client,
+            PageBufferClient client,
             URI location, String status,
             int pagesReceived,
             int requestsScheduled,
@@ -472,28 +477,28 @@ public Throwable getFailure()
         }
 
         @Override
-        public boolean addPages(HttpPageBufferClient client, List pages)
+        public boolean addPages(PageBufferClient client, List pages)
         {
             this.pages.addAll(pages);
             return true;
         }
 
         @Override
-        public void requestComplete(HttpPageBufferClient client)
+        public void requestComplete(PageBufferClient client)
         {
             completedRequests.getAndIncrement();
             awaitDone();
         }
 
         @Override
-        public void clientFinished(HttpPageBufferClient client)
+        public void clientFinished(PageBufferClient client)
         {
             finishedBuffers.getAndIncrement();
             awaitDone();
         }
 
         @Override
-        public void clientFailed(HttpPageBufferClient client, Throwable cause)
+        public void clientFailed(PageBufferClient client, Throwable cause)
         {
             failedBuffers.getAndIncrement();
             failure.compareAndSet(null, cause);
diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestingDriftClient.java b/presto-main/src/test/java/com/facebook/presto/operator/TestingDriftClient.java
new file mode 100644
index 0000000000000..3485ef46a3bac
--- /dev/null
+++ b/presto-main/src/test/java/com/facebook/presto/operator/TestingDriftClient.java
@@ -0,0 +1,29 @@
+/*
+ * 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.facebook.presto.operator;
+
+import com.facebook.drift.client.DriftClient;
+
+import java.util.Map;
+import java.util.Optional;
+
+public class TestingDriftClient
+        implements DriftClient
+{
+    @Override
+    public T get(Optional addressSelectionContext, Map headers)
+    {
+        throw new UnsupportedOperationException("this client is never expected to be called");
+    }
+}
diff --git a/presto-main/src/test/java/com/facebook/presto/server/TestInternalCommunicationConfig.java b/presto-main/src/test/java/com/facebook/presto/server/TestInternalCommunicationConfig.java
index a152a536aafb4..c25ee93ebff5c 100644
--- a/presto-main/src/test/java/com/facebook/presto/server/TestInternalCommunicationConfig.java
+++ b/presto-main/src/test/java/com/facebook/presto/server/TestInternalCommunicationConfig.java
@@ -13,6 +13,7 @@
  */
 package com.facebook.presto.server;
 
+import com.facebook.presto.server.InternalCommunicationConfig.CommunicationProtocol;
 import com.google.common.collect.ImmutableMap;
 import io.airlift.units.DataSize;
 import org.testng.annotations.Test;
@@ -39,7 +40,9 @@ public void testDefaults()
                 .setExcludeCipherSuites(null)
                 .setKerberosUseCanonicalHostname(true)
                 .setBinaryTransportEnabled(false)
-                .setMaxTaskUpdateSize(new DataSize(16, MEGABYTE)));
+                .setMaxTaskUpdateSize(new DataSize(16, MEGABYTE))
+                .setTaskCommunicationProtocol(CommunicationProtocol.HTTP)
+                .setServerInfoCommunicationProtocol(CommunicationProtocol.HTTP));
     }
 
     @Test
@@ -56,6 +59,8 @@ public void testExplicitPropertyMappings()
                 .put("internal-communication.kerberos.use-canonical-hostname", "false")
                 .put("experimental.internal-communication.binary-transport-enabled", "true")
                 .put("experimental.internal-communication.max-task-update-size", "512MB")
+                .put("internal-communication.task-communication-protocol", "THRIFT")
+                .put("internal-communication.server-info-communication-protocol", "THRIFT")
                 .build();
 
         InternalCommunicationConfig expected = new InternalCommunicationConfig()
@@ -68,7 +73,9 @@ public void testExplicitPropertyMappings()
                 .setKerberosEnabled(true)
                 .setKerberosUseCanonicalHostname(false)
                 .setBinaryTransportEnabled(true)
-                .setMaxTaskUpdateSize(new DataSize(512, MEGABYTE));
+                .setMaxTaskUpdateSize(new DataSize(512, MEGABYTE))
+                .setTaskCommunicationProtocol(CommunicationProtocol.THRIFT)
+                .setServerInfoCommunicationProtocol(CommunicationProtocol.THRIFT);
 
         assertFullMapping(properties, expected);
     }
diff --git a/presto-main/src/test/java/com/facebook/presto/server/TestThriftServerInfoIntegration.java b/presto-main/src/test/java/com/facebook/presto/server/TestThriftServerInfoIntegration.java
new file mode 100644
index 0000000000000..4118ddb0b58a5
--- /dev/null
+++ b/presto-main/src/test/java/com/facebook/presto/server/TestThriftServerInfoIntegration.java
@@ -0,0 +1,240 @@
+/*
+ * 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.facebook.presto.server;
+
+import com.facebook.airlift.bootstrap.Bootstrap;
+import com.facebook.airlift.bootstrap.LifeCycleManager;
+import com.facebook.drift.client.DriftClientFactory;
+import com.facebook.drift.client.address.AddressSelector;
+import com.facebook.drift.client.address.SimpleAddressSelector;
+import com.facebook.drift.codec.ThriftCodecManager;
+import com.facebook.drift.server.DriftServer;
+import com.facebook.drift.transport.netty.client.DriftNettyClientConfig;
+import com.facebook.drift.transport.netty.client.DriftNettyMethodInvokerFactory;
+import com.facebook.drift.transport.netty.server.DriftNettyServerModule;
+import com.facebook.drift.transport.netty.server.DriftNettyServerTransport;
+import com.facebook.presto.Session;
+import com.facebook.presto.execution.StateMachine;
+import com.facebook.presto.execution.TaskId;
+import com.facebook.presto.execution.TaskInfo;
+import com.facebook.presto.execution.TaskManager;
+import com.facebook.presto.execution.TaskSource;
+import com.facebook.presto.execution.TaskState;
+import com.facebook.presto.execution.TaskStatus;
+import com.facebook.presto.execution.buffer.BufferResult;
+import com.facebook.presto.execution.buffer.OutputBuffers;
+import com.facebook.presto.execution.buffer.OutputBuffers.OutputBufferId;
+import com.facebook.presto.execution.scheduler.TableWriteInfo;
+import com.facebook.presto.memory.MemoryPoolAssignmentsRequest;
+import com.facebook.presto.server.testing.TestingPrestoServer;
+import com.facebook.presto.server.thrift.ThriftServerInfoClient;
+import com.facebook.presto.server.thrift.ThriftServerInfoService;
+import com.facebook.presto.spi.NodeState;
+import com.facebook.presto.sql.planner.PlanFragment;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.net.HostAndPort;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.inject.Binder;
+import com.google.inject.Injector;
+import com.google.inject.Module;
+import com.google.inject.Provides;
+import com.google.inject.Scopes;
+import io.airlift.units.DataSize;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import javax.inject.Singleton;
+
+import java.util.List;
+import java.util.Optional;
+
+import static com.facebook.airlift.configuration.ConfigBinder.configBinder;
+import static com.facebook.drift.client.ExceptionClassifier.NORMAL_RESULT;
+import static com.facebook.drift.server.guice.DriftServerBinder.driftServerBinder;
+import static com.facebook.drift.transport.netty.client.DriftNettyMethodInvokerFactory.createStaticDriftNettyMethodInvokerFactory;
+import static com.facebook.presto.spi.NodeState.ACTIVE;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.fail;
+
+public class TestThriftServerInfoIntegration
+{
+    private LifeCycleManager lifeCycleManager;
+    private int thriftServerPort;
+
+    @BeforeClass
+    public void setup()
+            throws Exception
+    {
+        Bootstrap app = new Bootstrap(
+                new DriftNettyServerModule(),
+                new TestingThriftServerInfoModule());
+
+        app.setRequiredConfigurationProperties(ImmutableMap.of("presto.version", "test.0", "coordinator", "false"));
+
+        Injector injector = app
+                .strictConfig()
+                .doNotInitializeLogging()
+                .initialize();
+
+        lifeCycleManager = injector.getInstance(LifeCycleManager.class);
+        thriftServerPort = driftServerPort(injector.getInstance(DriftServer.class));
+    }
+
+    @AfterClass
+    public void teardown()
+    {
+        if (lifeCycleManager != null) {
+            lifeCycleManager.stop();
+        }
+    }
+
+    @Test
+    public void testServer()
+    {
+        AddressSelector addressSelector = new SimpleAddressSelector(
+                ImmutableSet.of(HostAndPort.fromParts("localhost", thriftServerPort)),
+                true);
+        try (DriftNettyMethodInvokerFactory invokerFactory = createStaticDriftNettyMethodInvokerFactory(new DriftNettyClientConfig())) {
+            DriftClientFactory clientFactory = new DriftClientFactory(new ThriftCodecManager(), invokerFactory, addressSelector, NORMAL_RESULT);
+            ThriftServerInfoClient client = clientFactory.createDriftClient(ThriftServerInfoClient.class).get();
+
+            // get buffer result
+            NodeState state = NodeState.valueOf(client.getServerState().get());
+            assertEquals(state, ACTIVE);
+        }
+        catch (Exception e) {
+            fail();
+        }
+    }
+
+    private static int driftServerPort(DriftServer server)
+    {
+        return ((DriftNettyServerTransport) server.getServerTransport()).getPort();
+    }
+
+    public static class TestingThriftServerInfoModule
+            implements Module
+    {
+        @Override
+        public void configure(Binder binder)
+        {
+            configBinder(binder).bindConfig(ServerConfig.class);
+
+            binder.bind(GracefulShutdownHandler.class).in(Scopes.SINGLETON);
+            binder.bind(ShutdownAction.class).to(TestingPrestoServer.TestShutdownAction.class).in(Scopes.SINGLETON);
+
+            binder.bind(ThriftServerInfoService.class).in(Scopes.SINGLETON);
+            driftServerBinder(binder).bindService(ThriftServerInfoService.class);
+        }
+
+        @Provides
+        @Singleton
+        public static TaskManager createTaskManager()
+        {
+            return new TaskManager() {
+                @Override
+                public List getAllTaskInfo()
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public TaskInfo getTaskInfo(TaskId taskId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public TaskStatus getTaskStatus(TaskId taskId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public ListenableFuture getTaskInfo(TaskId taskId, TaskState currentState)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public String getTaskInstanceId(TaskId taskId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public ListenableFuture getTaskStatus(TaskId taskId, TaskState currentState)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public void updateMemoryPoolAssignments(MemoryPoolAssignmentsRequest assignments)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public TaskInfo updateTask(Session session, TaskId taskId, Optional fragment, List sources, OutputBuffers outputBuffers, Optional tableWriteInfo)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public TaskInfo cancelTask(TaskId taskId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public TaskInfo abortTask(TaskId taskId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public ListenableFuture getTaskResults(TaskId taskId, OutputBufferId bufferId, long startingSequenceId, DataSize maxSize)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public void acknowledgeTaskResults(TaskId taskId, OutputBufferId bufferId, long sequenceId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public TaskInfo abortTaskResults(TaskId taskId, OutputBufferId bufferId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public void addStateChangeListener(TaskId taskId, StateMachine.StateChangeListener stateChangeListener)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public void removeRemoteSource(TaskId taskId, TaskId remoteSourceTaskId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+            };
+        }
+    }
+}
diff --git a/presto-main/src/test/java/com/facebook/presto/server/TestThriftTaskIntegration.java b/presto-main/src/test/java/com/facebook/presto/server/TestThriftTaskIntegration.java
new file mode 100644
index 0000000000000..d600f2fe1969e
--- /dev/null
+++ b/presto-main/src/test/java/com/facebook/presto/server/TestThriftTaskIntegration.java
@@ -0,0 +1,278 @@
+/*
+ * 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.facebook.presto.server;
+
+import com.facebook.airlift.bootstrap.Bootstrap;
+import com.facebook.airlift.bootstrap.LifeCycleManager;
+import com.facebook.airlift.concurrent.BoundedExecutor;
+import com.facebook.drift.client.DriftClientFactory;
+import com.facebook.drift.client.address.AddressSelector;
+import com.facebook.drift.client.address.SimpleAddressSelector;
+import com.facebook.drift.codec.ThriftCodecManager;
+import com.facebook.drift.server.DriftServer;
+import com.facebook.drift.transport.netty.client.DriftNettyClientConfig;
+import com.facebook.drift.transport.netty.client.DriftNettyMethodInvokerFactory;
+import com.facebook.drift.transport.netty.server.DriftNettyServerModule;
+import com.facebook.drift.transport.netty.server.DriftNettyServerTransport;
+import com.facebook.presto.Session;
+import com.facebook.presto.execution.StateMachine;
+import com.facebook.presto.execution.TaskId;
+import com.facebook.presto.execution.TaskInfo;
+import com.facebook.presto.execution.TaskManager;
+import com.facebook.presto.execution.TaskSource;
+import com.facebook.presto.execution.TaskState;
+import com.facebook.presto.execution.TaskStatus;
+import com.facebook.presto.execution.buffer.BufferResult;
+import com.facebook.presto.execution.buffer.OutputBuffers;
+import com.facebook.presto.execution.buffer.OutputBuffers.OutputBufferId;
+import com.facebook.presto.execution.scheduler.TableWriteInfo;
+import com.facebook.presto.memory.MemoryPoolAssignmentsRequest;
+import com.facebook.presto.server.thrift.ThriftTaskClient;
+import com.facebook.presto.server.thrift.ThriftTaskService;
+import com.facebook.presto.sql.planner.PlanFragment;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.net.HostAndPort;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.inject.Binder;
+import com.google.inject.Injector;
+import com.google.inject.Module;
+import com.google.inject.Provides;
+import com.google.inject.Scopes;
+import io.airlift.units.DataSize;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import javax.inject.Singleton;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+
+import static com.facebook.airlift.concurrent.Threads.daemonThreadsNamed;
+import static com.facebook.drift.client.ExceptionClassifier.NORMAL_RESULT;
+import static com.facebook.drift.server.guice.DriftServerBinder.driftServerBinder;
+import static com.facebook.drift.transport.netty.client.DriftNettyMethodInvokerFactory.createStaticDriftNettyMethodInvokerFactory;
+import static com.facebook.presto.execution.buffer.BufferResult.emptyResults;
+import static java.util.concurrent.Executors.newCachedThreadPool;
+import static java.util.concurrent.Executors.newScheduledThreadPool;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+
+public class TestThriftTaskIntegration
+{
+    private LifeCycleManager lifeCycleManager;
+    private int thriftServerPort;
+
+    @BeforeClass
+    public void setup()
+            throws Exception
+    {
+        Bootstrap app = new Bootstrap(
+                new DriftNettyServerModule(),
+                new TestingTaskThriftModule());
+
+        Injector injector = app
+                .strictConfig()
+                .doNotInitializeLogging()
+                .initialize();
+
+        lifeCycleManager = injector.getInstance(LifeCycleManager.class);
+        thriftServerPort = driftServerPort(injector.getInstance(DriftServer.class));
+    }
+
+    @AfterClass
+    public void teardown()
+    {
+        if (lifeCycleManager != null) {
+            lifeCycleManager.stop();
+        }
+    }
+
+    @Test
+    public void testServer()
+    {
+        AddressSelector addressSelector = new SimpleAddressSelector(
+                ImmutableSet.of(HostAndPort.fromParts("localhost", thriftServerPort)),
+                true);
+        try (DriftNettyMethodInvokerFactory invokerFactory = createStaticDriftNettyMethodInvokerFactory(new DriftNettyClientConfig())) {
+            DriftClientFactory clientFactory = new DriftClientFactory(new ThriftCodecManager(), invokerFactory, addressSelector, NORMAL_RESULT);
+            ThriftTaskClient client = clientFactory.createDriftClient(ThriftTaskClient.class).get();
+
+            // get buffer result
+            ListenableFuture result = client.getResults(TaskId.valueOf("queryid.0.0.0"), new OutputBufferId(1), 0, 100);
+            assertTrue(result.get().isBufferComplete());
+            assertTrue(result.get().getSerializedPages().isEmpty());
+            assertEquals(result.get().getToken(), 1);
+            assertEquals(result.get().getTaskInstanceId(), "test");
+
+            // ack buffer result
+            client.acknowledgeResults(TaskId.valueOf("queryid.0.0.0"), new OutputBufferId(1), 42).get();    // sync
+            client.acknowledgeResults(TaskId.valueOf("queryid.0.0.0"), new OutputBufferId(1), 42);          // fire and forget
+
+            // abort buffer result
+            client.abortResults(TaskId.valueOf("queryid.0.0.0"), new OutputBufferId(1)).get();
+        }
+        catch (Exception e) {
+            fail();
+        }
+    }
+
+    private static int driftServerPort(DriftServer server)
+    {
+        return ((DriftNettyServerTransport) server.getServerTransport()).getPort();
+    }
+
+    public static class TestingTaskThriftModule
+            implements Module
+    {
+        @Override
+        public void configure(Binder binder)
+        {
+            binder.bind(ThriftTaskService.class).in(Scopes.SINGLETON);
+
+            driftServerBinder(binder).bindService(ThriftTaskService.class);
+        }
+
+        @Provides
+        @Singleton
+        @ForAsyncRpc
+        public static ExecutorService createAsyncHttpResponseCoreExecutor()
+        {
+            return newCachedThreadPool(daemonThreadsNamed("async-http-response-%s"));
+        }
+
+        @Provides
+        @Singleton
+        @ForAsyncRpc
+        public static BoundedExecutor createAsyncHttpResponseExecutor(@ForAsyncRpc ExecutorService coreExecutor)
+        {
+            return new BoundedExecutor(coreExecutor, 100);
+        }
+
+        @Provides
+        @Singleton
+        @ForAsyncRpc
+        public static ScheduledExecutorService createAsyncHttpTimeoutExecutor()
+        {
+            return newScheduledThreadPool(10, daemonThreadsNamed("async-http-timeout-%s"));
+        }
+
+        @Provides
+        @Singleton
+        public static TaskManager createTaskManager()
+        {
+            return new TaskManager() {
+                @Override
+                public List getAllTaskInfo()
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public TaskInfo getTaskInfo(TaskId taskId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public TaskStatus getTaskStatus(TaskId taskId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public ListenableFuture getTaskInfo(TaskId taskId, TaskState currentState)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public String getTaskInstanceId(TaskId taskId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public ListenableFuture getTaskStatus(TaskId taskId, TaskState currentState)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public void updateMemoryPoolAssignments(MemoryPoolAssignmentsRequest assignments)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public TaskInfo updateTask(Session session, TaskId taskId, Optional fragment, List sources, OutputBuffers outputBuffers, Optional tableWriteInfo)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public TaskInfo cancelTask(TaskId taskId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public TaskInfo abortTask(TaskId taskId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public ListenableFuture getTaskResults(TaskId taskId, OutputBufferId bufferId, long startingSequenceId, DataSize maxSize)
+                {
+                    return Futures.immediateFuture(emptyResults("test", 1, true));
+                }
+
+                @Override
+                public void acknowledgeTaskResults(TaskId taskId, OutputBufferId bufferId, long sequenceId)
+                {
+                    assertEquals(taskId, TaskId.valueOf("queryid.0.0.0"));
+                    assertEquals(bufferId, new OutputBufferId(1));
+                    assertEquals(sequenceId, 42);
+                }
+
+                @Override
+                public TaskInfo abortTaskResults(TaskId taskId, OutputBufferId bufferId)
+                {
+                    assertEquals(taskId, TaskId.valueOf("queryid.0.0.0"));
+                    assertEquals(bufferId, new OutputBufferId(1));
+
+                    // null is not going to be consumed
+                    return null;
+                }
+
+                @Override
+                public void addStateChangeListener(TaskId taskId, StateMachine.StateChangeListener stateChangeListener)
+                {
+                    throw new UnsupportedOperationException();
+                }
+
+                @Override
+                public void removeRemoteSource(TaskId taskId, TaskId remoteSourceTaskId)
+                {
+                    throw new UnsupportedOperationException();
+                }
+            };
+        }
+    }
+}
diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml
index 8bd8f10ec7750..1f4ce56f63989 100644
--- a/presto-mongodb/pom.xml
+++ b/presto-mongodb/pom.xml
@@ -15,7 +15,6 @@
         ${project.parent.basedir}
         3.6.0
         1.5.0
-        4.0.32.Final
     
 
     
@@ -23,6 +22,12 @@
             org.mongodb
             mongo-java-driver
             ${mongo-java.version}
+            
+                
+                    io.netty
+                    *
+                
+            
         
 
         
@@ -159,6 +164,12 @@
             de.bwaldvogel
             mongo-java-server-core
             ${mongo-server.version}
+            
+                
+                    io.netty
+                    *
+                
+            
             test
         
 
@@ -174,12 +185,5 @@
             testng
             test
         
-
-        
-            io.netty
-            netty-transport
-            ${netty.version}
-            test
-        
     
 
diff --git a/presto-mongodb/src/test/java/com/facebook/presto/mongodb/SyncMemoryBackend.java b/presto-mongodb/src/test/java/com/facebook/presto/mongodb/SyncMemoryBackend.java
index a53d40a2cc039..828230a075424 100644
--- a/presto-mongodb/src/test/java/com/facebook/presto/mongodb/SyncMemoryBackend.java
+++ b/presto-mongodb/src/test/java/com/facebook/presto/mongodb/SyncMemoryBackend.java
@@ -17,8 +17,6 @@
 import de.bwaldvogel.mongo.backend.memory.MemoryBackend;
 import de.bwaldvogel.mongo.backend.memory.MemoryDatabase;
 import de.bwaldvogel.mongo.exception.MongoServerException;
-import io.netty.channel.Channel;
-import org.bson.BSONObject;
 
 public class SyncMemoryBackend
         extends MemoryBackend
@@ -38,12 +36,5 @@ public SyncMemoryDatabase(MongoBackend backend, String databaseName)
         {
             super(backend, databaseName);
         }
-
-        @Override
-        public synchronized BSONObject handleCommand(Channel channel, String command, BSONObject query)
-                throws MongoServerException
-        {
-            return super.handleCommand(channel, command, query);
-        }
     }
 }
diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/NodeState.java b/presto-spi/src/main/java/com/facebook/presto/spi/NodeState.java
index b156dea8d7575..ade84d631d65b 100644
--- a/presto-spi/src/main/java/com/facebook/presto/spi/NodeState.java
+++ b/presto-spi/src/main/java/com/facebook/presto/spi/NodeState.java
@@ -15,7 +15,35 @@
 
 public enum NodeState
 {
-    ACTIVE,
-    INACTIVE,
-    SHUTTING_DOWN
+    ACTIVE(1),
+    INACTIVE(2),
+    SHUTTING_DOWN(3);
+
+    private final int value;
+
+    NodeState(int value)
+    {
+        this.value = value;
+    }
+
+    /**
+     * Recover NodeState from the ordinal.
+     * In general, ThriftEnum is the right annotation to use.
+     * But given the class is in SPI, use the following workaround.
+     */
+    public static NodeState valueOf(int value)
+    {
+        for (NodeState nodeState : values()) {
+            if (nodeState.getValue() == value) {
+                return nodeState;
+            }
+        }
+        throw new IllegalArgumentException("Invalid NodeState value: " + value);
+    }
+
+    // the value will be used for SerDe like thrift
+    public int getValue()
+    {
+        return value;
+    }
 }