mirror of https://github.com/apache/druid.git
misc http segment discovery fixes (#4618)
* Use ConcurrentHashMap to store segment servers or else getInventory() would need to clone the values list * introduce unstableTimeout for segment servers * address review comment * add HttpServerInventoryViewConfigTest
This commit is contained in:
parent
729e44d767
commit
6d60ef67ce
|
@ -64,9 +64,9 @@ import javax.servlet.http.HttpServletResponse;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.net.URL;
|
import java.net.URL;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.BlockingQueue;
|
import java.util.concurrent.BlockingQueue;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.Executor;
|
import java.util.concurrent.Executor;
|
||||||
|
@ -96,7 +96,7 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
private volatile Predicate<Pair<DruidServerMetadata, DataSegment>> finalPredicate;
|
private volatile Predicate<Pair<DruidServerMetadata, DataSegment>> finalPredicate;
|
||||||
|
|
||||||
// For each queryable server, a name -> DruidServerHolder entry is kept
|
// For each queryable server, a name -> DruidServerHolder entry is kept
|
||||||
private final Map<String, DruidServerHolder> servers = new HashMap<>();
|
private final ConcurrentHashMap<String, DruidServerHolder> servers = new ConcurrentHashMap<>();
|
||||||
|
|
||||||
private volatile ExecutorService executor;
|
private volatile ExecutorService executor;
|
||||||
|
|
||||||
|
@ -159,13 +159,9 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
|
|
||||||
while (!Thread.interrupted() && lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) {
|
while (!Thread.interrupted() && lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) {
|
||||||
try {
|
try {
|
||||||
String name = queue.take();
|
DruidServerHolder holder = servers.get(queue.take());
|
||||||
|
if (holder != null) {
|
||||||
synchronized (servers) {
|
holder.updateSegmentsListAsync();
|
||||||
DruidServerHolder holder = servers.get(name);
|
|
||||||
if (holder != null) {
|
|
||||||
holder.updateSegmentsListAsync();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (InterruptedException ex) {
|
catch (InterruptedException ex) {
|
||||||
|
@ -274,31 +270,26 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
@Override
|
@Override
|
||||||
public DruidServer getInventoryValue(String containerKey)
|
public DruidServer getInventoryValue(String containerKey)
|
||||||
{
|
{
|
||||||
synchronized (servers) {
|
DruidServerHolder holder = servers.get(containerKey);
|
||||||
DruidServerHolder holder = servers.get(containerKey);
|
if (holder != null) {
|
||||||
if (holder != null) {
|
return holder.druidServer;
|
||||||
return holder.druidServer;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterable<DruidServer> getInventory()
|
public Iterable<DruidServer> getInventory()
|
||||||
{
|
{
|
||||||
synchronized (servers) {
|
return Iterables.transform(
|
||||||
return Iterables.transform(
|
servers.values(), new Function<DruidServerHolder, DruidServer>()
|
||||||
servers.values(), new Function<DruidServerHolder, DruidServer>()
|
{
|
||||||
|
@Override
|
||||||
|
public DruidServer apply(DruidServerHolder input)
|
||||||
{
|
{
|
||||||
@Override
|
return input.druidServer;
|
||||||
public DruidServer apply(DruidServerHolder input)
|
|
||||||
{
|
|
||||||
return input.druidServer;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
);
|
}
|
||||||
}
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void runSegmentCallbacks(
|
private void runSegmentCallbacks(
|
||||||
|
@ -369,24 +360,17 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
|
|
||||||
private DruidServer serverAddedOrUpdated(DruidServer server)
|
private DruidServer serverAddedOrUpdated(DruidServer server)
|
||||||
{
|
{
|
||||||
DruidServerHolder curr;
|
DruidServerHolder newHolder = servers.compute(
|
||||||
DruidServerHolder newHolder;
|
server.getName(),
|
||||||
synchronized (servers) {
|
(k, v) -> v == null ? new DruidServerHolder(server) : v.updatedHolder(server)
|
||||||
curr = servers.get(server.getName());
|
);
|
||||||
newHolder = curr == null ? new DruidServerHolder(server) : curr.updatedHolder(server);
|
|
||||||
servers.put(server.getName(), newHolder);
|
|
||||||
}
|
|
||||||
|
|
||||||
newHolder.updateSegmentsListAsync();
|
newHolder.updateSegmentsListAsync();
|
||||||
|
|
||||||
return newHolder.druidServer;
|
return newHolder.druidServer;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void serverRemoved(DruidServer server)
|
private void serverRemoved(DruidServer server)
|
||||||
{
|
{
|
||||||
synchronized (servers) {
|
servers.remove(server.getName());
|
||||||
servers.remove(server.getName());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public DruidServer serverUpdated(DruidServer oldServer, DruidServer newServer)
|
public DruidServer serverUpdated(DruidServer oldServer, DruidServer newServer)
|
||||||
|
@ -403,14 +387,8 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
@Override
|
@Override
|
||||||
public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment)
|
public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment)
|
||||||
{
|
{
|
||||||
synchronized (servers) {
|
DruidServerHolder holder = servers.get(serverKey);
|
||||||
DruidServerHolder holder = servers.get(serverKey);
|
return holder != null && holder.druidServer.getSegment(segment.getIdentifier()) != null;
|
||||||
if (holder != null) {
|
|
||||||
return holder.druidServer.getSegment(segment.getIdentifier()) != null;
|
|
||||||
} else {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private class DruidServerHolder
|
private class DruidServerHolder
|
||||||
|
@ -431,6 +409,9 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
|
|
||||||
private final CountDownLatch initializationLatch = new CountDownLatch(1);
|
private final CountDownLatch initializationLatch = new CountDownLatch(1);
|
||||||
|
|
||||||
|
private volatile boolean isUnstable = false;
|
||||||
|
private volatile long unstableStartTime = -1;
|
||||||
|
|
||||||
DruidServerHolder(DruidServer druidServer)
|
DruidServerHolder(DruidServer druidServer)
|
||||||
{
|
{
|
||||||
this(druidServer, null);
|
this(druidServer, null);
|
||||||
|
@ -588,6 +569,7 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
}
|
}
|
||||||
|
|
||||||
initializationLatch.countDown();
|
initializationLatch.countDown();
|
||||||
|
isUnstable = false;
|
||||||
}
|
}
|
||||||
catch (Exception ex) {
|
catch (Exception ex) {
|
||||||
log.error(ex, "error processing segment list response from server [%s]", druidServer.getName());
|
log.error(ex, "error processing segment list response from server [%s]", druidServer.getName());
|
||||||
|
@ -601,21 +583,26 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
public void onFailure(Throwable t)
|
public void onFailure(Throwable t)
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
if (t != null) {
|
String logMsg = StringUtils.nonStrictFormat(
|
||||||
log.error(
|
"failed to fetch segment list from server [%s]. Return code [%s], Reason: [%s]",
|
||||||
t,
|
druidServer.getName(),
|
||||||
"failed to fetch segment list from server [%s]. Return code [%s], Reason: [%s]",
|
responseHandler.status,
|
||||||
druidServer.getName(),
|
responseHandler.description
|
||||||
responseHandler.status,
|
);
|
||||||
responseHandler.description
|
|
||||||
);
|
if (hasUnstabilityTimeoutPassed()) {
|
||||||
|
if (t != null) {
|
||||||
|
log.error(t, logMsg);
|
||||||
|
} else {
|
||||||
|
log.error(logMsg);
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
log.error(
|
log.info("Temporary Failure. %s", logMsg);
|
||||||
"failed to fetch segment list from server [%s]. Return code [%s], Reason: [%s]",
|
if (t != null) {
|
||||||
druidServer.getName(),
|
log.debug(t, logMsg);
|
||||||
responseHandler.status,
|
} else {
|
||||||
responseHandler.description
|
log.debug(logMsg);
|
||||||
);
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// sleep for a bit so that retry does not happen immediately.
|
// sleep for a bit so that retry does not happen immediately.
|
||||||
|
@ -638,7 +625,17 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
}
|
}
|
||||||
catch (Throwable th) {
|
catch (Throwable th) {
|
||||||
queue.add(druidServer.getName());
|
queue.add(druidServer.getName());
|
||||||
log.makeAlert(th, "Fatal error while fetching segment list from server [%s].", druidServer.getName()).emit();
|
|
||||||
|
String logMsg = StringUtils.nonStrictFormat(
|
||||||
|
"Fatal error while fetching segment list from server [%s].", druidServer.getName()
|
||||||
|
);
|
||||||
|
|
||||||
|
if (hasUnstabilityTimeoutPassed()) {
|
||||||
|
log.makeAlert(th, logMsg).emit();
|
||||||
|
} else {
|
||||||
|
log.info("Temporary Failure. %s", logMsg);
|
||||||
|
log.debug(th, logMsg);
|
||||||
|
}
|
||||||
|
|
||||||
// sleep for a bit so that retry does not happen immediately.
|
// sleep for a bit so that retry does not happen immediately.
|
||||||
try {
|
try {
|
||||||
|
@ -651,6 +648,20 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer
|
||||||
throw Throwables.propagate(th);
|
throw Throwables.propagate(th);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private boolean hasUnstabilityTimeoutPassed()
|
||||||
|
{
|
||||||
|
if (isUnstable && (System.currentTimeMillis() - unstableStartTime) > config.getServerUnstabilityTimeout()) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!isUnstable) {
|
||||||
|
isUnstable = true;
|
||||||
|
unstableStartTime = System.currentTimeMillis();
|
||||||
|
}
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class BytesAccumulatingResponseHandler extends InputStreamResponseHandler
|
private static class BytesAccumulatingResponseHandler extends InputStreamResponseHandler
|
||||||
|
|
|
@ -24,25 +24,39 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.joda.time.Period;
|
import org.joda.time.Period;
|
||||||
|
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class HttpServerInventoryViewConfig
|
public class HttpServerInventoryViewConfig
|
||||||
{
|
{
|
||||||
|
// HTTP request timeout
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
private final long serverTimeout;
|
private final long serverTimeout;
|
||||||
|
|
||||||
|
// Requests to server may fail when it is shutsdown abruptly and there is a lag in coordinator
|
||||||
|
// discovering its disappearance. So, failure would be logged only after the acceptable
|
||||||
|
// unstableTimeout has passed.
|
||||||
|
@JsonProperty
|
||||||
|
private final long serverUnstabilityTimeout;
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
private final int numThreads;
|
private final int numThreads;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public HttpServerInventoryViewConfig(
|
public HttpServerInventoryViewConfig(
|
||||||
@JsonProperty("serverTimeout") Period serverTimeout,
|
@JsonProperty("serverTimeout") Period serverTimeout,
|
||||||
|
@JsonProperty("serverUnstabilityTimeout") Period serverUnstabilityTimeout,
|
||||||
@JsonProperty("numThreads") Integer numThreads
|
@JsonProperty("numThreads") Integer numThreads
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.serverTimeout = serverTimeout != null
|
this.serverTimeout = serverTimeout != null
|
||||||
? serverTimeout.toStandardDuration().getMillis()
|
? serverTimeout.toStandardDuration().getMillis()
|
||||||
: 4*60*1000; //4 mins
|
: TimeUnit.MINUTES.toMillis(4);
|
||||||
|
|
||||||
|
this.serverUnstabilityTimeout = serverUnstabilityTimeout != null
|
||||||
|
? serverUnstabilityTimeout.toStandardDuration().getMillis()
|
||||||
|
: TimeUnit.MINUTES.toMillis(1);
|
||||||
|
|
||||||
this.numThreads = numThreads != null ? numThreads.intValue() : 5;
|
this.numThreads = numThreads != null ? numThreads.intValue() : 5;
|
||||||
|
|
||||||
|
@ -55,6 +69,11 @@ public class HttpServerInventoryViewConfig
|
||||||
return serverTimeout;
|
return serverTimeout;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public long getServerUnstabilityTimeout()
|
||||||
|
{
|
||||||
|
return serverUnstabilityTimeout;
|
||||||
|
}
|
||||||
|
|
||||||
public int getNumThreads()
|
public int getNumThreads()
|
||||||
{
|
{
|
||||||
return numThreads;
|
return numThreads;
|
||||||
|
|
|
@ -0,0 +1,59 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you 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 io.druid.client;
|
||||||
|
|
||||||
|
import io.druid.segment.TestHelper;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class HttpServerInventoryViewConfigTest
|
||||||
|
{
|
||||||
|
@Test
|
||||||
|
public void testDeserializationWithDefaults() throws Exception
|
||||||
|
{
|
||||||
|
String json = "{}";
|
||||||
|
|
||||||
|
HttpServerInventoryViewConfig config = TestHelper.getJsonMapper().readValue(json, HttpServerInventoryViewConfig.class);
|
||||||
|
|
||||||
|
Assert.assertEquals(TimeUnit.MINUTES.toMillis(4), config.getServerTimeout());
|
||||||
|
Assert.assertEquals(TimeUnit.MINUTES.toMillis(1), config.getServerUnstabilityTimeout());
|
||||||
|
Assert.assertEquals(5, config.getNumThreads());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDeserializationWithNonDefaults() throws Exception
|
||||||
|
{
|
||||||
|
String json = "{\n"
|
||||||
|
+ " \"serverTimeout\": \"PT2M\",\n"
|
||||||
|
+ " \"serverUnstabilityTimeout\": \"PT3M\",\n"
|
||||||
|
+ " \"numThreads\": 7\n"
|
||||||
|
+ "}";
|
||||||
|
|
||||||
|
HttpServerInventoryViewConfig config = TestHelper.getJsonMapper().readValue(json, HttpServerInventoryViewConfig.class);
|
||||||
|
|
||||||
|
Assert.assertEquals(TimeUnit.MINUTES.toMillis(2), config.getServerTimeout());
|
||||||
|
Assert.assertEquals(TimeUnit.MINUTES.toMillis(3), config.getServerUnstabilityTimeout());
|
||||||
|
Assert.assertEquals(7, config.getNumThreads());
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue