remove ServerConfig from DruidNode as all information needs to be present in DruidNode serialized form (#4858)

* remove ServerConfig from DruidNode as all information needs to be present in DruidNode serialized form

* sanitize output of /druid/coordinator/v1/cluster endpoint
This commit is contained in:
Himanshu 2017-09-28 10:40:59 -05:00 committed by Parag Jain
parent 2c30d5ba55
commit f69c9280c4
44 changed files with 410 additions and 287 deletions

View File

@ -16,7 +16,7 @@ The broker node uses several of the global configs in [Configuration](../configu
|--------|-----------|-------|
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8082|
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.server.http.tls](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8282|
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8282|
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/broker|
### Query Configs

View File

@ -16,7 +16,7 @@ The coordinator node uses several of the global configs in [Configuration](../co
|--------|-----------|-------|
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8081|
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.server.http.tls](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8281|
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8281|
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/coordinator|
### Coordinator Operation

View File

@ -16,7 +16,7 @@ The historical node uses several of the global configs in [Configuration](../con
|--------|-----------|-------|
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8083|
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.server.http.tls](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8283|
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8283|
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/historical|
### General Configuration

View File

@ -15,7 +15,7 @@ The indexing service uses several of the global configs in [Configuration](../co
|--------|-----------|-------|
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8090|
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.server.http.tls](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8290|
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8290|
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/overlord|
#### MiddleManager Node Configs
@ -24,7 +24,7 @@ The indexing service uses several of the global configs in [Configuration](../co
|--------|-----------|-------|
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8091|
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.server.http.tls](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8291|
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8291|
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/middlemanager|
#### Task Logging

View File

@ -17,7 +17,7 @@ The realtime node uses several of the global configs in [Configuration](../confi
|--------|-----------|-------|
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8084|
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.server.http.tls](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8284|
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8284|
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/realtime|
### Realtime Operation

View File

@ -9,8 +9,8 @@ TLS Support
|Property|Description|Default|
|--------|-----------|-------|
|`druid.server.http.plaintext`|Enable/Disable HTTP connector.|`true`|
|`druid.server.http.tls`|Enable/Disable HTTPS connector.|`false`|
|`druid.enablePlaintextPort`|Enable/Disable HTTP connector.|`true`|
|`druid.enableTlsPort`|Enable/Disable HTTPS connector.|`false`|
Although not recommended but both HTTP and HTTPS connectors can be enabled at a time and respective ports are configurable using `druid.plaintextPort`
and `druid.tlsPort` properties on each node. Please see `Configuration` section of individual nodes to check the valid and default values for these ports.
@ -44,4 +44,4 @@ while creating the HttpClient. This binding can be achieved writing a [Druid ext
which can provide an instance of SSLContext. Druid comes with a simple extension present [here](../development/extensions-core/simple-client-sslcontext.html)
which should be useful enough for most simple cases, see [this](./including-extensions.html) for how to include extensions.
If this extension does not satisfy the requirements then please follow the extension [implementation](https://github.com/druid-io/druid/tree/master/extensions-core/simple-client-sslcontext)
to create your own extension.
to create your own extension.

View File

@ -36,12 +36,11 @@ import io.druid.guice.annotations.Json;
import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.query.lookup.namespace.ExtractionNamespace;
import io.druid.query.lookup.namespace.UriExtractionNamespace;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.lookup.namespace.cache.CacheScheduler;
import io.druid.server.lookup.namespace.cache.NamespaceExtractionCacheManager;
import org.easymock.EasyMock;
@ -525,7 +524,7 @@ public class NamespaceLookupExtractorFactoryTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, true, false)
);
}
}

View File

@ -29,7 +29,6 @@ import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.lookup.namespace.NamespaceExtractionModule;
import org.junit.Assert;
import org.junit.Test;
@ -50,7 +49,7 @@ public class OffHeapNamespaceExtractionCacheManagerTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, true, false)
);
}
}

View File

@ -29,7 +29,6 @@ import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.lookup.namespace.NamespaceExtractionModule;
import org.junit.Assert;
import org.junit.Test;
@ -50,7 +49,7 @@ public class OnHeapNamespaceExtractionCacheManagerTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, true, false)
);
}
}

View File

@ -44,12 +44,12 @@ import io.druid.guice.annotations.Self;
import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.path.PathSpec;
import io.druid.initialization.Initialization;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.JodaUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.java.util.common.guava.FunctionalIterable;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMergerV9;
@ -57,7 +57,6 @@ import io.druid.segment.IndexSpec;
import io.druid.segment.indexing.granularity.GranularitySpec;
import io.druid.segment.loading.DataSegmentPusher;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.ShardSpec;
import io.druid.timeline.partition.ShardSpecLookup;
@ -111,7 +110,7 @@ public class HadoopDruidIndexerConfig
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("hadoop-indexer", null, null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("hadoop-indexer", null, null, null, true, false)
);
JsonConfigProvider.bind(binder, "druid.hadoop.security.kerberos", HadoopKerberosConfig.class);
}

View File

@ -39,7 +39,6 @@ import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec;
import io.druid.segment.loading.DataSegmentPusher;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.timeline.DataSegment;
import org.joda.time.Interval;
@ -66,7 +65,7 @@ public class HadoopDruidConverterConfig
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("hadoop-converter", null, null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("hadoop-converter", null, null, null, true, false)
);
}
}

View File

@ -51,7 +51,6 @@ import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.apache.hadoop.conf.Configuration;
@ -136,7 +135,7 @@ public class DatasourcePathSpecTest
{
binder.bind(UsedSegmentLister.class).toInstance(segmentList);
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("dummy-node", null, null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("dummy-node", null, null, null, true, false)
);
}
}

View File

@ -62,7 +62,6 @@ import io.druid.java.util.common.lifecycle.LifecycleStop;
import io.druid.java.util.common.logger.Logger;
import io.druid.query.DruidMetrics;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.metrics.MonitorsConfig;
import io.druid.tasklogs.TaskLogPusher;
import io.druid.tasklogs.TaskLogStreamer;
@ -105,7 +104,6 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
private final ObjectMapper jsonMapper;
private final PortFinder portFinder;
private final PortFinder tlsPortFinder;
private final ServerConfig serverConfig;
private final CopyOnWriteArrayList<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
// Writes must be synchronized. This is only a ConcurrentMap so "informational" reads can occur without waiting.
@ -121,8 +119,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
Properties props,
TaskLogPusher taskLogPusher,
ObjectMapper jsonMapper,
@Self DruidNode node,
ServerConfig serverConfig
@Self DruidNode node
)
{
this.config = config;
@ -133,7 +130,6 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
this.node = node;
this.portFinder = new PortFinder(config.getStartPort());
this.tlsPortFinder = new PortFinder(config.getTlsStartPort());
this.serverConfig = serverConfig;
this.exec = MoreExecutors.listeningDecorator(
Execs.multiThreaded(workerConfig.getCapacity(), "forking-task-runner-%d")
);
@ -239,7 +235,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
int tlsChildPort = -1;
int childChatHandlerPort = -1;
if (serverConfig.isPlaintext()) {
if (node.isEnablePlaintextPort()) {
if (config.isSeparateIngestionEndpoint()) {
Pair<Integer, Integer> portPair = portFinder.findTwoConsecutiveUnusedPorts();
childPort = portPair.lhs;
@ -249,7 +245,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
}
}
if (serverConfig.isTls()) {
if (node.isEnableTlsPort()) {
tlsChildPort = tlsPortFinder.findUnusedPort();
}
@ -501,10 +497,10 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
}
}
if (serverConfig.isPlaintext()) {
if (node.isEnablePlaintextPort()) {
portFinder.markPortUnused(childPort);
}
if (serverConfig.isTls()) {
if (node.isEnableTlsPort()) {
tlsPortFinder.markPortUnused(tlsChildPort);
}
if (childChatHandlerPort > 0) {

View File

@ -26,7 +26,6 @@ import io.druid.indexing.common.config.TaskConfig;
import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig;
import io.druid.indexing.worker.config.WorkerConfig;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.tasklogs.TaskLogPusher;
import java.util.Properties;
@ -42,7 +41,6 @@ public class ForkingTaskRunnerFactory implements TaskRunnerFactory<ForkingTaskRu
private final ObjectMapper jsonMapper;
private final TaskLogPusher persistentTaskLogs;
private final DruidNode node;
private final ServerConfig serverConfig;
@Inject
public ForkingTaskRunnerFactory(
@ -52,8 +50,7 @@ public class ForkingTaskRunnerFactory implements TaskRunnerFactory<ForkingTaskRu
final Properties props,
final ObjectMapper jsonMapper,
final TaskLogPusher persistentTaskLogs,
@Self DruidNode node,
ServerConfig serverConfig
@Self DruidNode node
)
{
this.config = config;
@ -63,12 +60,11 @@ public class ForkingTaskRunnerFactory implements TaskRunnerFactory<ForkingTaskRu
this.jsonMapper = jsonMapper;
this.persistentTaskLogs = persistentTaskLogs;
this.node = node;
this.serverConfig = serverConfig;
}
@Override
public ForkingTaskRunner build()
{
return new ForkingTaskRunner(config, taskConfig, workerConfig, props, persistentTaskLogs, jsonMapper, node, serverConfig);
return new ForkingTaskRunner(config, taskConfig, workerConfig, props, persistentTaskLogs, jsonMapper, node);
}
}

View File

@ -112,7 +112,6 @@ import io.druid.server.DruidNode;
import io.druid.server.coordination.DataSegmentAnnouncer;
import io.druid.server.coordination.DataSegmentServerAnnouncer;
import io.druid.server.coordination.ServerType;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.metrics.NoopServiceEmitter;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
@ -620,7 +619,7 @@ public class TaskLifecycleTest
tb,
taskConfig,
emitter,
new DruidNode("dummy", "dummy", 10000, null, new ServerConfig())
new DruidNode("dummy", "dummy", 10000, null, true, false)
);
}

View File

@ -56,7 +56,6 @@ import io.druid.java.util.common.guava.CloseQuietly;
import io.druid.server.DruidNode;
import io.druid.server.coordinator.CoordinatorOverlordServiceConfig;
import io.druid.server.initialization.IndexerZkConfig;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.initialization.ZkPathsConfig;
import io.druid.server.metrics.NoopServiceEmitter;
import io.druid.server.security.AuthConfig;
@ -167,7 +166,7 @@ public class OverlordTest
setupServerAndCurator();
curator.start();
curator.blockUntilConnected();
druidNode = new DruidNode("hey", "what", 1234, null, new ServerConfig());
druidNode = new DruidNode("hey", "what", 1234, null, true, false);
ServiceEmitter serviceEmitter = new NoopServiceEmitter();
taskMaster = new TaskMaster(
new TaskQueueConfig(null, new Period(1), null, new Period(10)),

View File

@ -47,7 +47,6 @@ import io.druid.segment.loading.StorageLocationConfig;
import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
import io.druid.server.DruidNode;
import io.druid.server.initialization.IndexerZkConfig;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.initialization.ZkPathsConfig;
import io.druid.server.metrics.NoopServiceEmitter;
import org.apache.curator.framework.CuratorFramework;
@ -71,7 +70,7 @@ public class WorkerTaskMonitorTest
private static final String basePath = "/test/druid";
private static final String tasksPath = StringUtils.format("%s/indexer/tasks/worker", basePath);
private static final String statusPath = StringUtils.format("%s/indexer/status/worker", basePath);
private static final DruidNode DUMMY_NODE = new DruidNode("dummy", "dummy", 9000, null, new ServerConfig());
private static final DruidNode DUMMY_NODE = new DruidNode("dummy", "dummy", 9000, null, true, false);
private TestingCluster testingCluster;
private CuratorFramework cf;

View File

@ -37,7 +37,6 @@ import io.druid.guice.ManageLifecycle;
import io.druid.guice.annotations.EscalatedClient;
import io.druid.guice.annotations.Self;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.testing.IntegrationTestingConfig;
import io.druid.testing.IntegrationTestingConfigProvider;
import io.druid.testing.IntegrationTestingCuratorConfig;
@ -58,7 +57,7 @@ public class DruidTestModule implements Module
// Bind DruidNode instance to make Guice happy. This instance is currently unused.
binder.bind(DruidNode.class).annotatedWith(Self.class).toInstance(
new DruidNode("integration-tests", "localhost", 9191, null, null, new ServerConfig())
new DruidNode("integration-tests", "localhost", 9191, null, null, true, false)
);
}

View File

@ -123,6 +123,6 @@ public class ServiceAnnouncingChatHandlerProvider implements ChatHandlerProvider
private DruidNode makeDruidNode(String key)
{
return new DruidNode(key, node.getHost(), node.getPlaintextPort(), node.getTlsPort(), serverConfig);
return new DruidNode(key, node.getHost(), node.getPlaintextPort(), node.getTlsPort(), node.isEnablePlaintextPort(), node.isEnableTlsPort());
}
}

View File

@ -28,12 +28,12 @@ import com.google.inject.name.Named;
import io.druid.common.utils.SocketUtil;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE;
import io.druid.server.initialization.ServerConfig;
import javax.validation.constraints.Max;
import javax.validation.constraints.NotNull;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.Objects;
/**
*/
@ -60,18 +60,26 @@ public class DruidNode
@Max(0xffff)
private int plaintextPort = -1;
@JsonProperty
private boolean enablePlaintextPort = true;
@JsonProperty
@Max(0xffff)
private int tlsPort = -1;
@JacksonInject
@NotNull
private ServerConfig serverConfig;
@JsonProperty
private boolean enableTlsPort = false;
public DruidNode(String serviceName, String host, Integer plaintextPort, Integer tlsPort, ServerConfig serverConfig)
public DruidNode(
String serviceName,
String host,
Integer plaintextPort,
Integer tlsPort,
boolean enablePlaintextPort,
boolean enableTlsPort
)
{
this(serviceName, host, plaintextPort, null, tlsPort, serverConfig);
this.serverConfig = serverConfig;
this(serviceName, host, plaintextPort, null, tlsPort, enablePlaintextPort, enableTlsPort);
}
/**
@ -89,6 +97,7 @@ public class DruidNode
* host = "[2001:db8:85a3::8a2e:370:7334]", port = 123 -> host = 2001:db8:85a3::8a2e:370:7334, port = 123
* host = "2001:db8:85a3::8a2e:370:7334", port = 123 -> host = 2001:db8:85a3::8a2e:370:7334, port = 123
* host = null , port = 123 -> host = _default_, port = 123
*
*/
@JsonCreator
public DruidNode(
@ -97,20 +106,31 @@ public class DruidNode
@JsonProperty("plaintextPort") Integer plaintextPort,
@JacksonInject @Named("servicePort") @JsonProperty("port") Integer port,
@JacksonInject @Named("tlsServicePort") @JsonProperty("tlsPort") Integer tlsPort,
@JacksonInject ServerConfig serverConfig
@JsonProperty("enablePlaintextPort") Boolean enablePlaintextPort,
@JsonProperty("enableTlsPort") boolean enableTlsPort
)
{
init(serviceName, host, plaintextPort != null ? plaintextPort : port, tlsPort, serverConfig);
init(
serviceName,
host,
plaintextPort != null ? plaintextPort : port,
tlsPort,
enablePlaintextPort == null ? true : enablePlaintextPort.booleanValue(),
enableTlsPort
);
}
private void init(String serviceName, String host, Integer plainTextPort, Integer tlsPort, ServerConfig serverConfig)
private void init(String serviceName, String host, Integer plainTextPort, Integer tlsPort, boolean enablePlaintextPort, boolean enableTlsPort)
{
Preconditions.checkNotNull(serviceName);
if (!serverConfig.isTls() && !serverConfig.isPlaintext()) {
throw new IAE("At least one of the druid.server.http.plainText or druid.server.http.tls needs to be enabled");
if (!enableTlsPort && !enablePlaintextPort) {
throw new IAE("At least one of the druid.enablePlaintextPort or druid.enableTlsPort needs to be true.");
}
this.enablePlaintextPort = enablePlaintextPort;
this.enableTlsPort = enableTlsPort;
final boolean nullHost = host == null;
HostAndPort hostAndPort;
Integer portFromHostConfig;
@ -128,16 +148,16 @@ public class DruidNode
host = getDefaultHost();
}
if (serverConfig.isPlaintext() && serverConfig.isTls() && ((plainTextPort == null || tlsPort == null)
if (enablePlaintextPort && enableTlsPort && ((plainTextPort == null || tlsPort == null)
|| plainTextPort.equals(tlsPort))) {
// If both plainTExt and tls are enabled then do not allow plaintextPort to be null or
throw new IAE("plaintextPort and tlsPort cannot be null or same if both http and https connectors are enabled");
}
if (serverConfig.isTls() && (tlsPort == null || tlsPort < 0)) {
throw new IAE("A valid tlsPort needs to specified when druid.server.http.tls is set");
if (enableTlsPort && (tlsPort == null || tlsPort < 0)) {
throw new IAE("A valid tlsPort needs to specified when druid.enableTlsPort is set");
}
if (serverConfig.isPlaintext()) {
if (enablePlaintextPort) {
// to preserve backwards compatible behaviour
if (nullHost && plainTextPort == null) {
plainTextPort = -1;
@ -150,7 +170,7 @@ public class DruidNode
} else {
this.plaintextPort = -1;
}
if (serverConfig.isTls()) {
if (enableTlsPort) {
this.tlsPort = tlsPort;
} else {
this.tlsPort = -1;
@ -175,6 +195,16 @@ public class DruidNode
return plaintextPort;
}
public boolean isEnablePlaintextPort()
{
return enablePlaintextPort;
}
public boolean isEnableTlsPort()
{
return enableTlsPort;
}
public int getTlsPort()
{
return tlsPort;
@ -182,7 +212,7 @@ public class DruidNode
public DruidNode withService(String service)
{
return new DruidNode(service, host, plaintextPort, tlsPort, serverConfig);
return new DruidNode(service, host, plaintextPort, tlsPort, enablePlaintextPort, enableTlsPort);
}
public String getServiceScheme()
@ -195,7 +225,7 @@ public class DruidNode
*/
public String getHostAndPort()
{
if (serverConfig.isPlaintext()) {
if (enablePlaintextPort) {
if (plaintextPort < 0) {
return HostAndPort.fromString(host).toString();
} else {
@ -207,7 +237,7 @@ public class DruidNode
public String getHostAndTlsPort()
{
if (serverConfig.isTls()) {
if (enableTlsPort) {
return HostAndPort.fromParts(host, tlsPort).toString();
}
return null;
@ -215,7 +245,7 @@ public class DruidNode
public int getPortToUse()
{
if (serverConfig.isTls()) {
if (enableTlsPort) {
return getTlsPort();
} else {
return getPlaintextPort();
@ -246,33 +276,20 @@ public class DruidNode
if (o == null || getClass() != o.getClass()) {
return false;
}
DruidNode druidNode = (DruidNode) o;
if (plaintextPort != druidNode.plaintextPort) {
return false;
}
if (tlsPort != druidNode.tlsPort) {
return false;
}
if (serviceName != null ? !serviceName.equals(druidNode.serviceName) : druidNode.serviceName != null) {
return false;
}
if (host != null ? !host.equals(druidNode.host) : druidNode.host != null) {
return false;
}
return serverConfig != null ? serverConfig.equals(druidNode.serverConfig) : druidNode.serverConfig == null;
return port == druidNode.port &&
plaintextPort == druidNode.plaintextPort &&
enablePlaintextPort == druidNode.enablePlaintextPort &&
tlsPort == druidNode.tlsPort &&
enableTlsPort == druidNode.enableTlsPort &&
Objects.equals(serviceName, druidNode.serviceName) &&
Objects.equals(host, druidNode.host);
}
@Override
public int hashCode()
{
int result = serviceName != null ? serviceName.hashCode() : 0;
result = 31 * result + (host != null ? host.hashCode() : 0);
result = 31 * result + plaintextPort;
result = 31 * result + tlsPort;
result = 31 * result + (serverConfig != null ? serverConfig.hashCode() : 0);
return result;
return Objects.hash(serviceName, host, port, plaintextPort, enablePlaintextPort, tlsPort, enableTlsPort);
}
@Override
@ -281,9 +298,11 @@ public class DruidNode
return "DruidNode{" +
"serviceName='" + serviceName + '\'' +
", host='" + host + '\'' +
", port=" + port +
", plaintextPort=" + plaintextPort +
", enablePlaintextPort=" + enablePlaintextPort +
", tlsPort=" + tlsPort +
", serverConfig=" + serverConfig +
", enableTlsPort=" + enableTlsPort +
'}';
}
}

View File

@ -19,6 +19,10 @@
package io.druid.server.http;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.Collections2;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
import com.sun.jersey.spi.container.ResourceFilters;
@ -26,12 +30,14 @@ import io.druid.discovery.DiscoveryDruidNode;
import io.druid.discovery.DruidNodeDiscoveryProvider;
import io.druid.guice.LazySingleton;
import io.druid.java.util.common.StringUtils;
import io.druid.server.DruidNode;
import io.druid.server.http.security.StateResourceFilter;
import javax.ws.rs.GET;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.QueryParam;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.util.Collection;
@ -53,35 +59,31 @@ public class ClusterResource
@GET
@Produces(MediaType.APPLICATION_JSON)
public Response getClusterServers()
public Response getClusterServers(
@QueryParam("full") boolean full
)
{
ImmutableMap.Builder<String, Object> entityBuilder = new ImmutableMap.Builder<>();
entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR,
druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR)
.getAllNodes()
getNodes(DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR, full)
);
entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD,
druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD)
.getAllNodes()
getNodes(DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD, full)
);
entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_BROKER,
druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_BROKER)
.getAllNodes()
getNodes(DruidNodeDiscoveryProvider.NODE_TYPE_BROKER, full)
);
entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL,
druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL)
.getAllNodes()
getNodes(DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL, full)
);
Collection<DiscoveryDruidNode> mmNodes = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_MM)
.getAllNodes();
Collection<Object> mmNodes = getNodes(DruidNodeDiscoveryProvider.NODE_TYPE_MM, full);
if (!mmNodes.isEmpty()) {
entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_MM, mmNodes);
}
Collection<DiscoveryDruidNode> routerNodes = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_ROUTER)
.getAllNodes();
Collection<Object> routerNodes = getNodes(DruidNodeDiscoveryProvider.NODE_TYPE_ROUTER, full);
if (!routerNodes.isEmpty()) {
entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_ROUTER, routerNodes);
}
@ -93,7 +95,8 @@ public class ClusterResource
@Produces({MediaType.APPLICATION_JSON})
@Path("/{nodeType}")
public Response getClusterServers(
@PathParam("nodeType") String nodeType
@PathParam("nodeType") String nodeType,
@QueryParam("full") boolean full
)
{
if (nodeType == null || !DruidNodeDiscoveryProvider.ALL_NODE_TYPES.contains(nodeType)) {
@ -107,8 +110,74 @@ public class ClusterResource
.build();
} else {
return Response.status(Response.Status.OK).entity(
druidNodeDiscoveryProvider.getForNodeType(nodeType).getAllNodes()
getNodes(nodeType, full)
).build();
}
}
private Collection<Object> getNodes(String nodeType, boolean full)
{
Collection<DiscoveryDruidNode> discoveryDruidNodes = druidNodeDiscoveryProvider.getForNodeType(nodeType)
.getAllNodes();
if (full) {
return (Collection) discoveryDruidNodes;
} else {
return Collections2.transform(
discoveryDruidNodes,
(discoveryDruidNode) -> Node.from(discoveryDruidNode.getDruidNode())
);
}
}
@JsonInclude(JsonInclude.Include.NON_NULL)
private static class Node
{
private final String host;
private final String service;
private final Integer plaintextPort;
private final Integer tlsPort;
@JsonCreator
public Node(String host, String service, Integer plaintextPort, Integer tlsPort)
{
this.host = host;
this.service = service;
this.plaintextPort = plaintextPort;
this.tlsPort = tlsPort;
}
@JsonProperty
public String getHost()
{
return host;
}
@JsonProperty
public String getService()
{
return service;
}
@JsonProperty
public Integer getPlaintextPort()
{
return plaintextPort;
}
@JsonProperty
public Integer getTlsPort()
{
return tlsPort;
}
public static Node from(DruidNode druidNode)
{
return new Node(
druidNode.getHost(),
druidNode.getServiceName(),
druidNode.getPlaintextPort() > 0 ? druidNode.getPlaintextPort() : null,
druidNode.getTlsPort() > 0 ? druidNode.getTlsPort() : null
);
}
}
}

View File

@ -46,12 +46,6 @@ public class ServerConfig
@Min(1)
private long maxScatterGatherBytes = Long.MAX_VALUE;
@JsonProperty
private boolean plaintext = true;
@JsonProperty
private boolean tls = false;
public int getNumThreads()
{
return numThreads;
@ -72,16 +66,6 @@ public class ServerConfig
return maxScatterGatherBytes;
}
public boolean isPlaintext()
{
return plaintext;
}
public boolean isTls()
{
return tls;
}
@Override
public boolean equals(Object o)
{
@ -95,15 +79,13 @@ public class ServerConfig
return numThreads == that.numThreads &&
defaultQueryTimeout == that.defaultQueryTimeout &&
maxScatterGatherBytes == that.maxScatterGatherBytes &&
plaintext == that.plaintext &&
tls == that.tls &&
Objects.equals(maxIdleTime, that.maxIdleTime);
}
@Override
public int hashCode()
{
return Objects.hash(numThreads, maxIdleTime, defaultQueryTimeout, maxScatterGatherBytes, plaintext, tls);
return Objects.hash(numThreads, maxIdleTime, defaultQueryTimeout, maxScatterGatherBytes);
}
@Override
@ -114,8 +96,6 @@ public class ServerConfig
", maxIdleTime=" + maxIdleTime +
", defaultQueryTimeout=" + defaultQueryTimeout +
", maxScatterGatherBytes=" + maxScatterGatherBytes +
", plaintext=" + plaintext +
", tls=" + tls +
'}';
}
}

View File

@ -196,13 +196,13 @@ public class JettyServerModule extends JerseyServletModule
final List<ServerConnector> serverConnectors = new ArrayList<>();
if (config.isPlaintext()) {
if (node.isEnablePlaintextPort()) {
log.info("Creating http connector with port [%d]", node.getPlaintextPort());
final ServerConnector connector = new ServerConnector(server);
connector.setPort(node.getPlaintextPort());
serverConnectors.add(connector);
}
if (config.isTls()) {
if (node.isEnableTlsPort()) {
log.info("Creating https connector with port [%d]", node.getTlsPort());
final SslContextFactory sslContextFactory;
if (sslContextFactoryBinding == null) {

View File

@ -43,7 +43,6 @@ import io.druid.server.coordination.SegmentChangeRequestHistory;
import io.druid.server.coordination.SegmentChangeRequestLoad;
import io.druid.server.coordination.SegmentChangeRequestsSnapshot;
import io.druid.server.coordination.ServerType;
import io.druid.server.initialization.ServerConfig;
import io.druid.timeline.DataSegment;
import org.easymock.EasyMock;
import org.jboss.netty.buffer.ChannelBuffers;
@ -162,7 +161,7 @@ public class HttpServerInventoryViewTest
);
DiscoveryDruidNode druidNode = new DiscoveryDruidNode(
new DruidNode("service", "host", 8080, null, new ServerConfig()),
new DruidNode("service", "host", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL,
ImmutableMap.of(
DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0)

View File

@ -29,7 +29,6 @@ import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import org.junit.Assert;
import org.junit.Test;
@ -44,7 +43,7 @@ public class CacheMonitorTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, true, false)
);
}
}
@ -63,7 +62,7 @@ public class CacheMonitorTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, true, false)
);
binder.bind(Cache.class).toInstance(MapCache.create(0));
}

View File

@ -25,7 +25,6 @@ import io.druid.curator.CuratorTestBase;
import io.druid.discovery.DruidLeaderSelector;
import io.druid.java.util.common.logger.Logger;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import org.easymock.EasyMock;
import org.junit.After;
import org.junit.Assert;
@ -60,7 +59,7 @@ public class CuratorDruidLeaderSelectorTest extends CuratorTestBase
CuratorDruidLeaderSelector leaderSelector1 = new CuratorDruidLeaderSelector(
curator,
new DruidNode("s1", "h1", 8080, null, new ServerConfig()),
new DruidNode("s1", "h1", 8080, null, true, false),
latchPath
);
leaderSelector1.registerListener(
@ -92,7 +91,7 @@ public class CuratorDruidLeaderSelectorTest extends CuratorTestBase
CuratorDruidLeaderSelector leaderSelector2 = new CuratorDruidLeaderSelector(
curator,
new DruidNode("s2", "h2", 8080, null, new ServerConfig()),
new DruidNode("s2", "h2", 8080, null, true, false),
latchPath
);
leaderSelector2.registerListener(
@ -132,7 +131,7 @@ public class CuratorDruidLeaderSelectorTest extends CuratorTestBase
CuratorDruidLeaderSelector leaderSelector3 = new CuratorDruidLeaderSelector(
curator,
new DruidNode("s3", "h3", 8080, null, new ServerConfig()),
new DruidNode("s3", "h3", 8080, null, true, false),
latchPath
);
leaderSelector3.registerListener(

View File

@ -80,25 +80,25 @@ public class CuratorDruidNodeAnnouncerAndDiscoveryTest extends CuratorTestBase
);
DiscoveryDruidNode node1 = new DiscoveryDruidNode(
new DruidNode("s1", "h1", 8080, null, new ServerConfig()),
new DruidNode("s1", "h1", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR,
ImmutableMap.of()
);
DiscoveryDruidNode node2 = new DiscoveryDruidNode(
new DruidNode("s2", "h2", 8080, null, new ServerConfig()),
new DruidNode("s2", "h2", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR,
ImmutableMap.of()
);
DiscoveryDruidNode node3 = new DiscoveryDruidNode(
new DruidNode("s3", "h3", 8080, null, new ServerConfig()),
new DruidNode("s3", "h3", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD,
ImmutableMap.of()
);
DiscoveryDruidNode node4 = new DiscoveryDruidNode(
new DruidNode("s4", "h4", 8080, null, new ServerConfig()),
new DruidNode("s4", "h4", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD,
ImmutableMap.of()
);

View File

@ -42,7 +42,6 @@ import io.druid.initialization.Initialization;
import io.druid.java.util.common.StringUtils;
import io.druid.server.DruidNode;
import io.druid.server.initialization.BaseJettyTest;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.initialization.jetty.JettyServerInitializer;
import org.easymock.EasyMock;
import org.eclipse.jetty.server.Handler;
@ -73,7 +72,7 @@ public class DruidLeaderClientTest extends BaseJettyTest
@Override
protected Injector setupInjector()
{
final DruidNode node = new DruidNode("test", "localhost", null, null, new ServerConfig());
final DruidNode node = new DruidNode("test", "localhost", null, null, true, false);
discoveryDruidNode = new DiscoveryDruidNode(node, "test", ImmutableMap.of());
Injector injector = Initialization.makeInjectorWithModules(
@ -163,7 +162,7 @@ public class DruidLeaderClientTest extends BaseJettyTest
DruidNodeDiscovery druidNodeDiscovery = EasyMock.createMock(DruidNodeDiscovery.class);
EasyMock.expect(druidNodeDiscovery.getAllNodes()).andReturn(
ImmutableList.of(new DiscoveryDruidNode(
new DruidNode("test", "dummyhost", 64231, null, new ServerConfig()),
new DruidNode("test", "dummyhost", 64231, null, true, false),
"test",
ImmutableMap.of()
))

View File

@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import io.druid.server.DruidNode;
import io.druid.server.coordination.ServerType;
import io.druid.server.initialization.ServerConfig;
import org.junit.Assert;
import org.junit.Test;
@ -87,7 +86,7 @@ public class DruidNodeDiscoveryProviderTest
Assert.assertTrue(lookupNodeDiscovery.getAllNodes().isEmpty());
DiscoveryDruidNode node1 = new DiscoveryDruidNode(
new DruidNode("s1", "h1", 8080, null, new ServerConfig()),
new DruidNode("s1", "h1", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL,
ImmutableMap.of(
DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0),
@ -95,21 +94,21 @@ public class DruidNodeDiscoveryProviderTest
);
DiscoveryDruidNode node2 = new DiscoveryDruidNode(
new DruidNode("s2", "h2", 8080, null, new ServerConfig()),
new DruidNode("s2", "h2", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL,
ImmutableMap.of(
DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0))
);
DiscoveryDruidNode node3 = new DiscoveryDruidNode(
new DruidNode("s3", "h3", 8080, null, new ServerConfig()),
new DruidNode("s3", "h3", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL,
ImmutableMap.of(
LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier"))
);
DiscoveryDruidNode node4 = new DiscoveryDruidNode(
new DruidNode("s4", "h4", 8080, null, new ServerConfig()),
new DruidNode("s4", "h4", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_PEON,
ImmutableMap.of(
DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0),
@ -117,35 +116,35 @@ public class DruidNodeDiscoveryProviderTest
);
DiscoveryDruidNode node5 = new DiscoveryDruidNode(
new DruidNode("s5", "h5", 8080, null, new ServerConfig()),
new DruidNode("s5", "h5", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_PEON,
ImmutableMap.of(
DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0))
);
DiscoveryDruidNode node6 = new DiscoveryDruidNode(
new DruidNode("s6", "h6", 8080, null, new ServerConfig()),
new DruidNode("s6", "h6", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_PEON,
ImmutableMap.of(
LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier"))
);
DiscoveryDruidNode node7 = new DiscoveryDruidNode(
new DruidNode("s7", "h7", 8080, null, new ServerConfig()),
new DruidNode("s7", "h7", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_BROKER,
ImmutableMap.of(
LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier"))
);
DiscoveryDruidNode node7Clone = new DiscoveryDruidNode(
new DruidNode("s7", "h7", 8080, null, new ServerConfig()),
new DruidNode("s7", "h7", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_BROKER,
ImmutableMap.of(
LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier"))
);
DiscoveryDruidNode node8 = new DiscoveryDruidNode(
new DruidNode("s8", "h8", 8080, null, new ServerConfig()),
new DruidNode("s8", "h8", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR,
ImmutableMap.of()
);

View File

@ -34,7 +34,6 @@ import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self;
import io.druid.java.util.common.ISE;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import org.junit.Assert;
import org.junit.FixMethodOrder;
import org.junit.Rule;
@ -142,7 +141,7 @@ public class InitializationTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, true, false)
);
}
}

View File

@ -31,7 +31,6 @@ import io.druid.guice.JsonConfigurator;
import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.metrics.DataSourceTaskIdHolder;
import org.junit.Assert;
import org.junit.Before;
@ -51,7 +50,7 @@ public class LookupListeningAnnouncerConfigTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, true, false)
);
binder.bind(Key.get(
String.class,

View File

@ -91,6 +91,8 @@ public class ServiceAnnouncingChatHandlerProviderTest extends EasyMockSupport
EasyMock.expect(node.getHost()).andReturn(TEST_HOST);
EasyMock.expect(node.getPlaintextPort()).andReturn(TEST_PORT);
EasyMock.expect(node.isEnablePlaintextPort()).andReturn(true);
EasyMock.expect(node.isEnableTlsPort()).andReturn(false);
EasyMock.expect(node.getTlsPort()).andReturn(-1);
serviceAnnouncer.announce(EasyMock.capture(captured));
replayAll();
@ -117,7 +119,9 @@ public class ServiceAnnouncingChatHandlerProviderTest extends EasyMockSupport
resetAll();
EasyMock.expect(node.getHost()).andReturn(TEST_HOST);
EasyMock.expect(node.getPlaintextPort()).andReturn(TEST_PORT);
EasyMock.expect(node.isEnablePlaintextPort()).andReturn(true);
EasyMock.expect(node.getTlsPort()).andReturn(-1);
EasyMock.expect(node.isEnableTlsPort()).andReturn(false);
serviceAnnouncer.unannounce(EasyMock.capture(captured));
replayAll();

View File

@ -45,7 +45,6 @@ import io.druid.query.MapQueryToolChestWarehouse;
import io.druid.query.Query;
import io.druid.query.QueryToolChest;
import io.druid.server.initialization.BaseJettyTest;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.initialization.jetty.JettyServerInitUtils;
import io.druid.server.initialization.jetty.JettyServerInitializer;
import io.druid.server.log.RequestLogger;
@ -113,7 +112,7 @@ public class AsyncQueryForwardingServletTest extends BaseJettyTest
JsonConfigProvider.bindInstance(
binder,
Key.get(DruidNode.class, Self.class),
new DruidNode("test", "localhost", null, null, new ServerConfig())
new DruidNode("test", "localhost", null, null, true, false)
);
binder.bind(JettyServerInitializer.class).to(ProxyJettyServerInit.class).in(LazySingleton.class);
binder.bind(AuthorizerMapper.class).toInstance(

View File

@ -19,28 +19,25 @@
package io.druid.server;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.net.HostAndPort;
import io.druid.server.initialization.ServerConfig;
import io.druid.jackson.DefaultObjectMapper;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.lang.reflect.Field;
public class DruidNodeTest
{
private ServerConfig serverConfig;
private Field plainTextField;
private Field tlsField;
private final ObjectMapper mapper;
@Before
public void setUp() throws Exception
public DruidNodeTest()
{
serverConfig = new ServerConfig();
plainTextField = serverConfig.getClass().getDeclaredField("plaintext");
tlsField = serverConfig.getClass().getDeclaredField("tls");
plainTextField.setAccessible(true);
tlsField.setAccessible(true);
mapper = new DefaultObjectMapper();
mapper.setInjectableValues(
new InjectableValues.Std()
.addValue(Integer.class, Integer.MAX_VALUE)
.addValue(String.class, "DUMMY")
);
}
@Test
@ -50,159 +47,135 @@ public class DruidNodeTest
DruidNode node;
node = new DruidNode(service, null, null, null, serverConfig);
node = new DruidNode(service, null, null, null, true, false);
Assert.assertEquals(DruidNode.getDefaultHost(), node.getHost());
Assert.assertEquals(-1, node.getPlaintextPort());
// Hosts which report only ipv6 will have getDefaultHost() report something like fe80::6e40:8ff:fe93:9230
// but getHostAndPort() reports [fe80::6e40:8ff:fe93:9230]
Assert.assertEquals(HostAndPort.fromString(DruidNode.getDefaultHost()).toString(), node.getHostAndPort());
node = new DruidNode(service, "2001:db8:85a3::8a2e:370:7334", -1, null, serverConfig);
node = new DruidNode(service, "2001:db8:85a3::8a2e:370:7334", -1, null, true, false);
Assert.assertEquals("2001:db8:85a3::8a2e:370:7334", node.getHost());
Assert.assertEquals(-1, node.getPlaintextPort());
Assert.assertEquals("[2001:db8:85a3::8a2e:370:7334]", node.getHostAndPort());
node = new DruidNode(service, "abc:123", null, null, serverConfig);
node = new DruidNode(service, "abc:123", null, null, true, false);
Assert.assertEquals("abc", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals("abc:123", node.getHostAndPort());
node = new DruidNode(service, "2001:db8:85a3::8a2e:370:7334", null, null, serverConfig);
node = new DruidNode(service, "2001:db8:85a3::8a2e:370:7334", null, null, true, false);
Assert.assertEquals("2001:db8:85a3::8a2e:370:7334", node.getHost());
Assert.assertTrue(8080 <= node.getPlaintextPort());
node = new DruidNode(service, "[2001:db8:85a3::8a2e:370:7334]", null, null, serverConfig);
node = new DruidNode(service, "[2001:db8:85a3::8a2e:370:7334]", null, null, true, false);
Assert.assertEquals("2001:db8:85a3::8a2e:370:7334", node.getHost());
Assert.assertTrue(8080 <= node.getPlaintextPort());
node = new DruidNode(service, "abc", null, null, serverConfig);
node = new DruidNode(service, "abc", null, null, true, false);
Assert.assertEquals("abc", node.getHost());
Assert.assertTrue(8080 <= node.getPlaintextPort());
node = new DruidNode(service, "abc", 123, null, serverConfig);
node = new DruidNode(service, "abc", 123, null, true, false);
Assert.assertEquals("abc", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals("abc:123", node.getHostAndPort());
node = new DruidNode(service, "abc:123", 123, null, serverConfig);
node = new DruidNode(service, "abc:123", 123, null, true, false);
Assert.assertEquals("abc", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals("abc:123", node.getHostAndPort());
node = new DruidNode(service, "[2001:db8:85a3::8a2e:370:7334]:123", null, null, serverConfig);
node = new DruidNode(service, "[2001:db8:85a3::8a2e:370:7334]:123", null, null, true, false);
Assert.assertEquals("2001:db8:85a3::8a2e:370:7334", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals("[2001:db8:85a3::8a2e:370:7334]:123", node.getHostAndPort());
node = new DruidNode(service, "2001:db8:85a3::8a2e:370:7334", 123, null, serverConfig);
node = new DruidNode(service, "2001:db8:85a3::8a2e:370:7334", 123, null, true, false);
Assert.assertEquals("2001:db8:85a3::8a2e:370:7334", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals("[2001:db8:85a3::8a2e:370:7334]:123", node.getHostAndPort());
node = new DruidNode(service, "[2001:db8:85a3::8a2e:370:7334]", 123, null, serverConfig);
node = new DruidNode(service, "[2001:db8:85a3::8a2e:370:7334]", 123, null, true, false);
Assert.assertEquals("2001:db8:85a3::8a2e:370:7334", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals("[2001:db8:85a3::8a2e:370:7334]:123", node.getHostAndPort());
node = new DruidNode(service, null, 123, null, serverConfig);
node = new DruidNode(service, null, 123, null, true, false);
Assert.assertEquals(DruidNode.getDefaultHost(), node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
node = new DruidNode(service, null, 123, 123, serverConfig);
node = new DruidNode(service, null, 123, 123, true, false);
Assert.assertEquals(DruidNode.getDefaultHost(), node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals(-1, node.getTlsPort());
node = new DruidNode(service, "host", 123, 123, serverConfig);
node = new DruidNode(service, "host", 123, 123, true, false);
Assert.assertEquals("host", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals(-1, node.getTlsPort());
node = new DruidNode(service, "host:123", null, 123, serverConfig);
node = new DruidNode(service, "host:123", null, 123, true, false);
Assert.assertEquals("host", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals(-1, node.getTlsPort());
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, true);
node = new DruidNode("test", "host:123", null, 214, serverConfig);
node = new DruidNode("test", "host:123", null, 214, true, true);
Assert.assertEquals("host", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals(214, node.getTlsPort());
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, true);
node = new DruidNode("test", "host", 123, 214, serverConfig);
node = new DruidNode("test", "host", 123, 214, true, true);
Assert.assertEquals("host", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals(214, node.getTlsPort());
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, true);
node = new DruidNode("test", "host:123", 123, 214, serverConfig);
node = new DruidNode("test", "host:123", 123, 214, true, true);
Assert.assertEquals("host", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals(214, node.getTlsPort());
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, true);
node = new DruidNode("test", null, 123, 214, serverConfig);
node = new DruidNode("test", null, 123, 214, true, true);
Assert.assertEquals(DruidNode.getDefaultHost(), node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals(214, node.getTlsPort());
plainTextField.setBoolean(serverConfig, false);
tlsField.setBoolean(serverConfig, true);
node = new DruidNode("test", "host:123", null, 214, serverConfig);
node = new DruidNode("test", "host:123", null, 214, false, true);
Assert.assertEquals("host", node.getHost());
Assert.assertEquals(-1, node.getPlaintextPort());
Assert.assertEquals(214, node.getTlsPort());
plainTextField.setBoolean(serverConfig, false);
tlsField.setBoolean(serverConfig, true);
node = new DruidNode("test", "host:123", null, 123, serverConfig);
node = new DruidNode("test", "host:123", null, 123, false, true);
Assert.assertEquals("host", node.getHost());
Assert.assertEquals(-1, node.getPlaintextPort());
Assert.assertEquals(123, node.getTlsPort());
plainTextField.setBoolean(serverConfig, false);
tlsField.setBoolean(serverConfig, true);
node = new DruidNode("test", null, null, 123, serverConfig);
node = new DruidNode("test", null, null, 123, false, true);
Assert.assertEquals(DruidNode.getDefaultHost(), node.getHost());
Assert.assertEquals(-1, node.getPlaintextPort());
Assert.assertEquals(123, node.getTlsPort());
plainTextField.setBoolean(serverConfig, false);
tlsField.setBoolean(serverConfig, true);
node = new DruidNode("test", null, -1, 123, serverConfig);
node = new DruidNode("test", null, -1, 123, false, true);
Assert.assertEquals(DruidNode.getDefaultHost(), node.getHost());
Assert.assertEquals(-1, node.getPlaintextPort());
Assert.assertEquals(123, node.getTlsPort());
plainTextField.setBoolean(serverConfig, false);
tlsField.setBoolean(serverConfig, true);
node = new DruidNode("test", "host", -1, 123, serverConfig);
node = new DruidNode("test", "host", -1, 123, false, true);
Assert.assertEquals("host", node.getHost());
Assert.assertEquals(-1, node.getPlaintextPort());
Assert.assertEquals(123, node.getTlsPort());
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, false);
node = new DruidNode("test", "host", -1, 123, serverConfig);
node = new DruidNode("test", "host", -1, 123, true, false);
Assert.assertEquals("host", node.getHost());
Assert.assertEquals(-1, node.getPlaintextPort());
Assert.assertEquals(-1, node.getTlsPort());
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, false);
node = new DruidNode("test", "host:123", 123, null, serverConfig);
node = new DruidNode("test", "host:123", 123, null, true, false);
Assert.assertEquals("host", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals(-1, node.getTlsPort());
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, false);
node = new DruidNode("test", "host:123", null, 123, serverConfig);
node = new DruidNode("test", "host:123", null, 123, true, false);
Assert.assertEquals("host", node.getHost());
Assert.assertEquals(123, node.getPlaintextPort());
Assert.assertEquals(-1, node.getTlsPort());
@ -211,89 +184,73 @@ public class DruidNodeTest
@Test(expected = IllegalArgumentException.class)
public void testConflictingPorts() throws Exception
{
new DruidNode("test/service", "abc:123", 456, null, new ServerConfig());
new DruidNode("test/service", "abc:123", 456, null, true, false);
}
@Test(expected = IllegalArgumentException.class)
public void testAtLeastTlsOrPlainTextIsSet() throws Exception
{
plainTextField.setBoolean(serverConfig, false);
tlsField.setBoolean(serverConfig, false);
new DruidNode("test", "host:123", null, 123, serverConfig);
new DruidNode("test", "host:123", null, 123, false, false);
}
@Test(expected = IllegalArgumentException.class)
public void testSamePlainTextAndTlsPort() throws Exception
{
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, true);
new DruidNode("test", "host:123", null, 123, serverConfig);
new DruidNode("test", "host:123", null, 123, true, true);
}
@Test(expected = IllegalArgumentException.class)
public void testSamePlainTextAndTlsPort1() throws Exception
{
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, true);
new DruidNode("test", "host", 123, 123, serverConfig);
new DruidNode("test", "host", 123, 123, true, true);
}
@Test(expected = IllegalArgumentException.class)
public void testNullTlsPort() throws Exception
{
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, true);
new DruidNode("test", "host:123", null, null, serverConfig);
new DruidNode("test", "host:123", null, null, true, true);
}
@Test(expected = IllegalArgumentException.class)
public void testNullPlainTextAndTlsPort1() throws Exception
{
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, true);
new DruidNode("test", "host", null, null, serverConfig);
new DruidNode("test", "host", null, null, true, true);
}
@Test(expected = IllegalArgumentException.class)
public void testNullTlsPort1() throws Exception
{
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, true);
new DruidNode("test", "host:123", 123, null, serverConfig);
new DruidNode("test", "host:123", 123, null, true, true);
}
@Test(expected = IllegalArgumentException.class)
public void testNullPlainTextAndTlsPort() throws Exception
{
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, true);
new DruidNode("test", null, null, null, serverConfig);
new DruidNode("test", null, null, null, true, true);
}
@Test(expected = IllegalArgumentException.class)
public void testConflictingPlainTextPort() throws Exception
{
plainTextField.setBoolean(serverConfig, true);
tlsField.setBoolean(serverConfig, true);
new DruidNode("test", "host:123", 321, null, serverConfig);
new DruidNode("test", "host:123", 321, null, true, true);
}
@Test(expected = IllegalArgumentException.class)
public void testInvalidIPv6WithPort() throws Exception
{
new DruidNode("test/service", "[abc:fff]:123", 456, null, new ServerConfig());
new DruidNode("test/service", "[abc:fff]:123", 456, null, true, false);
}
@Test(expected = IllegalArgumentException.class)
public void testInvalidIPv6() throws Exception
{
new DruidNode("test/service", "abc:fff", 456, null, new ServerConfig());
new DruidNode("test/service", "abc:fff", 456, null, true, false);
}
@Test(expected = IllegalArgumentException.class)
public void testConflictingPortsNonsense() throws Exception
{
new DruidNode("test/service", "[2001:db8:85a3::8a2e:370:7334]:123", 456, null, new ServerConfig());
new DruidNode("test/service", "[2001:db8:85a3::8a2e:370:7334]:123", 456, null, true, false);
}
@Test
@ -302,11 +259,10 @@ public class DruidNodeTest
final String serviceName = "serviceName";
final String host = "some.host";
final int port = 9898;
final ServerConfig serverConfig = new ServerConfig();
Assert.assertEquals(new DruidNode(serviceName, host, port, null, serverConfig), new DruidNode(serviceName, host, port, null, serverConfig));
Assert.assertNotEquals(new DruidNode(serviceName, host, port, null, serverConfig), new DruidNode(serviceName, host, -1, null, serverConfig));
Assert.assertNotEquals(new DruidNode(serviceName, host, port, null, serverConfig), new DruidNode(serviceName, "other.host", port, null, serverConfig));
Assert.assertNotEquals(new DruidNode(serviceName, host, port, null, serverConfig), new DruidNode("otherServiceName", host, port, null, serverConfig));
Assert.assertEquals(new DruidNode(serviceName, host, port, null, true, false), new DruidNode(serviceName, host, port, null, true, false));
Assert.assertNotEquals(new DruidNode(serviceName, host, port, null, true, false), new DruidNode(serviceName, host, -1, null, true, false));
Assert.assertNotEquals(new DruidNode(serviceName, host, port, null, true, false), new DruidNode(serviceName, "other.host", port, null, true, false));
Assert.assertNotEquals(new DruidNode(serviceName, host, port, null, true, false), new DruidNode("otherServiceName", host, port, null, true, false));
}
@Test
@ -316,11 +272,140 @@ public class DruidNodeTest
final String serviceName = "serviceName";
final String host = "some.host";
final int port = 9898;
final ServerConfig serverConfig = new ServerConfig();
Assert.assertEquals(new DruidNode(serviceName, host, port, null, serverConfig).hashCode(), new DruidNode(serviceName, host, port, null, serverConfig).hashCode());
Assert.assertEquals(new DruidNode(serviceName, host, port, null, true, false).hashCode(), new DruidNode(serviceName, host, port, null, true, false).hashCode());
// Potential hash collision if hashCode method ever changes
Assert.assertNotEquals(new DruidNode(serviceName, host, port, null, serverConfig).hashCode(), new DruidNode(serviceName, host, -1, null, serverConfig).hashCode());
Assert.assertNotEquals(new DruidNode(serviceName, host, port, null, serverConfig).hashCode(), new DruidNode(serviceName, "other.host", port, null, serverConfig).hashCode());
Assert.assertNotEquals(new DruidNode(serviceName, host, port, null, serverConfig).hashCode(), new DruidNode("otherServiceName", host, port, null, serverConfig).hashCode());
Assert.assertNotEquals(new DruidNode(serviceName, host, port, null, true, false).hashCode(), new DruidNode(serviceName, host, -1, null, true, false).hashCode());
Assert.assertNotEquals(new DruidNode(serviceName, host, port, null, true, false).hashCode(), new DruidNode(serviceName, "other.host", port, null, true, false).hashCode());
Assert.assertNotEquals(new DruidNode(serviceName, host, port, null, true, false).hashCode(), new DruidNode("otherServiceName", host, port, null, true, false).hashCode());
}
@Test
public void testSerde1() throws Exception
{
DruidNode actual = mapper.readValue(
mapper.writeValueAsString(new DruidNode("service", "host", 1234, null, 5678, true, true)),
DruidNode.class
);
Assert.assertEquals("service", actual.getServiceName());
Assert.assertEquals("host", actual.getHost());
Assert.assertTrue(actual.isEnablePlaintextPort());
Assert.assertTrue(actual.isEnableTlsPort());
Assert.assertEquals(1234, actual.getPlaintextPort());
Assert.assertEquals(5678, actual.getTlsPort());
}
@Test
public void testSerde2() throws Exception
{
DruidNode actual = mapper.readValue(
mapper.writeValueAsString(new DruidNode("service", "host", 1234, null, 5678, null, false)),
DruidNode.class
);
Assert.assertEquals("service", actual.getServiceName());
Assert.assertEquals("host", actual.getHost());
Assert.assertTrue(actual.isEnablePlaintextPort());
Assert.assertFalse(actual.isEnableTlsPort());
Assert.assertEquals(1234, actual.getPlaintextPort());
Assert.assertEquals(-1, actual.getTlsPort());
}
@Test
public void testSerde3() throws Exception
{
DruidNode actual = mapper.readValue(
mapper.writeValueAsString(new DruidNode("service", "host", 1234, null, 5678, false, true)),
DruidNode.class
);
Assert.assertEquals("service", actual.getServiceName());
Assert.assertEquals("host", actual.getHost());
Assert.assertFalse(actual.isEnablePlaintextPort());
Assert.assertTrue(actual.isEnableTlsPort());
Assert.assertEquals(-1, actual.getPlaintextPort());
Assert.assertEquals(5678, actual.getTlsPort());
}
@Test
public void testDeserialization1() throws Exception
{
String json = "{\n"
+ " \"service\":\"service\",\n"
+ " \"host\":\"host\",\n"
+ " \"plaintextPort\":1234,\n"
+ " \"tlsPort\":5678,\n"
+ " \"enablePlaintextPort\":true,\n"
+ " \"enableTlsPort\":true\n"
+ "}\n";
DruidNode actual = mapper.readValue(json, DruidNode.class);
Assert.assertEquals(new DruidNode("service", "host", 1234, null, 5678, true, true), actual);
Assert.assertEquals("https", actual.getServiceScheme());
Assert.assertEquals("host:1234", actual.getHostAndPort());
Assert.assertEquals("host:5678", actual.getHostAndTlsPort());
Assert.assertEquals("host:5678", actual.getHostAndPortToUse());
}
@Test
public void testDeserialization2() throws Exception
{
String json = "{\n"
+ " \"service\":\"service\",\n"
+ " \"host\":\"host\",\n"
+ " \"plaintextPort\":1234,\n"
+ " \"tlsPort\":5678,\n"
+ " \"enablePlaintextPort\":true"
+ "}\n";
DruidNode actual = mapper.readValue(json, DruidNode.class);
Assert.assertEquals(new DruidNode("service", "host", 1234, null, 5678, true, false), actual);
Assert.assertEquals("http", actual.getServiceScheme());
Assert.assertEquals("host:1234", actual.getHostAndPort());
Assert.assertNull(actual.getHostAndTlsPort());
Assert.assertEquals("host:1234", actual.getHostAndPortToUse());
}
@Test
public void testDeserialization3() throws Exception
{
String json = "{\n"
+ " \"service\":\"service\",\n"
+ " \"host\":\"host\",\n"
+ " \"plaintextPort\":1234,\n"
+ " \"tlsPort\":5678"
+ "}\n";
DruidNode actual = mapper.readValue(json, DruidNode.class);
Assert.assertEquals(new DruidNode("service", "host", 1234, null, 5678, null, false), actual);
Assert.assertEquals("http", actual.getServiceScheme());
Assert.assertEquals("host:1234", actual.getHostAndPort());
Assert.assertNull(actual.getHostAndTlsPort());
Assert.assertEquals("host:1234", actual.getHostAndPortToUse());
}
@Test
public void testDeserialization4() throws Exception
{
String json = "{\n"
+ " \"service\":\"service\",\n"
+ " \"host\":\"host\",\n"
+ " \"port\":1234,\n"
+ " \"tlsPort\":5678"
+ "}\n";
DruidNode actual = mapper.readValue(json, DruidNode.class);
Assert.assertEquals(new DruidNode("service", "host", null, 1234, 5678, null, false), actual);
Assert.assertEquals("http", actual.getServiceScheme());
Assert.assertEquals("host:1234", actual.getHostAndPort());
Assert.assertNull(actual.getHostAndTlsPort());
Assert.assertEquals("host:1234", actual.getHostAndPortToUse());
}
}

View File

@ -44,7 +44,6 @@ 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.ServerConfig;
import io.druid.server.initialization.ZkPathsConfig;
import io.druid.server.lookup.cache.LookupCoordinatorManager;
import io.druid.server.metrics.NoopServiceEmitter;
@ -151,7 +150,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
druidCoordinatorConfig
);
loadQueuePeon.start();
druidNode = new DruidNode("hey", "what", 1234, null, new ServerConfig());
druidNode = new DruidNode("hey", "what", 1234, null, true, false);
loadManagementPeons = new ConcurrentHashMap<>();
scheduledExecutorFactory = new ScheduledExecutorFactory()
{

View File

@ -71,7 +71,7 @@ public class JettyQosTest extends BaseJettyTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", null, null, true, false)
);
binder.bind(JettyServerInitializer.class).to(JettyServerInit.class).in(LazySingleton.class);
Jerseys.addResource(binder, SlowResource.class);

View File

@ -82,7 +82,7 @@ public class JettyTest extends BaseJettyTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", null, null, true, false)
);
binder.bind(JettyServerInitializer.class).to(JettyServerInit.class).in(LazySingleton.class);

View File

@ -28,7 +28,6 @@ import io.druid.discovery.DruidNodeDiscoveryProvider;
import io.druid.discovery.LookupNodeService;
import io.druid.server.DruidNode;
import io.druid.server.http.HostAndPortWithScheme;
import io.druid.server.initialization.ServerConfig;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Before;
@ -53,21 +52,21 @@ public class LookupNodeDiscoveryTest
.andReturn(druidNodeDiscovery);
DiscoveryDruidNode node1 = new DiscoveryDruidNode(
new DruidNode("s1", "h1", 8080, null, new ServerConfig()),
new DruidNode("s1", "h1", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL,
ImmutableMap.of(
LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier1"))
);
DiscoveryDruidNode node2 = new DiscoveryDruidNode(
new DruidNode("s2", "h2", 8080, null, new ServerConfig()),
new DruidNode("s2", "h2", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_PEON,
ImmutableMap.of(
LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier1"))
);
DiscoveryDruidNode node3 = new DiscoveryDruidNode(
new DruidNode("s3", "h3", 8080, null, new ServerConfig()),
new DruidNode("s3", "h3", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_PEON,
ImmutableMap.of(
LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier2"))

View File

@ -30,7 +30,6 @@ import io.druid.guice.JsonConfigProvider;
import io.druid.guice.annotations.Self;
import io.druid.initialization.Initialization;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import org.junit.Assert;
import org.junit.Test;
@ -48,7 +47,7 @@ public class MetricsModuleTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, true, false)
);
}
})
@ -73,7 +72,7 @@ public class MetricsModuleTest
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test-inject", null, null, null, true, false)
);
binder.bind(Key.get(
String.class,

View File

@ -43,7 +43,6 @@ import io.druid.query.timeseries.TimeseriesQuery;
import io.druid.server.DruidNode;
import io.druid.server.coordinator.rules.IntervalLoadRule;
import io.druid.server.coordinator.rules.Rule;
import io.druid.server.initialization.ServerConfig;
import org.easymock.EasyMock;
import org.joda.time.Interval;
import org.junit.After;
@ -75,19 +74,19 @@ public class TieredBrokerHostSelectorTest
druidNodeDiscoveryProvider = EasyMock.createStrictMock(DruidNodeDiscoveryProvider.class);
node1 = new DiscoveryDruidNode(
new DruidNode("hotBroker", "hotHost", 8080, null, new ServerConfig()),
new DruidNode("hotBroker", "hotHost", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_BROKER,
ImmutableMap.of()
);
node2 = new DiscoveryDruidNode(
new DruidNode("coldBroker", "coldHost1", 8080, null, new ServerConfig()),
new DruidNode("coldBroker", "coldHost1", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_BROKER,
ImmutableMap.of()
);
node3 = new DiscoveryDruidNode(
new DruidNode("coldBroker", "coldHost2", 8080, null, new ServerConfig()),
new DruidNode("coldBroker", "coldHost2", 8080, null, true, false),
DruidNodeDiscoveryProvider.NODE_TYPE_BROKER,
ImmutableMap.of()
);

View File

@ -36,7 +36,6 @@ import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.MetadataStorageTablesConfig;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import java.util.List;
@ -106,7 +105,7 @@ public class CreateTables extends GuiceRunnable
binder, Key.get(MetadataStorageTablesConfig.class), MetadataStorageTablesConfig.fromBase(base)
);
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("tools", "localhost", -1, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("tools", "localhost", -1, null, true, false)
);
}
}

View File

@ -40,7 +40,6 @@ import io.druid.java.util.common.logger.Logger;
import io.druid.segment.loading.DataSegmentFinder;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.timeline.DataSegment;
import java.io.IOException;
@ -87,7 +86,7 @@ public class InsertSegment extends GuiceRunnable
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("tools", "localhost", -1, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("tools", "localhost", -1, null, true, false)
);
}
}

View File

@ -39,7 +39,6 @@ import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.MetadataStorageTablesConfig;
import io.druid.segment.loading.DataSegmentKiller;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.tasklogs.TaskLogKiller;
import java.util.List;
@ -92,7 +91,7 @@ public class ResetCluster extends GuiceRunnable
public void configure(Binder binder)
{
JsonConfigProvider.bindInstance(
binder, Key.get(DruidNode.class, Self.class), new DruidNode("tools", "localhost", -1, null, new ServerConfig())
binder, Key.get(DruidNode.class, Self.class), new DruidNode("tools", "localhost", -1, null, true, false)
);
JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class);
}

View File

@ -41,7 +41,6 @@ import io.druid.java.util.common.Pair;
import io.druid.java.util.common.StringUtils;
import io.druid.math.expr.ExprMacroTable;
import io.druid.server.DruidNode;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.security.AuthConfig;
import io.druid.server.security.AuthTestUtils;
import io.druid.sql.calcite.planner.Calcites;
@ -163,7 +162,7 @@ public class DruidAvaticaHandlerTest
);
final DruidAvaticaHandler handler = new DruidAvaticaHandler(
druidMeta,
new DruidNode("dummy", "dummy", 1, null, new ServerConfig()),
new DruidNode("dummy", "dummy", 1, null, true, false),
new AvaticaMonitor()
);
final int port = new Random().nextInt(9999) + 10000;
@ -622,7 +621,7 @@ public class DruidAvaticaHandlerTest
final DruidAvaticaHandler handler = new DruidAvaticaHandler(
smallFrameDruidMeta,
new DruidNode("dummy", "dummy", 1, null, new ServerConfig()),
new DruidNode("dummy", "dummy", 1, null, true, false),
new AvaticaMonitor()
);
final int port = new Random().nextInt(9999) + 20000;