Skip to content

Commit

Permalink
Abstract RemoteNodeState HTTP implementation to HttpRemoteNodeState
Browse files Browse the repository at this point in the history
  • Loading branch information
highker committed Jan 17, 2020
1 parent 7416a8c commit 8ffd650
Show file tree
Hide file tree
Showing 3 changed files with 121 additions and 93 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -177,7 +177,7 @@ 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()));
new HttpRemoteNodeState(httpClient, uriBuilderFrom(node.getInternalUri()).appendPath("/v1/info/state").build()));
}

// Schedule refresh
Expand Down
Original file line number Diff line number Diff line change
@@ -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<Optional<NodeState>> nodeState = new AtomicReference<>(Optional.empty());
private final AtomicReference<Future<?>> 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<NodeState> 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<JsonResponse<NodeState>> responseFuture = httpClient.executeAsync(request, createFullJsonResponseHandler(jsonCodec(NodeState.class)));
future.compareAndSet(null, responseFuture);

Futures.addCallback(responseFuture, new FutureCallback<JsonResponse<NodeState>>()
{
@Override
public void onSuccess(@Nullable JsonResponse<NodeState> 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());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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<Optional<NodeState>> nodeState = new AtomicReference<>(Optional.empty());
private final AtomicReference<Future<?>> 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<NodeState> 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<JsonResponse<NodeState>> responseFuture = httpClient.executeAsync(request, createFullJsonResponseHandler(jsonCodec(NodeState.class)));
future.compareAndSet(null, responseFuture);

Futures.addCallback(responseFuture, new FutureCallback<JsonResponse<NodeState>>()
{
@Override
public void onSuccess(@Nullable JsonResponse<NodeState> 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<NodeState> 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();
}

0 comments on commit 8ffd650

Please sign in to comment.