You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by or...@apache.org on 2020/11/16 13:15:59 UTC

[camel] branch master updated: Added test infra services for HDFS v2 (#4615)

This is an automated email from the ASF dual-hosted git repository.

orpiske pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel.git


The following commit(s) were added to refs/heads/master by this push:
     new 9e0fee2  Added test infra services for HDFS v2 (#4615)
9e0fee2 is described below

commit 9e0fee221268f20ab3ff81d8614fee8ed88a9533
Author: Otavio Rodolfo Piske <or...@users.noreply.github.com>
AuthorDate: Mon Nov 16 14:15:20 2020 +0100

    Added test infra services for HDFS v2 (#4615)
    
    Also converted one test case to the new infra
---
 components/camel-hdfs/pom.xml                      | 17 ++++
 .../component/hdfs/integration/HdfsAppendTest.java | 40 +++++++---
 test-infra/camel-test-infra-hdfs/pom.xml           | 57 ++++++++++++++
 .../src/main/resources/META-INF/MANIFEST.MF        |  0
 .../test/infra/hdfs/v2/common/HDFSProperties.java  | 27 +++++++
 .../v2/services/ContainerLocalHDFSService.java     | 88 +++++++++++++++++++++
 .../infra/hdfs/v2/services/DataNodeContainer.java  | 91 ++++++++++++++++++++++
 .../test/infra/hdfs/v2/services/HDFSPorts.java     | 31 ++++++++
 .../test/infra/hdfs/v2/services/HDFSService.java   | 50 ++++++++++++
 .../infra/hdfs/v2/services/HDFSServiceFactory.java | 46 +++++++++++
 .../hdfs/v2/services/HadoopBaseContainer.java      | 41 ++++++++++
 .../infra/hdfs/v2/services/NameNodeContainer.java  | 48 ++++++++++++
 .../infra/hdfs/v2/services/RemoteHDFSService.java  | 54 +++++++++++++
 .../camel/test/infra/hdfs/v2/services/Dockerfile   | 57 ++++++++++++++
 .../test/infra/hdfs/v2/services/core-site.xml      | 27 +++++++
 .../test/infra/hdfs/v2/services/hdfs-site.xml      | 48 ++++++++++++
 .../test/infra/hdfs/v2/services/run-datanode.sh    | 19 +++++
 .../test/infra/hdfs/v2/services/run-namenode.sh    | 23 ++++++
 test-infra/pom.xml                                 |  1 +
 19 files changed, 754 insertions(+), 11 deletions(-)

diff --git a/components/camel-hdfs/pom.xml b/components/camel-hdfs/pom.xml
index 5b1fd21..aaa287fc 100644
--- a/components/camel-hdfs/pom.xml
+++ b/components/camel-hdfs/pom.xml
@@ -147,6 +147,23 @@
             <optional>true</optional>
         </dependency>
 
+        <!-- test infra -->
+        <dependency>
+            <groupId>org.apache.camel</groupId>
+            <artifactId>camel-test-infra-common</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.camel</groupId>
+            <artifactId>camel-test-infra-hdfs</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
     </dependencies>
 
     <!-- skip tests on Windows due CAMEL-8445 -->
diff --git a/components/camel-hdfs/src/test/java/org/apache/camel/component/hdfs/integration/HdfsAppendTest.java b/components/camel-hdfs/src/test/java/org/apache/camel/component/hdfs/integration/HdfsAppendTest.java
index 9b270ec..83634b1 100644
--- a/components/camel-hdfs/src/test/java/org/apache/camel/component/hdfs/integration/HdfsAppendTest.java
+++ b/components/camel-hdfs/src/test/java/org/apache/camel/component/hdfs/integration/HdfsAppendTest.java
@@ -18,6 +18,8 @@ package org.apache.camel.component.hdfs.integration;
 
 import org.apache.camel.Exchange;
 import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.infra.hdfs.v2.services.HDFSService;
+import org.apache.camel.test.infra.hdfs.v2.services.HDFSServiceFactory;
 import org.apache.camel.test.junit5.CamelTestSupport;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -26,13 +28,19 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
-@Disabled("Must run manual")
 public class HdfsAppendTest extends CamelTestSupport {
+    @RegisterExtension
+    public static HDFSService service = HDFSServiceFactory.createService();
+
+    private static final Logger LOG = LoggerFactory.getLogger(HdfsAppendTest.class);
+
     private static final int ITERATIONS = 10;
 
     @Override
@@ -47,7 +55,10 @@ public class HdfsAppendTest extends CamelTestSupport {
 
         Configuration conf = new Configuration();
         conf.addResource("hdfs-test.xml");
-        Path file = new Path("hdfs://localhost:9000/tmp/test/test-camel-simple-write-file1");
+        String path = String.format("hdfs://%s:%d/tmp/test/test-camel-simple-write-file1", service.getHDFSHost(),
+                service.getPort());
+
+        Path file = new Path(path);
         FileSystem fs = FileSystem.get(file.toUri(), conf);
         if (fs.exists(file)) {
             fs.delete(file, true);
@@ -65,7 +76,8 @@ public class HdfsAppendTest extends CamelTestSupport {
             @Override
             public void configure() throws Exception {
                 from("direct:start1")
-                        .to("hdfs://localhost:9000/tmp/test/test-camel-simple-write-file1?append=true&fileSystemType=HDFS");
+                        .toF("hdfs://%s:%d/tmp/test/test-camel-simple-write-file1?append=true&fileSystemType=HDFS",
+                                service.getHDFSHost(), service.getPort());
             }
         });
         startCamelContext();
@@ -75,14 +87,16 @@ public class HdfsAppendTest extends CamelTestSupport {
         }
 
         Configuration conf = new Configuration();
-        Path file = new Path("hdfs://localhost:9000/tmp/test/test-camel-simple-write-file1");
+        String path = String.format("hdfs://%s:%d/tmp/test/test-camel-simple-write-file1", service.getHDFSHost(),
+                service.getPort());
+        Path file = new Path(path);
         FileSystem fs = FileSystem.get(file.toUri(), conf);
         FSDataInputStream in = fs.open(file);
         byte[] buffer = new byte[5];
         int ret = 0;
         for (int i = 0; i < 20; ++i) {
             ret = in.read(buffer);
-            System.out.println("> " + new String(buffer));
+            LOG.info("> {}", new String(buffer));
         }
         ret = in.read(buffer);
         assertEquals(-1, ret);
@@ -95,7 +109,8 @@ public class HdfsAppendTest extends CamelTestSupport {
         context.addRoutes(new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                from("direct:start1").to("hdfs://localhost:9000/tmp/test-dynamic/?append=true&fileSystemType=HDFS");
+                from("direct:start1").toF("hdfs://%s:%d/tmp/test-dynamic/?append=true&fileSystemType=HDFS",
+                        service.getHDFSHost(), service.getPort());
             }
         });
         startCamelContext();
@@ -105,13 +120,16 @@ public class HdfsAppendTest extends CamelTestSupport {
         }
 
         Configuration conf = new Configuration();
-        Path file = new Path("hdfs://localhost:9000/tmp/test-dynamic/camel-hdfs.log");
+        String path = String.format("hdfs://%s:%d/tmp/test-dynamic/camel-hdfs.log", service.getHDFSHost(),
+                service.getPort());
+
+        Path file = new Path(path);
         FileSystem fs = FileSystem.get(file.toUri(), conf);
         FSDataInputStream in = fs.open(file);
         byte[] buffer = new byte[5];
         for (int i = 0; i < ITERATIONS; ++i) {
             assertEquals(5, in.read(buffer));
-            System.out.println("> " + new String(buffer));
+            LOG.info("> {}", new String(buffer));
         }
         int ret = in.read(buffer);
         assertEquals(-1, ret);
@@ -125,10 +143,10 @@ public class HdfsAppendTest extends CamelTestSupport {
 
         Thread.sleep(250);
         Configuration conf = new Configuration();
-        Path dir = new Path("hdfs://localhost:9000/tmp/test");
+        Path dir = new Path(String.format("hdfs://%s:%d/tmp/test", service.getHDFSHost(), service.getPort()));
         FileSystem fs = FileSystem.get(dir.toUri(), conf);
         fs.delete(dir, true);
-        dir = new Path("hdfs://localhost:9000/tmp/test-dynamic");
+        dir = new Path(String.format("hdfs://%s:%d/tmp/test-dynamic", service.getHDFSHost(), service.getPort()));
         fs.delete(dir, true);
     }
 }
diff --git a/test-infra/camel-test-infra-hdfs/pom.xml b/test-infra/camel-test-infra-hdfs/pom.xml
new file mode 100644
index 0000000..735f7b5
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/pom.xml
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *      http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>camel-test-infra-parent</artifactId>
+        <groupId>org.apache.camel</groupId>
+        <relativePath>../camel-test-infra-parent/pom.xml</relativePath>
+        <version>3.7.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>camel-test-infra-hdfs</artifactId>
+    <name>Camel :: Test Infra :: HDFS</name>
+    <description>HDFS test infrastructure for Camel</description>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.camel</groupId>
+            <artifactId>camel-test-infra-common</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.testcontainers</groupId>
+            <artifactId>testcontainers</artifactId>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+</project>
\ No newline at end of file
diff --git a/test-infra/camel-test-infra-hdfs/src/main/resources/META-INF/MANIFEST.MF b/test-infra/camel-test-infra-hdfs/src/main/resources/META-INF/MANIFEST.MF
new file mode 100644
index 0000000..e69de29
diff --git a/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/common/HDFSProperties.java b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/common/HDFSProperties.java
new file mode 100644
index 0000000..057a014
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/common/HDFSProperties.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.camel.test.infra.hdfs.v2.common;
+
+public final class HDFSProperties {
+    public static final String HDFS_HOST = "hdfs.host";
+    public static final String HDFS_PORT = "hdfs.port";
+
+    private HDFSProperties() {
+
+    }
+}
diff --git a/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/ContainerLocalHDFSService.java b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/ContainerLocalHDFSService.java
new file mode 100644
index 0000000..c41efff
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/ContainerLocalHDFSService.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.camel.test.infra.hdfs.v2.services;
+
+import org.apache.camel.test.infra.common.services.ContainerService;
+import org.apache.camel.test.infra.hdfs.v2.common.HDFSProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.Network;
+
+public class ContainerLocalHDFSService implements HDFSService, ContainerService<NameNodeContainer> {
+    private static final Logger LOG = LoggerFactory.getLogger(ContainerLocalHDFSService.class);
+    private final NameNodeContainer nameNodeContainer;
+    private final DataNodeContainer dataNodeContainer;
+
+    public ContainerLocalHDFSService() {
+        Network network = Network.newNetwork();
+
+        nameNodeContainer = new NameNodeContainer(network);
+        dataNodeContainer = new DataNodeContainer(network);
+    }
+
+    @Override
+    public String getHDFSHost() {
+        return nameNodeContainer.getContainerIpAddress();
+    }
+
+    @Override
+    public int getPort() {
+        return nameNodeContainer.getIpcPort();
+    }
+
+    @Override
+    public NameNodeContainer getContainer() {
+        return nameNodeContainer;
+    }
+
+    @Override
+    public void registerProperties() {
+        System.setProperty(HDFSProperties.HDFS_HOST, getHDFSHost());
+        System.getProperty(HDFSProperties.HDFS_PORT, String.valueOf(getPort()));
+    }
+
+    @Override
+    public void initialize() {
+        nameNodeContainer.start();
+
+        registerProperties();
+
+        String hdfsNameNodeWeb = getNameNodeWebAddress();
+        LOG.info("HDFS Name node web UI running at address http://{}", hdfsNameNodeWeb);
+
+        dataNodeContainer.start();
+
+        String hdfsDataNodeWeb = getHdfsDataNodeWeb();
+        LOG.info("HDFS Data node web UI running at address http://{}", hdfsDataNodeWeb);
+        LOG.info("HDFS Data node running at address {}:{}", getHDFSHost(), getPort());
+    }
+
+    private String getHdfsDataNodeWeb() {
+        return dataNodeContainer.getContainerIpAddress() + ":" + dataNodeContainer.getHttpPort();
+    }
+
+    private String getNameNodeWebAddress() {
+        return nameNodeContainer.getContainerIpAddress() + ":" + nameNodeContainer.getHttpPort();
+    }
+
+    @Override
+    public void shutdown() {
+        dataNodeContainer.stop();
+        nameNodeContainer.stop();
+    }
+}
diff --git a/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/DataNodeContainer.java b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/DataNodeContainer.java
new file mode 100644
index 0000000..2f1c7f1
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/DataNodeContainer.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.camel.test.infra.hdfs.v2.services;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.Network;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.dockerclient.DockerClientConfigUtils;
+
+public class DataNodeContainer extends HadoopBaseContainer<DataNodeContainer> {
+    private static final Logger LOG = LoggerFactory.getLogger(DataNodeContainer.class);
+    private static String dataNodeHost = "localhost";
+
+    static {
+        String dockerHost = System.getenv("DOCKER_HOST");
+
+        if (dockerHost != null && !dockerHost.isEmpty()) {
+            try {
+                URI dockerHostUri = new URI(dockerHost);
+                dataNodeHost = DockerClientConfigUtils.getDockerHostIpAddress(dockerHostUri);
+
+            } catch (URISyntaxException e) {
+                LOG.warn("Using 'localhost' as the docker host because the URI '{}' for did not parse correctly: {}",
+                        dockerHost, e.getMessage(), e);
+            }
+        }
+    }
+
+    public DataNodeContainer(Network network) {
+        this(network, dataNodeHost);
+    }
+
+    public DataNodeContainer(Network network, String name) {
+        super(network, name);
+
+        withCommand("sh", "-c", "/hadoop/run-datanode.sh");
+
+        withExposedPorts(HDFSPorts.DATA_NODE_HTTP_PORT, HDFSPorts.DATA_NODE_DATA_TRANSFER_PORT, HDFSPorts.DATA_NODE_IPC_PORT);
+
+        waitingFor(Wait.forHttp("/").forPort(HDFSPorts.DATA_NODE_HTTP_PORT));
+
+        /*
+         We need the name to be a valid hostname: the files are uploaded
+         directly to the dataNode host using the *hostname*. By default, the hostname
+         is not valid and no accessible from outside, therefore we trick the container
+         into using the localhost name so when the data node is resolved, it actually
+         points to the port on the local host that is redirected inside the container.
+         */
+        withCreateContainerCmdModifier(
+                createContainerCmd -> {
+                    createContainerCmd.withHostName(name);
+                    createContainerCmd.withName(name);
+                });
+
+        addFixedExposedPort(HDFSPorts.DATA_NODE_HTTP_PORT, HDFSPorts.DATA_NODE_HTTP_PORT);
+        addFixedExposedPort(HDFSPorts.DATA_NODE_DATA_TRANSFER_PORT, HDFSPorts.DATA_NODE_DATA_TRANSFER_PORT);
+        addFixedExposedPort(HDFSPorts.DATA_NODE_IPC_PORT, HDFSPorts.DATA_NODE_IPC_PORT);
+    }
+
+    public int getHttpPort() {
+        return getMappedPort(HDFSPorts.DATA_NODE_HTTP_PORT);
+    }
+
+    public int getDataTransferPort() {
+        return HDFSPorts.DATA_NODE_DATA_TRANSFER_PORT;
+    }
+
+    public int getIpcPort() {
+        return HDFSPorts.DATA_NODE_IPC_PORT;
+    }
+
+}
diff --git a/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/HDFSPorts.java b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/HDFSPorts.java
new file mode 100644
index 0000000..8b68fd7
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/HDFSPorts.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.camel.test.infra.hdfs.v2.services;
+
+final class HDFSPorts {
+    public static final int NAME_NODE_HTTP_PORT = 50070;
+    public static final int NAME_NODE_IPC_PORT = 8020;
+
+    public static final int DATA_NODE_HTTP_PORT = 50075;
+    public static final int DATA_NODE_DATA_TRANSFER_PORT = 50010;
+    public static final int DATA_NODE_IPC_PORT = 50020;
+
+    private HDFSPorts() {
+    }
+
+}
diff --git a/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/HDFSService.java b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/HDFSService.java
new file mode 100644
index 0000000..e796626
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/HDFSService.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.camel.test.infra.hdfs.v2.services;
+
+import org.apache.camel.test.infra.common.services.TestService;
+import org.junit.jupiter.api.extension.AfterAllCallback;
+import org.junit.jupiter.api.extension.BeforeAllCallback;
+import org.junit.jupiter.api.extension.ExtensionContext;
+
+public interface HDFSService extends TestService, BeforeAllCallback, AfterAllCallback {
+
+    /**
+     * Gets the hostname of the HDFS server
+     * 
+     * @return
+     */
+    String getHDFSHost();
+
+    /**
+     * Gets the port used by the HDFS server
+     * 
+     * @return
+     */
+    int getPort();
+
+    @Override
+    default void beforeAll(ExtensionContext extensionContext) throws Exception {
+        initialize();
+    }
+
+    @Override
+    default void afterAll(ExtensionContext extensionContext) throws Exception {
+        shutdown();
+    }
+}
diff --git a/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/HDFSServiceFactory.java b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/HDFSServiceFactory.java
new file mode 100644
index 0000000..a70989b
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/HDFSServiceFactory.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.camel.test.infra.hdfs.v2.services;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public final class HDFSServiceFactory {
+    private static final Logger LOG = LoggerFactory.getLogger(HDFSServiceFactory.class);
+
+    private HDFSServiceFactory() {
+
+    }
+
+    public static HDFSService createService() {
+        String instanceType = System.getProperty("hdfs.instance.type");
+
+        if (instanceType == null || instanceType.equals("local-hdfs-container")) {
+            return new ContainerLocalHDFSService();
+        }
+
+        if (instanceType.equals("remote")) {
+            return new RemoteHDFSService();
+        }
+
+        LOG.error("Invalid HDFS instance type: {}. Must be either 'remote' or 'local-hdfs-container",
+                instanceType);
+        throw new UnsupportedOperationException(String.format("Invalid HDFS instance type: %s", instanceType));
+
+    }
+}
diff --git a/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/HadoopBaseContainer.java b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/HadoopBaseContainer.java
new file mode 100644
index 0000000..908602b
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/HadoopBaseContainer.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.camel.test.infra.hdfs.v2.services;
+
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.Network;
+import org.testcontainers.images.builder.ImageFromDockerfile;
+
+abstract class HadoopBaseContainer<T extends GenericContainer<T>> extends GenericContainer<T> {
+
+    public HadoopBaseContainer(Network network, String name) {
+        super(new ImageFromDockerfile("hadoop-2x:ckc", false)
+                .withFileFromClasspath(".",
+                        "org/apache/camel/test/infra/hdfs/v2/services/"));
+
+        withNetwork(network);
+
+        withCreateContainerCmdModifier(
+                createContainerCmd -> {
+                    createContainerCmd.withHostName(name);
+                    createContainerCmd.withName(name);
+                });
+    }
+
+    abstract int getHttpPort();
+}
diff --git a/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/NameNodeContainer.java b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/NameNodeContainer.java
new file mode 100644
index 0000000..7ef0745
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/NameNodeContainer.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.camel.test.infra.hdfs.v2.services;
+
+import org.testcontainers.containers.Network;
+import org.testcontainers.containers.wait.strategy.Wait;
+
+public class NameNodeContainer extends HadoopBaseContainer<NameNodeContainer> {
+
+    public NameNodeContainer(Network network) {
+        this(network, "namenode");
+    }
+
+    public NameNodeContainer(Network network, String name) {
+        super(network, name);
+
+        withCommand("sh", "-c", "/hadoop/run-namenode.sh");
+        withExposedPorts(HDFSPorts.NAME_NODE_HTTP_PORT, HDFSPorts.NAME_NODE_IPC_PORT);
+
+        waitingFor(Wait.forHttp("/").forPort(HDFSPorts.NAME_NODE_HTTP_PORT));
+
+        addFixedExposedPort(HDFSPorts.NAME_NODE_HTTP_PORT, HDFSPorts.NAME_NODE_HTTP_PORT);
+        addFixedExposedPort(HDFSPorts.NAME_NODE_IPC_PORT, HDFSPorts.NAME_NODE_IPC_PORT);
+    }
+
+    public int getHttpPort() {
+        return getMappedPort(HDFSPorts.NAME_NODE_HTTP_PORT);
+    }
+
+    public int getIpcPort() {
+        return HDFSPorts.NAME_NODE_IPC_PORT;
+    }
+}
diff --git a/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/RemoteHDFSService.java b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/RemoteHDFSService.java
new file mode 100644
index 0000000..8745b1b
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/java/org/apache/camel/test/infra/hdfs/v2/services/RemoteHDFSService.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.camel.test.infra.hdfs.v2.services;
+
+import org.apache.camel.test.infra.hdfs.v2.common.HDFSProperties;
+
+public class RemoteHDFSService implements HDFSService {
+
+    @Override
+    public void registerProperties() {
+        // NO-OP
+    }
+
+    @Override
+    public void initialize() {
+        registerProperties();
+    }
+
+    @Override
+    public void shutdown() {
+
+    }
+
+    @Override
+    public String getHDFSHost() {
+        return System.getProperty(HDFSProperties.HDFS_HOST);
+    }
+
+    @Override
+    public int getPort() {
+        String strPort = System.getProperty(HDFSProperties.HDFS_PORT);
+
+        if (strPort != null) {
+            return Integer.parseInt(strPort);
+        }
+
+        return 8020;
+    }
+}
diff --git a/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/Dockerfile b/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/Dockerfile
new file mode 100644
index 0000000..1bba317
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/Dockerfile
@@ -0,0 +1,57 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+#  this work for additional information regarding copyright ownership.
+#  The ASF licenses this file to You under the Apache License, Version 2.0
+#  (the "License"); you may not use this file except in compliance with
+#  the License.  You may obtain a copy of the License at
+#
+#         http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+FROM fedora:31 as builder
+ARG HADOOP_VERSION
+ENV HADOOP_VERSION ${HADOOP_VERSION:-2.10.0}
+RUN curl https://archive.apache.org/dist/hadoop/common/hadoop-${HADOOP_VERSION}/hadoop-${HADOOP_VERSION}.tar.gz -o hadoop.tar.gz
+RUN mkdir -p hadoop && tar --strip-components=1 -xvf hadoop.tar.gz -C /hadoop && rm -f hadoop.tar.gz
+
+FROM fedora:31
+MAINTAINER Otavio Rodolfo Piske <an...@gmail.com>
+ARG HADOOP_VERSION
+ENV HADOOP_VERSION ${HADOOP_VERSION:-2.10.0}
+EXPOSE 8020 9000 50010 50020 50070 50075
+RUN dnf install -y java-1.8.0-openjdk-headless tar gzip rsync which procps-ng
+ENV JAVA_HOME /etc/alternatives/jre
+COPY --from=builder /hadoop /hadoop
+ADD core-site.xml /hadoop/etc/hadoop/core-site.xml
+ADD hdfs-site.xml /hadoop/etc/hadoop/hdfs-site.xml
+ADD run-datanode.sh /hadoop
+ADD run-namenode.sh /hadoop
+RUN chmod +x /hadoop/*.sh
+ENV HADOOP_HOME /hadoop
+ENV HADOOP_LOG_DIR=${HADOOP_HOME}/logs
+# Uncomment this line for enabling debug log
+# ENV HADOOP_ROOT_LOGGER=DEBUG,RFA,console
+VOLUME /hdfs/
+WORKDIR /hadoop
+
+# To run this image manually, follow these steps:
+
+# Create a network:
+# docker network create hadoop-tmp
+
+# Namenode starts with (format + runtime):
+# bin/hdfs --config /hadoop/etc/hadoop/  namenode -format && bin/hdfs --config /hadoop/etc/hadoop/  namenode
+
+# All in one shot from docker:
+# docker run --rm -it -p 8020:8020 -p 9000:9000 -p 50070:50070 --network hadoop-tmp --name namenode hadoop /hadoop/run-namenode.sh
+
+# Datanode starts with:
+# docker run -p 50075:50075 -p 50010:50010 -p 50020:50020 --rm -it --network hadoop-tmp --name datanode-1 hadoop /hadoop/run-datanode.sh datanode-1
+
+# If you need more nodes, just increase the number on the datanode name (ie.: datanode-2, datanode-3, etc)s
+# docker run --rm -it --network hadoop-tmp --name datanode-2 hadoop /hadoop/run-datanode.sh
diff --git a/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/core-site.xml b/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/core-site.xml
new file mode 100644
index 0000000..c817310
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/core-site.xml
@@ -0,0 +1,27 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+         http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+
+-->
+
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<configuration>
+    <property>
+        <name>fs.defaultFS</name>
+        <value>hdfs://namenode:8020</value>
+    </property>
+</configuration>
\ No newline at end of file
diff --git a/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/hdfs-site.xml b/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/hdfs-site.xml
new file mode 100644
index 0000000..6a09c85
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/hdfs-site.xml
@@ -0,0 +1,48 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+         http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+
+-->
+
+<!-- This one is used by the HDFS instance on the containers -->
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<configuration>
+    <property>
+        <name>dfs.replication</name>
+        <value>1</value>
+    </property>
+    <property>
+        <name>dfs.namenode.name.dir</name>
+        <value>file:///hdfs/namenode</value>
+    </property>
+    <property>
+        <name>dfs.datanode.data.dir</name>
+        <value>file:///hdfs/datanode</value>
+    </property>
+    <property>
+        <name>dfs.permissions.enabled</name>
+        <value>false</value>
+    </property>
+    <property>
+        <name>dfs.client.use.datanode.hostname</name>
+        <value>true</value>
+    </property>
+    <property>
+        <name>dfs.datanode.use.datanode.hostname</name>
+        <value>true</value>
+    </property>
+</configuration>
diff --git a/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/run-datanode.sh b/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/run-datanode.sh
new file mode 100755
index 0000000..d1cfc3d
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/run-datanode.sh
@@ -0,0 +1,19 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+#  this work for additional information regarding copyright ownership.
+#  The ASF licenses this file to You under the Apache License, Version 2.0
+#  (the "License"); you may not use this file except in compliance with
+#  the License.  You may obtain a copy of the License at
+#
+#         http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+cd "${HDP_HOME}"
+
+bin/hdfs --config /hadoop/etc/hadoop/ datanode
\ No newline at end of file
diff --git a/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/run-namenode.sh b/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/run-namenode.sh
new file mode 100755
index 0000000..269d7c1
--- /dev/null
+++ b/test-infra/camel-test-infra-hdfs/src/test/resources/org/apache/camel/test/infra/hdfs/v2/services/run-namenode.sh
@@ -0,0 +1,23 @@
+#!/bin/bash
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+#  this work for additional information regarding copyright ownership.
+#  The ASF licenses this file to You under the Apache License, Version 2.0
+#  (the "License"); you may not use this file except in compliance with
+#  the License.  You may obtain a copy of the License at
+#
+#         http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+cd "${HDP_HOME}"
+
+echo "Formatting the HDFS"
+bin/hdfs --config /hadoop/etc/hadoop/ namenode -format
+
+echo "Running Hadoop's namenode"
+bin/hdfs --config /hadoop/etc/hadoop/ namenode
\ No newline at end of file
diff --git a/test-infra/pom.xml b/test-infra/pom.xml
index 5123e11..1def8ae 100644
--- a/test-infra/pom.xml
+++ b/test-infra/pom.xml
@@ -51,6 +51,7 @@
         <module>camel-test-infra-messaging-common</module>
         <module>camel-test-infra-artemis</module>
         <module>camel-test-infra-dispatch-router</module>
+        <module>camel-test-infra-hdfs</module>
 
     </modules>
 </project>
\ No newline at end of file