mirror of https://github.com/apache/druid.git
Local docker for running integration tests (#5314)
This commit is contained in:
parent
971d45ab3f
commit
23b0a091b6
|
@ -15,7 +15,18 @@ describing the cluster.
|
||||||
Integration Testing Using Docker
|
Integration Testing Using Docker
|
||||||
-------------------
|
-------------------
|
||||||
|
|
||||||
## Installing Docker
|
For running integration tests using docker there are 2 approaches.
|
||||||
|
If your platform supports docker natively, you can simply set `DOCKER_IP`
|
||||||
|
environment variable to localhost and skip to [Running tests](#running-tests) section.
|
||||||
|
|
||||||
|
```
|
||||||
|
export DOCKER_IP=127.0.0.1
|
||||||
|
```
|
||||||
|
|
||||||
|
The other approach is to use separate virtual machine to run docker
|
||||||
|
containers with help of `docker-machine` tool.
|
||||||
|
|
||||||
|
## Installing Docker Machine
|
||||||
|
|
||||||
Please refer to instructions at [https://github.com/druid-io/docker-druid/blob/master/docker-install.md](https://github.com/druid-io/docker-druid/blob/master/docker-install.md).
|
Please refer to instructions at [https://github.com/druid-io/docker-druid/blob/master/docker-install.md](https://github.com/druid-io/docker-druid/blob/master/docker-install.md).
|
||||||
|
|
||||||
|
|
|
@ -24,10 +24,14 @@ RUN find /var/lib/mysql -type f -exec touch {} \; && service mysql start \
|
||||||
# Setup supervisord
|
# Setup supervisord
|
||||||
ADD supervisord.conf /etc/supervisor/conf.d/supervisord.conf
|
ADD supervisord.conf /etc/supervisor/conf.d/supervisord.conf
|
||||||
|
|
||||||
# unless advertised.host.name is set to docker host ip, publishing data fails
|
# internal docker_ip:9092 endpoint is used to access Kafka from other Docker containers
|
||||||
|
# external docker ip:9093 endpoint is used to access Kafka from test code
|
||||||
# run this last to avoid rebuilding the image every time the ip changes
|
# run this last to avoid rebuilding the image every time the ip changes
|
||||||
ADD docker_ip docker_ip
|
ADD docker_ip docker_ip
|
||||||
RUN perl -pi -e "s/#advertised.listeners=.*/advertised.listeners=PLAINTEXT:\/\/$(cat docker_ip):9092/" /usr/local/kafka/config/server.properties
|
RUN perl -pi -e "s/#listeners=.*/listeners=INTERNAL:\/\/$(resolveip -s $HOSTNAME):9092,EXTERNAL:\/\/$(resolveip -s $HOSTNAME):9093/" /usr/local/kafka/config/server.properties
|
||||||
|
RUN perl -pi -e "s/#advertised.listeners=.*/advertised.listeners=INTERNAL:\/\/$(resolveip -s $HOSTNAME):9092,EXTERNAL:\/\/$(cat docker_ip):9093/" /usr/local/kafka/config/server.properties
|
||||||
|
RUN perl -pi -e "s/#listener.security.protocol.map=.*/listener.security.protocol.map=INTERNAL:PLAINTEXT,EXTERNAL:PLAINTEXT\ninter.broker.listener.name=INTERNAL/" /usr/local/kafka/config/server.properties
|
||||||
|
RUN perl
|
||||||
|
|
||||||
# Expose ports:
|
# Expose ports:
|
||||||
# - 8081: HTTP (coordinator)
|
# - 8081: HTTP (coordinator)
|
||||||
|
|
|
@ -28,7 +28,7 @@ mvn -B dependency:copy-dependencies -DoutputDirectory=$SHARED_DIR/docker/lib
|
||||||
docker build -t druid/cluster $SHARED_DIR/docker
|
docker build -t druid/cluster $SHARED_DIR/docker
|
||||||
|
|
||||||
# Start zookeeper and kafka
|
# Start zookeeper and kafka
|
||||||
docker run -d --privileged --name druid-zookeeper-kafka -p 2181:2181 -p 9092:9092 -v $SHARED_DIR:/shared -v $DOCKERDIR/zookeeper.conf:$SUPERVISORDIR/zookeeper.conf -v $DOCKERDIR/kafka.conf:$SUPERVISORDIR/kafka.conf druid/cluster
|
docker run -d --privileged --name druid-zookeeper-kafka -p 2181:2181 -p 9092:9092 -p 9093:9093 -v $SHARED_DIR:/shared -v $DOCKERDIR/zookeeper.conf:$SUPERVISORDIR/zookeeper.conf -v $DOCKERDIR/kafka.conf:$SUPERVISORDIR/kafka.conf druid/cluster
|
||||||
|
|
||||||
# Start MYSQL
|
# Start MYSQL
|
||||||
docker run -d --privileged --name druid-metadata-storage -v $SHARED_DIR:/shared -v $DOCKERDIR/metadata-storage.conf:$SUPERVISORDIR/metadata-storage.conf druid/cluster
|
docker run -d --privileged --name druid-metadata-storage -v $SHARED_DIR:/shared -v $DOCKERDIR/metadata-storage.conf:$SUPERVISORDIR/metadata-storage.conf druid/cluster
|
||||||
|
|
|
@ -85,11 +85,24 @@ public class DockerConfigProvider implements IntegrationTestingConfigProvider
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getKafkaHost()
|
public String getZookeeperInternalHosts()
|
||||||
{
|
{
|
||||||
return dockerIp + ":9092";
|
// docker container name
|
||||||
|
return "druid-zookeeper-kafka:2181";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getKafkaHost()
|
||||||
|
{
|
||||||
|
return dockerIp + ":9093";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getKafkaInternalHost()
|
||||||
|
{
|
||||||
|
// docker container name
|
||||||
|
return "druid-zookeeper-kafka:9092";
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getProperty(String prop)
|
public String getProperty(String prop)
|
||||||
|
|
|
@ -39,8 +39,18 @@ public interface IntegrationTestingConfig
|
||||||
|
|
||||||
String getZookeeperHosts();
|
String getZookeeperHosts();
|
||||||
|
|
||||||
|
default String getZookeeperInternalHosts()
|
||||||
|
{
|
||||||
|
return getZookeeperHosts();
|
||||||
|
}
|
||||||
|
|
||||||
String getKafkaHost();
|
String getKafkaHost();
|
||||||
|
|
||||||
|
default String getKafkaInternalHost()
|
||||||
|
{
|
||||||
|
return getKafkaHost();
|
||||||
|
}
|
||||||
|
|
||||||
String getProperty(String prop);
|
String getProperty(String prop);
|
||||||
|
|
||||||
String getUsername();
|
String getUsername();
|
||||||
|
|
|
@ -142,7 +142,7 @@ public class ITKafkaIndexingServiceTest extends AbstractIndexerTest
|
||||||
try {
|
try {
|
||||||
LOG.info("supervisorSpec name: [%s]", INDEXER_FILE);
|
LOG.info("supervisorSpec name: [%s]", INDEXER_FILE);
|
||||||
Properties consumerProperties = new Properties();
|
Properties consumerProperties = new Properties();
|
||||||
consumerProperties.put("bootstrap.servers", config.getKafkaHost());
|
consumerProperties.put("bootstrap.servers", config.getKafkaInternalHost());
|
||||||
addFilteredProperties(consumerProperties);
|
addFilteredProperties(consumerProperties);
|
||||||
|
|
||||||
spec = getTaskAsString(INDEXER_FILE)
|
spec = getTaskAsString(INDEXER_FILE)
|
||||||
|
|
|
@ -196,7 +196,7 @@ public class ITKafkaTest extends AbstractIndexerTest
|
||||||
LOG.info("indexerFile name: [%s]", INDEXER_FILE);
|
LOG.info("indexerFile name: [%s]", INDEXER_FILE);
|
||||||
|
|
||||||
Properties consumerProperties = new Properties();
|
Properties consumerProperties = new Properties();
|
||||||
consumerProperties.put("zookeeper.connect", config.getZookeeperHosts());
|
consumerProperties.put("zookeeper.connect", config.getZookeeperInternalHosts());
|
||||||
consumerProperties.put("zookeeper.connection.timeout.ms", "15000");
|
consumerProperties.put("zookeeper.connection.timeout.ms", "15000");
|
||||||
consumerProperties.put("zookeeper.sync.time.ms", "5000");
|
consumerProperties.put("zookeeper.sync.time.ms", "5000");
|
||||||
consumerProperties.put("group.id", Long.toString(System.currentTimeMillis()));
|
consumerProperties.put("group.id", Long.toString(System.currentTimeMillis()));
|
||||||
|
|
Loading…
Reference in New Issue