mirror of https://github.com/apache/druid.git
Add a ServerType for peons (#4295)
* Add a ServerType for peons * Add toString() method, toString() test, unsupported type check * Use ServerType enum in DruidServer and DruidServerMetadata
This commit is contained in:
parent
8ec3a29af0
commit
e043bf88ec
|
@ -58,7 +58,7 @@ public class DruidServer implements Comparable
|
|||
public DruidServer(
|
||||
DruidNode node,
|
||||
DruidServerConfig config,
|
||||
String type
|
||||
ServerType type
|
||||
)
|
||||
{
|
||||
this(
|
||||
|
@ -76,7 +76,7 @@ public class DruidServer implements Comparable
|
|||
@JsonProperty("name") String name,
|
||||
@JsonProperty("host") String host,
|
||||
@JsonProperty("maxSize") long maxSize,
|
||||
@JsonProperty("type") String type,
|
||||
@JsonProperty("type") ServerType type,
|
||||
@JsonProperty("tier") String tier,
|
||||
@JsonProperty("priority") int priority
|
||||
)
|
||||
|
@ -87,6 +87,7 @@ public class DruidServer implements Comparable
|
|||
this.segments = new ConcurrentHashMap<String, DataSegment>();
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getName()
|
||||
{
|
||||
return metadata.getName();
|
||||
|
@ -97,6 +98,7 @@ public class DruidServer implements Comparable
|
|||
return metadata;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getHost()
|
||||
{
|
||||
return metadata.getHost();
|
||||
|
@ -107,16 +109,19 @@ public class DruidServer implements Comparable
|
|||
return currSize;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getMaxSize()
|
||||
{
|
||||
return metadata.getMaxSize();
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public ServerType getType()
|
||||
{
|
||||
return metadata.getType();
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getTier()
|
||||
{
|
||||
return metadata.getTier();
|
||||
|
@ -127,6 +132,7 @@ public class DruidServer implements Comparable
|
|||
return metadata.segmentReplicatable();
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getPriority()
|
||||
{
|
||||
return metadata.getPriority();
|
||||
|
|
|
@ -19,20 +19,22 @@
|
|||
|
||||
package io.druid.guice;
|
||||
|
||||
import io.druid.server.coordination.ServerType;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class NodeTypeConfig
|
||||
{
|
||||
private final String nodeType;
|
||||
private final ServerType nodeType;
|
||||
|
||||
public NodeTypeConfig(
|
||||
String nodeType
|
||||
ServerType nodeType
|
||||
)
|
||||
{
|
||||
this.nodeType = nodeType;
|
||||
}
|
||||
|
||||
public String getNodeType()
|
||||
public ServerType getNodeType()
|
||||
{
|
||||
return nodeType;
|
||||
}
|
||||
|
|
|
@ -38,7 +38,7 @@ public class DruidServerMetadata
|
|||
@JsonProperty("name") String name,
|
||||
@JsonProperty("host") String host,
|
||||
@JsonProperty("maxSize") long maxSize,
|
||||
@JsonProperty("type") String type,
|
||||
@JsonProperty("type") ServerType type,
|
||||
@JsonProperty("tier") String tier,
|
||||
@JsonProperty("priority") int priority
|
||||
)
|
||||
|
@ -47,7 +47,7 @@ public class DruidServerMetadata
|
|||
this.host = host;
|
||||
this.maxSize = maxSize;
|
||||
this.tier = tier;
|
||||
this.type = ServerType.fromString(type);
|
||||
this.type = type;
|
||||
this.priority = priority;
|
||||
}
|
||||
|
||||
|
|
|
@ -19,10 +19,33 @@
|
|||
|
||||
package io.druid.server.coordination;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonValue;
|
||||
|
||||
/**
|
||||
* This enum represents types of druid services that hold segments.
|
||||
* <p>
|
||||
* These types are externally visible (e.g., from the output of /druid/coordinator/v1/servers).
|
||||
* <p>
|
||||
* For backwards compatibility, when presenting these types externally, the toString() representation
|
||||
* of the enum should be used.
|
||||
* <p>
|
||||
* The toString() method converts the enum name() to lowercase and replaces underscores with hyphens,
|
||||
* which is the format expected for the server type string prior to the patch that introduced ServerType:
|
||||
* https://github.com/druid-io/druid/pull/4148
|
||||
*/
|
||||
public enum ServerType
|
||||
{
|
||||
HISTORICAL,
|
||||
BRIDGE,
|
||||
INDEXER_EXECUTOR {
|
||||
@Override
|
||||
public boolean isSegmentReplicationTarget()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
},
|
||||
|
||||
REALTIME {
|
||||
@Override
|
||||
public boolean isSegmentReplicationTarget()
|
||||
|
@ -33,12 +56,12 @@ public enum ServerType
|
|||
|
||||
/**
|
||||
* Indicates this type of node is able to be a target of segment replication.
|
||||
|
||||
*
|
||||
* @return true if it is available for replication
|
||||
*
|
||||
* @see io.druid.server.coordinator.rules.LoadRule
|
||||
*/
|
||||
boolean isSegmentReplicationTarget()
|
||||
public boolean isSegmentReplicationTarget()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
@ -48,13 +71,21 @@ public enum ServerType
|
|||
*
|
||||
* @return true if it is available for broadcast.
|
||||
*/
|
||||
boolean isSegmentBroadcastTarget()
|
||||
public boolean isSegmentBroadcastTarget()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
static ServerType fromString(String type)
|
||||
@JsonCreator
|
||||
public static ServerType fromString(String type)
|
||||
{
|
||||
return ServerType.valueOf(type.toUpperCase());
|
||||
return ServerType.valueOf(type.toUpperCase().replace("-", "_"));
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonValue
|
||||
public String toString()
|
||||
{
|
||||
return name().toLowerCase().replace("_", "-");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -69,6 +69,8 @@ public class DruidCluster
|
|||
case BRIDGE:
|
||||
addHistorical(serverHolder);
|
||||
break;
|
||||
case INDEXER_EXECUTOR:
|
||||
throw new IAE("unsupported server type[%s]", serverHolder.getServer().getType());
|
||||
default:
|
||||
throw new IAE("unknown server type[%s]", serverHolder.getServer().getType());
|
||||
}
|
||||
|
|
|
@ -51,7 +51,7 @@ public class ServersResource
|
|||
return new ImmutableMap.Builder<String, Object>()
|
||||
.put("host", input.getHost())
|
||||
.put("tier", input.getTier())
|
||||
.put("type", input.getType())
|
||||
.put("type", input.getType().toString())
|
||||
.put("priority", input.getPriority())
|
||||
.put("currSize", input.getCurrSize())
|
||||
.put("maxSize", input.getMaxSize())
|
||||
|
@ -63,7 +63,7 @@ public class ServersResource
|
|||
return new ImmutableMap.Builder<String, Object>()
|
||||
.put("host", input.getHost())
|
||||
.put("maxSize", input.getMaxSize())
|
||||
.put("type", input.getType())
|
||||
.put("type", input.getType().toString())
|
||||
.put("tier", input.getTier())
|
||||
.put("priority", input.getPriority())
|
||||
.put("segments", input.getSegments())
|
||||
|
|
|
@ -39,6 +39,7 @@ import io.druid.query.QueryToolChestWarehouse;
|
|||
import io.druid.query.QueryWatcher;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import io.druid.server.metrics.NoopServiceEmitter;
|
||||
import io.druid.timeline.DataSegment;
|
||||
|
@ -98,7 +99,7 @@ public class BrokerServerViewTest extends CuratorTestBase
|
|||
"localhost:1234",
|
||||
"localhost:1234",
|
||||
10000000L,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"default_tier",
|
||||
0
|
||||
);
|
||||
|
@ -164,7 +165,7 @@ public class BrokerServerViewTest extends CuratorTestBase
|
|||
input,
|
||||
input,
|
||||
10000000L,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"default_tier",
|
||||
0
|
||||
);
|
||||
|
|
|
@ -35,6 +35,7 @@ import io.druid.query.Query;
|
|||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.VersionedIntervalTimeline;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
|
@ -158,7 +159,7 @@ public class CachingClusteredClientFunctionalityTest {
|
|||
@Override
|
||||
public QueryableDruidServer pick(TreeMap<Integer, Set<QueryableDruidServer>> prioritizedServers, DataSegment segment) {
|
||||
return new QueryableDruidServer(
|
||||
new DruidServer("localhost", "localhost", 100, "historical", "a", 10),
|
||||
new DruidServer("localhost", "localhost", 100, ServerType.HISTORICAL, "a", 10),
|
||||
EasyMock.createNiceMock(DirectDruidClient.class)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -122,6 +122,7 @@ import io.druid.query.topn.TopNQueryConfig;
|
|||
import io.druid.query.topn.TopNQueryQueryToolChest;
|
||||
import io.druid.query.topn.TopNResultValue;
|
||||
import io.druid.segment.TestHelper;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.VersionedIntervalTimeline;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
|
@ -328,11 +329,11 @@ public class CachingClusteredClientTest
|
|||
client = makeClient(MoreExecutors.sameThreadExecutor());
|
||||
|
||||
servers = new DruidServer[]{
|
||||
new DruidServer("test1", "test1", 10, "historical", "bye", 0),
|
||||
new DruidServer("test2", "test2", 10, "historical", "bye", 0),
|
||||
new DruidServer("test3", "test3", 10, "historical", "bye", 0),
|
||||
new DruidServer("test4", "test4", 10, "historical", "bye", 0),
|
||||
new DruidServer("test5", "test5", 10, "historical", "bye", 0)
|
||||
new DruidServer("test1", "test1", 10, ServerType.HISTORICAL, "bye", 0),
|
||||
new DruidServer("test2", "test2", 10, ServerType.HISTORICAL, "bye", 0),
|
||||
new DruidServer("test3", "test3", 10, ServerType.HISTORICAL, "bye", 0),
|
||||
new DruidServer("test4", "test4", 10, ServerType.HISTORICAL, "bye", 0),
|
||||
new DruidServer("test5", "test5", 10, ServerType.HISTORICAL, "bye", 0)
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -32,6 +32,7 @@ import io.druid.jackson.DefaultObjectMapper;
|
|||
import io.druid.java.util.common.Pair;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.TimelineLookup;
|
||||
|
@ -91,7 +92,7 @@ public class CoordinatorServerViewTest extends CuratorTestBase
|
|||
"localhost:1234",
|
||||
"localhost:1234",
|
||||
10000000L,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"default_tier",
|
||||
0
|
||||
);
|
||||
|
@ -158,7 +159,7 @@ public class CoordinatorServerViewTest extends CuratorTestBase
|
|||
input,
|
||||
input,
|
||||
10000000L,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"default_tier",
|
||||
0
|
||||
);
|
||||
|
|
|
@ -42,6 +42,7 @@ import io.druid.query.QueryRunnerTestHelper;
|
|||
import io.druid.query.ReflectionQueryToolChestWarehouse;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.timeboundary.TimeBoundaryQuery;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.metrics.NoopServiceEmitter;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
|
@ -149,12 +150,12 @@ public class DirectDruidClientTest
|
|||
);
|
||||
|
||||
QueryableDruidServer queryableDruidServer1 = new QueryableDruidServer(
|
||||
new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0),
|
||||
new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0),
|
||||
client1
|
||||
);
|
||||
serverSelector.addServerAndUpdateSegment(queryableDruidServer1, serverSelector.getSegment());
|
||||
QueryableDruidServer queryableDruidServer2 = new QueryableDruidServer(
|
||||
new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0),
|
||||
new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0),
|
||||
client2
|
||||
);
|
||||
serverSelector.addServerAndUpdateSegment(queryableDruidServer2, serverSelector.getSegment());
|
||||
|
@ -253,7 +254,7 @@ public class DirectDruidClientTest
|
|||
);
|
||||
|
||||
QueryableDruidServer queryableDruidServer1 = new QueryableDruidServer(
|
||||
new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0),
|
||||
new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0),
|
||||
client1
|
||||
);
|
||||
serverSelector.addServerAndUpdateSegment(queryableDruidServer1, serverSelector.getSegment());
|
||||
|
@ -322,7 +323,7 @@ public class DirectDruidClientTest
|
|||
);
|
||||
|
||||
QueryableDruidServer queryableDruidServer = new QueryableDruidServer(
|
||||
new DruidServer("test1", hostName, 0, "historical", DruidServer.DEFAULT_TIER, 0),
|
||||
new DruidServer("test1", hostName, 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0),
|
||||
client1
|
||||
);
|
||||
|
||||
|
|
|
@ -43,6 +43,7 @@ import io.druid.server.coordination.BatchDataSegmentAnnouncer;
|
|||
import io.druid.server.coordination.CuratorDataSegmentServerAnnouncer;
|
||||
import io.druid.server.coordination.DataSegmentServerAnnouncer;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.initialization.BatchDataSegmentAnnouncerConfig;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import io.druid.timeline.DataSegment;
|
||||
|
@ -124,7 +125,7 @@ public class BatchServerInventoryViewTest
|
|||
"id",
|
||||
"host",
|
||||
Long.MAX_VALUE,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"tier",
|
||||
0
|
||||
);
|
||||
|
@ -443,7 +444,7 @@ public class BatchServerInventoryViewTest
|
|||
"id",
|
||||
"host",
|
||||
Long.MAX_VALUE,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"tier",
|
||||
0
|
||||
),
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.Sets;
|
|||
import io.druid.client.ImmutableSegmentLoadInfo;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
import junit.framework.Assert;
|
||||
|
@ -51,7 +52,7 @@ public class ImmutableSegmentLoadInfoTest
|
|||
null,
|
||||
NoneShardSpec.instance(),
|
||||
0, 0
|
||||
), Sets.newHashSet(new DruidServerMetadata("a", "host", 10, "historical", "tier", 1))
|
||||
), Sets.newHashSet(new DruidServerMetadata("a", "host", 10, ServerType.HISTORICAL, "tier", 1))
|
||||
);
|
||||
|
||||
ImmutableSegmentLoadInfo serde = mapper.readValue(
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.collect.Lists;
|
|||
import io.druid.client.DirectDruidClient;
|
||||
import io.druid.client.DruidServer;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
import org.easymock.EasyMock;
|
||||
|
@ -43,11 +44,11 @@ public class TierSelectorStrategyTest
|
|||
{
|
||||
DirectDruidClient client = EasyMock.createMock(DirectDruidClient.class);
|
||||
QueryableDruidServer lowPriority = new QueryableDruidServer(
|
||||
new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0),
|
||||
new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0),
|
||||
client
|
||||
);
|
||||
QueryableDruidServer highPriority = new QueryableDruidServer(
|
||||
new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 1),
|
||||
new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 1),
|
||||
client
|
||||
);
|
||||
|
||||
|
@ -62,11 +63,11 @@ public class TierSelectorStrategyTest
|
|||
{
|
||||
DirectDruidClient client = EasyMock.createMock(DirectDruidClient.class);
|
||||
QueryableDruidServer lowPriority = new QueryableDruidServer(
|
||||
new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0),
|
||||
new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0),
|
||||
client
|
||||
);
|
||||
QueryableDruidServer highPriority = new QueryableDruidServer(
|
||||
new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 1),
|
||||
new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 1),
|
||||
client
|
||||
);
|
||||
|
||||
|
@ -81,15 +82,15 @@ public class TierSelectorStrategyTest
|
|||
{
|
||||
DirectDruidClient client = EasyMock.createMock(DirectDruidClient.class);
|
||||
QueryableDruidServer lowPriority = new QueryableDruidServer(
|
||||
new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, -1),
|
||||
new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, -1),
|
||||
client
|
||||
);
|
||||
QueryableDruidServer mediumPriority = new QueryableDruidServer(
|
||||
new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0),
|
||||
new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 0),
|
||||
client
|
||||
);
|
||||
QueryableDruidServer highPriority = new QueryableDruidServer(
|
||||
new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 1),
|
||||
new DruidServer("test1", "localhost", 0, ServerType.HISTORICAL, DruidServer.DEFAULT_TIER, 1),
|
||||
client
|
||||
);
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ package io.druid.query;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -41,9 +42,9 @@ public class LocatedSegmentDescriptorSerdeTest
|
|||
new SegmentDescriptor(new Interval(100, 200), "version", 100),
|
||||
65535,
|
||||
Arrays.asList(
|
||||
new DruidServerMetadata("server1", "host1", 30000L, "historical", "tier1", 0),
|
||||
new DruidServerMetadata("server2", "host2", 40000L, "historical", "tier1", 1),
|
||||
new DruidServerMetadata("server3", "host3", 50000L, "realtime", "tier2", 2)
|
||||
new DruidServerMetadata("server1", "host1", 30000L, ServerType.HISTORICAL, "tier1", 0),
|
||||
new DruidServerMetadata("server2", "host2", 40000L, ServerType.HISTORICAL, "tier1", 1),
|
||||
new DruidServerMetadata("server3", "host3", 50000L, ServerType.REALTIME, "tier2", 2)
|
||||
)
|
||||
);
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import io.druid.client.ImmutableSegmentLoadInfo;
|
|||
import io.druid.client.coordinator.CoordinatorClient;
|
||||
import io.druid.query.SegmentDescriptor;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.partition.NumberedShardSpec;
|
||||
import junit.framework.Assert;
|
||||
|
@ -318,15 +319,15 @@ public class CoordinatorBasedSegmentHandoffNotifierTest
|
|||
|
||||
private DruidServerMetadata createRealtimeServerMetadata(String name)
|
||||
{
|
||||
return createServerMetadata(name, "realtime");
|
||||
return createServerMetadata(name, ServerType.REALTIME);
|
||||
}
|
||||
|
||||
private DruidServerMetadata createHistoricalServerMetadata(String name)
|
||||
{
|
||||
return createServerMetadata(name, "historical");
|
||||
return createServerMetadata(name, ServerType.HISTORICAL);
|
||||
}
|
||||
|
||||
private DruidServerMetadata createServerMetadata(String name, String type)
|
||||
private DruidServerMetadata createServerMetadata(String name, ServerType type)
|
||||
{
|
||||
return new DruidServerMetadata(
|
||||
name,
|
||||
|
|
|
@ -46,6 +46,7 @@ import io.druid.client.TimelineServerView;
|
|||
import io.druid.client.selector.ServerSelector;
|
||||
import io.druid.query.TableDataSource;
|
||||
import io.druid.query.metadata.SegmentMetadataQueryConfig;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.security.AuthConfig;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.VersionedIntervalTimeline;
|
||||
|
@ -79,7 +80,7 @@ public class ClientInfoResourceTest
|
|||
public void setup()
|
||||
{
|
||||
VersionedIntervalTimeline<String, ServerSelector> timeline = new VersionedIntervalTimeline<>(Ordering.<String>natural());
|
||||
DruidServer server = new DruidServer("name", "host", 1234, "historical", "tier", 0);
|
||||
DruidServer server = new DruidServer("name", "host", 1234, ServerType.HISTORICAL, "tier", 0);
|
||||
|
||||
addSegment(timeline, server, "1960-02-13/1961-02-14", ImmutableList.of("d5"), ImmutableList.of("m5"), "v0");
|
||||
|
||||
|
|
|
@ -30,6 +30,7 @@ public class ServerTypeTest
|
|||
Assert.assertTrue(ServerType.HISTORICAL.isSegmentReplicationTarget());
|
||||
Assert.assertTrue(ServerType.BRIDGE.isSegmentReplicationTarget());
|
||||
Assert.assertFalse(ServerType.REALTIME.isSegmentReplicationTarget());
|
||||
Assert.assertFalse(ServerType.INDEXER_EXECUTOR.isSegmentReplicationTarget());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -38,6 +39,16 @@ public class ServerTypeTest
|
|||
Assert.assertEquals(ServerType.HISTORICAL, ServerType.fromString("historical"));
|
||||
Assert.assertEquals(ServerType.BRIDGE, ServerType.fromString("bridge"));
|
||||
Assert.assertEquals(ServerType.REALTIME, ServerType.fromString("realtime"));
|
||||
Assert.assertEquals(ServerType.INDEXER_EXECUTOR, ServerType.fromString("indexer-executor"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToString()
|
||||
{
|
||||
Assert.assertEquals(ServerType.HISTORICAL.toString(), "historical");
|
||||
Assert.assertEquals(ServerType.BRIDGE.toString(), "bridge");
|
||||
Assert.assertEquals(ServerType.REALTIME.toString(), "realtime");
|
||||
Assert.assertEquals(ServerType.INDEXER_EXECUTOR.toString(), "indexer-executor");
|
||||
}
|
||||
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
|
|
|
@ -82,7 +82,7 @@ public class ZkCoordinatorTest extends CuratorTestBase
|
|||
"dummyServer",
|
||||
"dummyHost",
|
||||
0,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
);
|
||||
|
@ -515,7 +515,7 @@ public class ZkCoordinatorTest extends CuratorTestBase
|
|||
}
|
||||
);
|
||||
binder.bind(DruidServerMetadata.class)
|
||||
.toInstance(new DruidServerMetadata("dummyServer", "dummyHost", 0, "historical", "normal", 0));
|
||||
.toInstance(new DruidServerMetadata("dummyServer", "dummyHost", 0, ServerType.HISTORICAL, "normal", 0));
|
||||
binder.bind(DataSegmentAnnouncer.class).toInstance(announcer);
|
||||
binder.bind(DataSegmentServerAnnouncer.class).toInstance(EasyMock.createNiceMock(DataSegmentServerAnnouncer.class));
|
||||
binder.bind(CuratorFramework.class).toInstance(curator);
|
||||
|
|
|
@ -35,6 +35,7 @@ import io.druid.server.coordination.BatchDataSegmentAnnouncer;
|
|||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.SegmentChangeRequestHistory;
|
||||
import io.druid.server.coordination.SegmentChangeRequestsSnapshot;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.initialization.BatchDataSegmentAnnouncerConfig;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import io.druid.timeline.DataSegment;
|
||||
|
@ -106,7 +107,7 @@ public class BatchDataSegmentAnnouncerTest
|
|||
"id",
|
||||
"host",
|
||||
Long.MAX_VALUE,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"tier",
|
||||
0
|
||||
),
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.util.concurrent.MoreExecutors;
|
|||
import io.druid.client.ImmutableDruidDataSource;
|
||||
import io.druid.client.ImmutableDruidServer;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.easymock.EasyMock;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -61,7 +62,7 @@ public class CostBalancerStrategyTest
|
|||
serverHolderList.add(
|
||||
new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("DruidServer_Name_" + i, "localhost", 10000000L, "historical", "hot", 1),
|
||||
new DruidServerMetadata("DruidServer_Name_" + i, "localhost", 10000000L, ServerType.HISTORICAL, "hot", 1),
|
||||
3000L,
|
||||
ImmutableMap.of("DUMMY", EasyMock.createMock(ImmutableDruidDataSource.class)),
|
||||
ImmutableMap.copyOf(segments)
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.util.concurrent.MoreExecutors;
|
|||
import io.druid.client.ImmutableDruidDataSource;
|
||||
import io.druid.client.ImmutableDruidServer;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.easymock.EasyMock;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -60,7 +61,7 @@ public class DiskNormalizedCostBalancerStrategyTest
|
|||
serverHolderList.add(
|
||||
new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("DruidServer_Name_" + i, "localhost", 10000000L, "historical", "hot", 1),
|
||||
new DruidServerMetadata("DruidServer_Name_" + i, "localhost", 10000000L, ServerType.HISTORICAL, "hot", 1),
|
||||
3000L,
|
||||
ImmutableMap.of("DUMMY", EasyMock.createMock(ImmutableDruidDataSource.class)),
|
||||
ImmutableMap.copyOf(segments)
|
||||
|
|
|
@ -86,7 +86,7 @@ public class DruidClusterTest
|
|||
|
||||
private static final ServerHolder newRealtime = new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name1", "host2", 100L, ServerType.REALTIME.name(), "tier1", 0),
|
||||
new DruidServerMetadata("name1", "host2", 100L, ServerType.REALTIME, "tier1", 0),
|
||||
0L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
@ -102,7 +102,7 @@ public class DruidClusterTest
|
|||
|
||||
private static final ServerHolder newHistorical = new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name1", "host2", 100L, ServerType.HISTORICAL.name(), "tier1", 0),
|
||||
new DruidServerMetadata("name1", "host2", 100L, ServerType.HISTORICAL, "tier1", 0),
|
||||
0L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
@ -125,7 +125,7 @@ public class DruidClusterTest
|
|||
ImmutableSet.of(
|
||||
new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name1", "host1", 100L, ServerType.REALTIME.name(), "tier1", 0),
|
||||
new DruidServerMetadata("name1", "host1", 100L, ServerType.REALTIME, "tier1", 0),
|
||||
0L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
@ -145,7 +145,7 @@ public class DruidClusterTest
|
|||
ImmutableList.of(
|
||||
new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name1", "host1", 100L, ServerType.HISTORICAL.name(), "tier1", 0),
|
||||
new DruidServerMetadata("name1", "host1", 100L, ServerType.HISTORICAL, "tier1", 0),
|
||||
0L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
|
|
@ -33,6 +33,7 @@ import com.metamx.emitter.service.ServiceEventBuilder;
|
|||
import io.druid.client.DruidServer;
|
||||
import io.druid.metadata.MetadataRuleManager;
|
||||
import io.druid.segment.IndexIO;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.coordinator.helper.DruidCoordinatorRuleRunner;
|
||||
import io.druid.server.coordinator.rules.ForeverLoadRule;
|
||||
import io.druid.server.coordinator.rules.IntervalDropRule;
|
||||
|
@ -140,7 +141,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverHot",
|
||||
"hostHot",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -156,7 +157,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm",
|
||||
"hostNorm",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -172,7 +173,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverCold",
|
||||
"hostCold",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"cold",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -248,7 +249,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverHot",
|
||||
"hostHot",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -259,7 +260,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverHot2",
|
||||
"hostHot2",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -275,7 +276,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverCold",
|
||||
"hostCold",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"cold",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -342,7 +343,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm",
|
||||
"hostNorm",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
);
|
||||
|
@ -361,7 +362,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverHot",
|
||||
"hostHot",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -443,7 +444,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm",
|
||||
"hostNorm",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -503,7 +504,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm",
|
||||
"hostNorm",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -558,7 +559,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm",
|
||||
"hostNorm",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
);
|
||||
|
@ -629,7 +630,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm",
|
||||
"hostNorm",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
);
|
||||
|
@ -639,7 +640,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm2",
|
||||
"hostNorm2",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
);
|
||||
|
@ -717,7 +718,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"server1",
|
||||
"host1",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
);
|
||||
|
@ -726,7 +727,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm2",
|
||||
"hostNorm2",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
);
|
||||
|
@ -807,7 +808,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"server1",
|
||||
"host1",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
);
|
||||
|
@ -815,7 +816,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm2",
|
||||
"hostNorm2",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
);
|
||||
|
@ -888,7 +889,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"server1",
|
||||
"host1",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
);
|
||||
|
@ -897,7 +898,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm2",
|
||||
"hostNorm2",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
);
|
||||
|
@ -906,7 +907,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm3",
|
||||
"hostNorm3",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
);
|
||||
|
@ -1008,7 +1009,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverHot",
|
||||
"hostHot",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -1019,7 +1020,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverHot2",
|
||||
"hostHot2",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -1133,7 +1134,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverHot",
|
||||
"hostHot",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -1149,7 +1150,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm",
|
||||
"hostNorm",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
DruidServer.DEFAULT_TIER,
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -1223,7 +1224,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm1",
|
||||
"hostNorm1",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
);
|
||||
|
@ -1234,7 +1235,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverNorm2",
|
||||
"hostNorm2",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"normal",
|
||||
0
|
||||
);
|
||||
|
@ -1340,7 +1341,7 @@ public class DruidCoordinatorRuleRunnerTest
|
|||
"serverHot",
|
||||
"hostHot",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
DruidServer.DEFAULT_TIER,
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
|
|
@ -42,6 +42,7 @@ import io.druid.metadata.MetadataRuleManager;
|
|||
import io.druid.metadata.MetadataSegmentManager;
|
||||
import io.druid.server.DruidNode;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.coordinator.rules.ForeverLoadRule;
|
||||
import io.druid.server.coordinator.rules.Rule;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
|
@ -215,7 +216,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
|
|||
EasyMock.replay(metadataRuleManager);
|
||||
EasyMock.expect(druidServer.toImmutableDruidServer()).andReturn(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("from", null, 5L, "historical", null, 0),
|
||||
new DruidServerMetadata("from", null, 5L, ServerType.HISTORICAL, null, 0),
|
||||
1L,
|
||||
null,
|
||||
ImmutableMap.of("dummySegment", segment)
|
||||
|
@ -226,7 +227,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
|
|||
druidServer2 = EasyMock.createMock(DruidServer.class);
|
||||
EasyMock.expect(druidServer2.toImmutableDruidServer()).andReturn(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("to", null, 5L, "historical", null, 0),
|
||||
new DruidServerMetadata("to", null, 5L, ServerType.HISTORICAL, null, 0),
|
||||
1L,
|
||||
null,
|
||||
ImmutableMap.of("dummySegment2", segment)
|
||||
|
@ -284,7 +285,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
|
|||
EasyMock.replay(immutableDruidDataSource);
|
||||
|
||||
// Setup ServerInventoryView
|
||||
druidServer = new DruidServer("server1", "localhost", 5L, "historical", tier, 0);
|
||||
druidServer = new DruidServer("server1", "localhost", 5L, ServerType.HISTORICAL, tier, 0);
|
||||
loadManagementPeons.put("server1", loadQueuePeon);
|
||||
EasyMock.expect(serverInventoryView.getInventory()).andReturn(
|
||||
ImmutableList.of(druidServer)
|
||||
|
|
|
@ -85,7 +85,7 @@ public class ServerHolderTest
|
|||
// available size of 100
|
||||
final ServerHolder h1 = new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name1", "host1", 100L, ServerType.HISTORICAL.name(), "tier1", 0),
|
||||
new DruidServerMetadata("name1", "host1", 100L, ServerType.HISTORICAL, "tier1", 0),
|
||||
0L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
@ -102,7 +102,7 @@ public class ServerHolderTest
|
|||
// available size of 100
|
||||
final ServerHolder h2 = new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name1", "host1", 200L, ServerType.HISTORICAL.name(), "tier1", 0),
|
||||
new DruidServerMetadata("name1", "host1", 200L, ServerType.HISTORICAL, "tier1", 0),
|
||||
100L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
@ -119,7 +119,7 @@ public class ServerHolderTest
|
|||
// available size of 10
|
||||
final ServerHolder h3 = new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name1", "host1", 1000L, ServerType.HISTORICAL.name(), "tier1", 0),
|
||||
new DruidServerMetadata("name1", "host1", 1000L, ServerType.HISTORICAL, "tier1", 0),
|
||||
990L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
@ -136,7 +136,7 @@ public class ServerHolderTest
|
|||
// available size of 50
|
||||
final ServerHolder h4 = new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name1", "host1", 50L, ServerType.HISTORICAL.name(), "tier1", 0),
|
||||
new DruidServerMetadata("name1", "host1", 50L, ServerType.HISTORICAL, "tier1", 0),
|
||||
0L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
@ -160,7 +160,7 @@ public class ServerHolderTest
|
|||
{
|
||||
final ServerHolder h1 = new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name1", "host1", 100L, ServerType.HISTORICAL.name(), "tier1", 0),
|
||||
new DruidServerMetadata("name1", "host1", 100L, ServerType.HISTORICAL, "tier1", 0),
|
||||
0L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
@ -176,7 +176,7 @@ public class ServerHolderTest
|
|||
|
||||
final ServerHolder h2 = new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name2", "host1", 200L, ServerType.HISTORICAL.name(), "tier1", 0),
|
||||
new DruidServerMetadata("name2", "host1", 200L, ServerType.HISTORICAL, "tier1", 0),
|
||||
100L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
@ -192,7 +192,7 @@ public class ServerHolderTest
|
|||
|
||||
final ServerHolder h3 = new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name1", "host2", 200L, ServerType.HISTORICAL.name(), "tier1", 0),
|
||||
new DruidServerMetadata("name1", "host2", 200L, ServerType.HISTORICAL, "tier1", 0),
|
||||
100L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
@ -208,7 +208,7 @@ public class ServerHolderTest
|
|||
|
||||
final ServerHolder h4 = new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name1", "host1", 200L, ServerType.HISTORICAL.name(), "tier2", 0),
|
||||
new DruidServerMetadata("name1", "host1", 200L, ServerType.HISTORICAL, "tier2", 0),
|
||||
100L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
@ -224,7 +224,7 @@ public class ServerHolderTest
|
|||
|
||||
final ServerHolder h5 = new ServerHolder(
|
||||
new ImmutableDruidServer(
|
||||
new DruidServerMetadata("name1", "host1", 100L, ServerType.REALTIME.name(), "tier1", 0),
|
||||
new DruidServerMetadata("name1", "host1", 100L, ServerType.REALTIME, "tier1", 0),
|
||||
0L,
|
||||
ImmutableMap.of(
|
||||
"src1",
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.collect.Maps;
|
|||
import com.google.common.collect.MinMaxPriorityQueue;
|
||||
import com.google.common.collect.Ordering;
|
||||
import io.druid.client.DruidServer;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.coordinator.CoordinatorStats;
|
||||
import io.druid.server.coordinator.DruidCluster;
|
||||
import io.druid.server.coordinator.DruidCoordinatorRuntimeParams;
|
||||
|
@ -107,7 +108,7 @@ public class BroadcastDistributionRuleTest
|
|||
"serverHot2",
|
||||
"hostHot2",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
).addDataSegment(smallSegment.getIdentifier(), smallSegment)
|
||||
|
@ -121,7 +122,7 @@ public class BroadcastDistributionRuleTest
|
|||
"serverHot1",
|
||||
"hostHot1",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
).addDataSegment(largeSegments.get(0).getIdentifier(), largeSegments.get(0))
|
||||
|
@ -135,7 +136,7 @@ public class BroadcastDistributionRuleTest
|
|||
"serverNorm1",
|
||||
"hostNorm1",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
DruidServer.DEFAULT_TIER,
|
||||
0
|
||||
).addDataSegment(largeSegments.get(1).getIdentifier(), largeSegments.get(1))
|
||||
|
@ -149,7 +150,7 @@ public class BroadcastDistributionRuleTest
|
|||
"serverNorm2",
|
||||
"hostNorm2",
|
||||
100,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
DruidServer.DEFAULT_TIER,
|
||||
0
|
||||
).addDataSegment(largeSegments.get(2).getIdentifier(), largeSegments.get(2))
|
||||
|
@ -164,7 +165,7 @@ public class BroadcastDistributionRuleTest
|
|||
"serverHot3",
|
||||
"hostHot3",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
).addDataSegment(largeSegments2.get(0).getIdentifier(), largeSegments2.get(0))
|
||||
|
@ -178,7 +179,7 @@ public class BroadcastDistributionRuleTest
|
|||
"serverNorm3",
|
||||
"hostNorm3",
|
||||
100,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
DruidServer.DEFAULT_TIER,
|
||||
0
|
||||
).addDataSegment(largeSegments2.get(1).getIdentifier(), largeSegments2.get(1))
|
||||
|
|
|
@ -34,6 +34,7 @@ import com.metamx.emitter.core.LoggingEmitter;
|
|||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import io.druid.client.DruidServer;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.coordinator.BalancerStrategy;
|
||||
import io.druid.server.coordinator.CoordinatorStats;
|
||||
import io.druid.server.coordinator.CostBalancerStrategyFactory;
|
||||
|
@ -168,7 +169,7 @@ public class LoadRuleTest
|
|||
"serverHot",
|
||||
"hostHot",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -184,7 +185,7 @@ public class LoadRuleTest
|
|||
"serverNorm",
|
||||
"hostNorm",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
DruidServer.DEFAULT_TIER,
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -268,7 +269,7 @@ public class LoadRuleTest
|
|||
"serverHot",
|
||||
"hostHot",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
);
|
||||
|
@ -277,7 +278,7 @@ public class LoadRuleTest
|
|||
"serverNorm",
|
||||
"hostNorm",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
DruidServer.DEFAULT_TIER,
|
||||
0
|
||||
);
|
||||
|
@ -386,7 +387,7 @@ public class LoadRuleTest
|
|||
"serverHot",
|
||||
"hostHot",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
).toImmutableDruidServer(),
|
||||
|
@ -469,7 +470,7 @@ public class LoadRuleTest
|
|||
"serverHot",
|
||||
"hostHot",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
);
|
||||
|
@ -477,7 +478,7 @@ public class LoadRuleTest
|
|||
"serverHo2t",
|
||||
"hostHot2",
|
||||
1000,
|
||||
"historical",
|
||||
ServerType.HISTORICAL,
|
||||
"hot",
|
||||
0
|
||||
);
|
||||
|
|
|
@ -25,6 +25,7 @@ import io.druid.client.CoordinatorServerView;
|
|||
import io.druid.client.DruidDataSource;
|
||||
import io.druid.client.DruidServer;
|
||||
import io.druid.client.indexing.IndexingServiceClient;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.security.Access;
|
||||
import io.druid.server.security.Action;
|
||||
import io.druid.server.security.AuthConfig;
|
||||
|
@ -328,7 +329,7 @@ public class DatasourcesResourceTest
|
|||
@Test
|
||||
public void testGetSegmentDataSourceIntervals()
|
||||
{
|
||||
server = new DruidServer("who", "host", 1234, "historical", "tier1", 0);
|
||||
server = new DruidServer("who", "host", 1234, ServerType.HISTORICAL, "tier1", 0);
|
||||
server.addDataSegment(dataSegmentList.get(0).getIdentifier(), dataSegmentList.get(0));
|
||||
server.addDataSegment(dataSegmentList.get(1).getIdentifier(), dataSegmentList.get(1));
|
||||
server.addDataSegment(dataSegmentList.get(2).getIdentifier(), dataSegmentList.get(2));
|
||||
|
@ -378,7 +379,7 @@ public class DatasourcesResourceTest
|
|||
@Test
|
||||
public void testGetSegmentDataSourceSpecificInterval()
|
||||
{
|
||||
server = new DruidServer("who", "host", 1234, "historical", "tier1", 0);
|
||||
server = new DruidServer("who", "host", 1234, ServerType.HISTORICAL, "tier1", 0);
|
||||
server.addDataSegment(dataSegmentList.get(0).getIdentifier(), dataSegmentList.get(0));
|
||||
server.addDataSegment(dataSegmentList.get(1).getIdentifier(), dataSegmentList.get(1));
|
||||
server.addDataSegment(dataSegmentList.get(2).getIdentifier(), dataSegmentList.get(2));
|
||||
|
|
|
@ -22,6 +22,7 @@ package io.druid.server.http;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.client.DruidServer;
|
||||
import io.druid.client.InventoryView;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.security.AuthConfig;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.easymock.EasyMock;
|
||||
|
@ -92,7 +93,7 @@ public class IntervalsResourceTest
|
|||
5
|
||||
)
|
||||
);
|
||||
server = new DruidServer("who", "host", 1234, "historical", "tier1", 0);
|
||||
server = new DruidServer("who", "host", 1234, ServerType.HISTORICAL, "tier1", 0);
|
||||
server.addDataSegment(dataSegmentList.get(0).getIdentifier(), dataSegmentList.get(0));
|
||||
server.addDataSegment(dataSegmentList.get(1).getIdentifier(), dataSegmentList.get(1));
|
||||
server.addDataSegment(dataSegmentList.get(2).getIdentifier(), dataSegmentList.get(2));
|
||||
|
|
|
@ -24,6 +24,8 @@ import com.google.common.collect.ImmutableList;
|
|||
import io.druid.client.CoordinatorServerView;
|
||||
import io.druid.client.DruidServer;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import org.easymock.EasyMock;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -41,7 +43,7 @@ public class ServersResourceTest {
|
|||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
DruidServer dummyServer = new DruidServer("dummy", "host", 1234L, "historical", "tier", 0);
|
||||
DruidServer dummyServer = new DruidServer("dummy", "host", 1234L, ServerType.HISTORICAL, "tier", 0);
|
||||
DataSegment segment = DataSegment.builder()
|
||||
.dataSource("dataSource")
|
||||
.interval(new Interval("2016-03-22T14Z/2016-03-22T15Z"))
|
||||
|
@ -65,7 +67,7 @@ public class ServersResourceTest {
|
|||
String result = objectMapper.writeValueAsString(res.getEntity());
|
||||
String expected = "[{\"host\":\"host\","
|
||||
+ "\"maxSize\":1234,"
|
||||
+ "\"type\":\"HISTORICAL\","
|
||||
+ "\"type\":\"historical\","
|
||||
+ "\"tier\":\"tier\","
|
||||
+ "\"priority\":0,"
|
||||
+ "\"segments\":{\"dataSource_2016-03-22T14:00:00.000Z_2016-03-22T15:00:00.000Z_v0\":"
|
||||
|
@ -80,7 +82,7 @@ public class ServersResourceTest {
|
|||
{
|
||||
Response res = serversResource.getClusterServers(null, "simple");
|
||||
String result = objectMapper.writeValueAsString(res.getEntity());
|
||||
String expected = "[{\"host\":\"host\",\"tier\":\"tier\",\"type\":\"HISTORICAL\",\"priority\":0,\"currSize\":1,\"maxSize\":1234}]";
|
||||
String expected = "[{\"host\":\"host\",\"tier\":\"tier\",\"type\":\"historical\",\"priority\":0,\"currSize\":1,\"maxSize\":1234}]";
|
||||
Assert.assertEquals(expected, result);
|
||||
}
|
||||
|
||||
|
@ -91,7 +93,7 @@ public class ServersResourceTest {
|
|||
String result = objectMapper.writeValueAsString(res.getEntity());
|
||||
String expected = "{\"host\":\"host\","
|
||||
+ "\"maxSize\":1234,"
|
||||
+ "\"type\":\"HISTORICAL\","
|
||||
+ "\"type\":\"historical\","
|
||||
+ "\"tier\":\"tier\","
|
||||
+ "\"priority\":0,"
|
||||
+ "\"segments\":{\"dataSource_2016-03-22T14:00:00.000Z_2016-03-22T15:00:00.000Z_v0\":"
|
||||
|
@ -106,8 +108,29 @@ public class ServersResourceTest {
|
|||
{
|
||||
Response res = serversResource.getServer(server.getName(), "simple");
|
||||
String result = objectMapper.writeValueAsString(res.getEntity());
|
||||
String expected = "{\"host\":\"host\",\"tier\":\"tier\",\"type\":\"HISTORICAL\",\"priority\":0,\"currSize\":1,\"maxSize\":1234}";
|
||||
String expected = "{\"host\":\"host\",\"tier\":\"tier\",\"type\":\"historical\",\"priority\":0,\"currSize\":1,\"maxSize\":1234}";
|
||||
Assert.assertEquals(expected, result);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDruidServerSerde() throws Exception
|
||||
{
|
||||
DruidServer server = new DruidServer("dummy", "dummyHost", 1234, ServerType.HISTORICAL, "dummyTier", 1);
|
||||
String serverJson = objectMapper.writeValueAsString(server);
|
||||
String expected = "{\"name\":\"dummy\",\"host\":\"dummyHost\",\"maxSize\":1234,\"type\":\"historical\",\"tier\":\"dummyTier\",\"priority\":1}";
|
||||
Assert.assertEquals(expected, serverJson);
|
||||
DruidServer deserializedServer = objectMapper.readValue(serverJson, DruidServer.class);
|
||||
Assert.assertEquals(server, deserializedServer);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDruidServerMetadataSerde() throws Exception
|
||||
{
|
||||
DruidServerMetadata metadata = new DruidServerMetadata("dummy", "host", 1234, ServerType.HISTORICAL, "tier", 1);
|
||||
String metadataJson = objectMapper.writeValueAsString(metadata);
|
||||
String expected = "{\"name\":\"dummy\",\"host\":\"host\",\"maxSize\":1234,\"type\":\"historical\",\"tier\":\"tier\",\"priority\":1}";
|
||||
Assert.assertEquals(expected, metadataJson);
|
||||
DruidServerMetadata deserializedMetadata = objectMapper.readValue(metadataJson, DruidServerMetadata.class);
|
||||
Assert.assertEquals(metadata, deserializedMetadata);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,6 +37,7 @@ import io.druid.guice.NodeTypeConfig;
|
|||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.query.QuerySegmentWalker;
|
||||
import io.druid.query.lookup.LookupModule;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.http.SegmentListerResource;
|
||||
import io.druid.server.metrics.QueryCountStatsProvider;
|
||||
import io.druid.server.QueryResource;
|
||||
|
@ -82,7 +83,7 @@ public class CliHistorical extends ServerRunnable
|
|||
binder.bind(ZkCoordinator.class).in(ManageLifecycle.class);
|
||||
binder.bind(QuerySegmentWalker.class).to(ServerManager.class).in(LazySingleton.class);
|
||||
|
||||
binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig("historical"));
|
||||
binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(ServerType.HISTORICAL));
|
||||
binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class);
|
||||
binder.bind(QueryCountStatsProvider.class).to(QueryResource.class).in(LazySingleton.class);
|
||||
Jerseys.addResource(binder, QueryResource.class);
|
||||
|
|
|
@ -84,6 +84,7 @@ import io.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider;
|
|||
import io.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierConfig;
|
||||
import io.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierFactory;
|
||||
import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.http.SegmentListerResource;
|
||||
import io.druid.server.metrics.QueryCountStatsProvider;
|
||||
import io.druid.server.QueryResource;
|
||||
|
@ -207,7 +208,7 @@ public class CliPeon extends GuiceRunnable
|
|||
Jerseys.addResource(binder, QueryResource.class);
|
||||
Jerseys.addResource(binder, SegmentListerResource.class);
|
||||
LifecycleModule.register(binder, QueryResource.class);
|
||||
binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(nodeType));
|
||||
binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(ServerType.fromString(nodeType)));
|
||||
LifecycleModule.register(binder, Server.class);
|
||||
}
|
||||
|
||||
|
|
|
@ -40,6 +40,7 @@ import io.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierC
|
|||
import io.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierFactory;
|
||||
import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
|
||||
import io.druid.server.QueryResource;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.server.coordination.ZkCoordinator;
|
||||
import io.druid.server.http.SegmentListerResource;
|
||||
import io.druid.server.initialization.jetty.JettyServerInitializer;
|
||||
|
@ -103,7 +104,7 @@ public class RealtimeModule implements Module
|
|||
binder.install(new CacheModule());
|
||||
|
||||
binder.bind(QuerySegmentWalker.class).to(RealtimeManager.class).in(ManageLifecycle.class);
|
||||
binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig("realtime"));
|
||||
binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(ServerType.REALTIME));
|
||||
binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class);
|
||||
binder.bind(QueryCountStatsProvider.class).to(QueryResource.class).in(LazySingleton.class);
|
||||
Jerseys.addResource(binder, QueryResource.class);
|
||||
|
|
|
@ -27,6 +27,7 @@ import io.druid.client.selector.ServerSelector;
|
|||
import io.druid.query.DataSource;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.server.coordination.DruidServerMetadata;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.TimelineLookup;
|
||||
|
||||
|
@ -51,7 +52,7 @@ public class TestServerInventoryView implements TimelineServerView
|
|||
@Override
|
||||
public void registerSegmentCallback(Executor exec, final SegmentCallback callback)
|
||||
{
|
||||
final DruidServerMetadata dummyServer = new DruidServerMetadata("dummy", "dummy", 0, "historical", "dummy", 0);
|
||||
final DruidServerMetadata dummyServer = new DruidServerMetadata("dummy", "dummy", 0, ServerType.HISTORICAL, "dummy", 0);
|
||||
|
||||
for (final DataSegment segment : segments) {
|
||||
exec.execute(
|
||||
|
|
Loading…
Reference in New Issue