You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by eo...@apache.org on 2018/10/03 06:15:41 UTC

[bookkeeper] branch master updated: Fix Kerberos tests on JDK11:

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

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


The following commit(s) were added to refs/heads/master by this push:
     new b6db915  Fix Kerberos tests on JDK11:
b6db915 is described below

commit b6db915f7b787b100a4feeb5999bf288cf8737ed
Author: Enrico Olivelli <eo...@gmail.com>
AuthorDate: Wed Oct 3 08:15:35 2018 +0200

    Fix Kerberos tests on JDK11:
    
    
    Descriptions of the changes in this PR:
    - move to Kerby (MiniKDC code copied from Apache ZooKeeper repository)
    - adapt tests to run with new KDC
    - force TCP on krb5.conf for tests (this is needed for Kerby, which opens only TCP by default config)
    
    
    Master Issue: #1710
    
    
    
    
    Author: Enrico Olivelli <eo...@apache.org>
    
    Reviewers: Ivan Kelly <iv...@apache.org>, Sijie Guo <si...@apache.org>
    
    This closes #1716 from eolivelli/fix/kerby-11
---
 bookkeeper-server/pom.xml                          |  33 +-
 .../bookkeeper/meta/LedgerManagerIteratorTest.java |   4 +-
 .../bookkeeper/sasl/GSSAPIBookKeeperTest.java      |  39 +--
 .../java/org/apache/bookkeeper/sasl/MiniKdc.java   | 349 +++++++++++++++++++++
 pom.xml                                            |   4 +-
 5 files changed, 395 insertions(+), 34 deletions(-)

diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml
index 90dcf0a..e154a36 100644
--- a/bookkeeper-server/pom.xml
+++ b/bookkeeper-server/pom.xml
@@ -121,9 +121,28 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-minikdc</artifactId>
-      <scope>test</scope>
+        <groupId>org.apache.kerby</groupId>
+        <artifactId>kerby-config</artifactId>
+        <version>${kerby.version}</version>
+        <scope>test</scope>
+        <exclusions>
+            <exclusion>
+                <groupId>org.slf4j</groupId>
+                <artifactId>*</artifactId>
+            </exclusion>
+        </exclusions>
+    </dependency>
+    <dependency>
+        <groupId>org.apache.kerby</groupId>
+        <artifactId>kerb-simplekdc</artifactId>
+        <version>${kerby.version}</version>
+        <scope>test</scope>
+        <exclusions>
+            <exclusion>
+                <groupId>org.slf4j</groupId>
+                <artifactId>*</artifactId>
+            </exclusion>
+        </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.zookeeper</groupId>
@@ -135,14 +154,6 @@
   <build>
     <plugins>
       <plugin>
-        <!-- for mini-kdc -->
-        <groupId>org.apache.felix</groupId>
-        <artifactId>maven-bundle-plugin</artifactId>
-        <version>${maven-bundle-plugin.version}</version>
-        <inherited>true</inherited>
-        <extensions>true</extensions>
-      </plugin>
-      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
         <version>${maven-jar-plugin.version}</version>
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java
index 1804b2a..ac8223b 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerIteratorTest.java
@@ -39,6 +39,7 @@ import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -52,7 +53,6 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
 import org.apache.bookkeeper.util.MathUtils;
 import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.bookkeeper.versioning.Version;
-import org.apache.mina.util.ConcurrentHashSet;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.ZooDefs;
 import org.junit.After;
@@ -445,7 +445,7 @@ public class LedgerManagerIteratorTest extends LedgerManagerTestCase {
         final long start = MathUtils.nowInNano();
         final CountDownLatch latch = new CountDownLatch(1);
         ArrayList<Thread> threads = new ArrayList<>();
-        final ConcurrentHashSet<Long> createdLedgers = new ConcurrentHashSet<>();
+        final ConcurrentSkipListSet<Long> createdLedgers = new ConcurrentSkipListSet<>();
         for (int i = 0; i < numWriters; ++i) {
             Thread thread = new Thread(safeWrapper(() -> {
                 LedgerManager writerLM = getIndependentLedgerManager();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java
index db39ae7..a538950 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/GSSAPIBookKeeperTest.java
@@ -43,14 +43,13 @@ import org.apache.bookkeeper.client.LedgerEntry;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.conf.TestBKConfiguration;
 import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
-import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.zookeeper.KeeperException;
-import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Rule;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
@@ -66,23 +65,24 @@ public class GSSAPIBookKeeperTest extends BookKeeperClusterTestCase {
     private static final byte[] PASSWD = "testPasswd".getBytes();
     private static final byte[] ENTRY = "TestEntry".getBytes();
 
-    private MiniKdc kdc;
-    private Properties conf;
+    private static MiniKdc kdc;
+    private static Properties conf;
 
-    @Rule
-    public TemporaryFolder kdcDir = new TemporaryFolder();
+    @ClassRule
+    public static TemporaryFolder kdcDir = new TemporaryFolder();
 
-    @Rule
-    public TemporaryFolder kerberosWorkDir = new TemporaryFolder();
+    @ClassRule
+    public static TemporaryFolder kerberosWorkDir = new TemporaryFolder();
 
-    @Before
-    public void startMiniKdc() throws Exception {
+    @BeforeClass
+    public static void startMiniKdc() throws Exception {
 
         conf = MiniKdc.createConf();
         kdc = new MiniKdc(conf, kdcDir.getRoot());
         kdc.start();
 
-        ServerConfiguration bookieConf = newServerConfiguration();
+        // this is just to calculate "localhostName" the same way the bookie does
+        ServerConfiguration bookieConf = TestBKConfiguration.newServerConfiguration();
         bookieConf.setUseHostNameAsBookieID(true);
         String localhostName = Bookie.getBookieAddress(bookieConf).getHostName();
 
@@ -127,16 +127,17 @@ public class GSSAPIBookKeeperTest extends BookKeeperClusterTestCase {
 
         File krb5file = new File(kerberosWorkDir.getRoot(), "krb5.conf");
         try (FileWriter writer = new FileWriter(krb5file)) {
-            writer.write("[libdefaults]\n"
+            String conf = "[libdefaults]\n"
                 + " default_realm = " + kdc.getRealm() + "\n"
+                + " udp_preference_limit = 1\n" // force use TCP
                 + "\n"
                 + "\n"
                 + "[realms]\n"
                 + " " + kdc.getRealm() + "  = {\n"
                 + "  kdc = " + kdc.getHost() + ":" + kdc.getPort() + "\n"
-                + " }"
-            );
-
+                + " }";
+            writer.write(conf);
+            LOG.info("krb5.conf:\n" + conf);
         }
 
         System.setProperty("java.security.auth.login.config", jaasFile.getAbsolutePath());
@@ -145,8 +146,8 @@ public class GSSAPIBookKeeperTest extends BookKeeperClusterTestCase {
 
     }
 
-    @After
-    public void stopMiniKdc() {
+    @AfterClass
+    public static void stopMiniKdc() {
         System.clearProperty("java.security.auth.login.config");
         System.clearProperty("java.security.krb5.conf");
         if (kdc != null) {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/MiniKdc.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/MiniKdc.java
new file mode 100644
index 0000000..f5cb161
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/sasl/MiniKdc.java
@@ -0,0 +1,349 @@
+/**
+ * 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.bookkeeper.sasl;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import org.apache.kerby.kerberos.kerb.KrbException;
+import org.apache.kerby.kerberos.kerb.server.KdcConfigKey;
+import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
+import org.apache.kerby.util.IOUtil;
+import org.apache.kerby.util.NetworkUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Mini KDC based on Apache Directory Server that can be embedded in testcases
+ * or used from command line as a standalone KDC.
+ *
+ * <p><b>From within testcases:</b>
+ *
+ * <p>MiniKdc sets one System property when started and un-set when stopped:
+ * <ul>
+ *   <li>sun.security.krb5.debug: set to the debug value provided in the
+ *   configuration</li>
+ * </ul>
+ * Because of this, multiple MiniKdc instances cannot be started in parallel.
+ * For example, running testcases in parallel that start a KDC each. To
+ * accomplish this a single MiniKdc should be used for all testcases running
+ * in parallel.
+ *
+ * <p>MiniKdc default configuration values are:
+ * <ul>
+ *   <li>org.name=EXAMPLE (used to create the REALM)</li>
+ *   <li>org.domain=COM (used to create the REALM)</li>
+ *   <li>kdc.bind.address=localhost</li>
+ *   <li>kdc.port=0 (ephemeral port)</li>
+ *   <li>instance=DefaultKrbServer</li>
+ *   <li>max.ticket.lifetime=86400000 (1 day)</li>
+ *   <li>max.renewable.lifetime=604800000 (7 days)</li>
+ *   <li>transport=TCP</li>
+ *   <li>debug=false</li>
+ * </ul>
+ * The generated krb5.conf forces TCP connections.
+ * This code is originally from HDFS, see the file name MiniKdc there
+ * in case of bug fixing, history, etc.
+ * https://github.com/apache/hadoop/blob/trunk/hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java
+ */
+public class MiniKdc {
+
+    public static final String JAVA_SECURITY_KRB5_CONF =
+            "java.security.krb5.conf";
+    public static final String SUN_SECURITY_KRB5_DEBUG =
+            "sun.security.krb5.debug";
+
+
+    private static final Logger LOG = LoggerFactory.getLogger(MiniKdc.class);
+
+    public static final String ORG_NAME = "org.name";
+    public static final String ORG_DOMAIN = "org.domain";
+    public static final String KDC_BIND_ADDRESS = "kdc.bind.address";
+    public static final String KDC_PORT = "kdc.port";
+    public static final String INSTANCE = "instance";
+    public static final String MAX_TICKET_LIFETIME = "max.ticket.lifetime";
+    public static final String MAX_RENEWABLE_LIFETIME = "max.renewable.lifetime";
+    public static final String TRANSPORT = "transport";
+    public static final String DEBUG = "debug";
+
+    private static final Set<String> PROPERTIES = new HashSet<String>();
+    private static final Properties DEFAULT_CONFIG = new Properties();
+
+    static {
+        PROPERTIES.add(ORG_NAME);
+        PROPERTIES.add(ORG_DOMAIN);
+        PROPERTIES.add(KDC_BIND_ADDRESS);
+        PROPERTIES.add(KDC_BIND_ADDRESS);
+        PROPERTIES.add(KDC_PORT);
+        PROPERTIES.add(INSTANCE);
+        PROPERTIES.add(TRANSPORT);
+        PROPERTIES.add(MAX_TICKET_LIFETIME);
+        PROPERTIES.add(MAX_RENEWABLE_LIFETIME);
+
+        DEFAULT_CONFIG.setProperty(KDC_BIND_ADDRESS, "localhost");
+        DEFAULT_CONFIG.setProperty(KDC_PORT, "0");
+        DEFAULT_CONFIG.setProperty(INSTANCE, "DefaultKrbServer");
+        DEFAULT_CONFIG.setProperty(ORG_NAME, "EXAMPLE");
+        DEFAULT_CONFIG.setProperty(ORG_DOMAIN, "COM");
+        DEFAULT_CONFIG.setProperty(TRANSPORT, "TCP");
+        DEFAULT_CONFIG.setProperty(MAX_TICKET_LIFETIME, "86400000");
+        DEFAULT_CONFIG.setProperty(MAX_RENEWABLE_LIFETIME, "604800000");
+        DEFAULT_CONFIG.setProperty(DEBUG, "false");
+    }
+
+    /**
+     * Convenience method that returns MiniKdc default configuration.
+     *
+     * <p>The returned configuration is a copy, it can be customized before using
+     * it to create a MiniKdc.
+     * @return a MiniKdc default configuration.
+     */
+    public static Properties createConf() {
+        return (Properties) DEFAULT_CONFIG.clone();
+    }
+
+    private Properties conf;
+    private SimpleKdcServer simpleKdc;
+    private int port;
+    private String realm;
+    private File workDir;
+    private File krb5conf;
+    private String transport;
+    private boolean krb5Debug;
+
+    public void setTransport(String transport) {
+        this.transport = transport;
+    }
+
+    /**
+     * Creates a MiniKdc.
+     *
+     * @param conf MiniKdc configuration.
+     * @param workDir working directory, it should be the build directory. Under
+     * this directory an ApacheDS working directory will be created, this
+     * directory will be deleted when the MiniKdc stops.
+     * @throws Exception thrown if the MiniKdc could not be created.
+     */
+    public MiniKdc(Properties conf, File workDir) throws Exception {
+        if (!conf.keySet().containsAll(PROPERTIES)) {
+            Set<String> missingProperties = new HashSet<String>(PROPERTIES);
+            missingProperties.removeAll(conf.keySet());
+            throw new IllegalArgumentException("Missing configuration properties: "
+                    + missingProperties);
+        }
+        this.workDir = new File(workDir, Long.toString(System.currentTimeMillis()));
+        if (!this.workDir.exists()
+                && !this.workDir.mkdirs()) {
+            throw new RuntimeException("Cannot create directory " + this.workDir);
+        }
+        LOG.info("Configuration:");
+        LOG.info("---------------------------------------------------------------");
+        for (Map.Entry<?, ?> entry : conf.entrySet()) {
+            LOG.info("  {}: {}", entry.getKey(), entry.getValue());
+        }
+        LOG.info("---------------------------------------------------------------");
+        this.conf = conf;
+        port = Integer.parseInt(conf.getProperty(KDC_PORT));
+        String orgName = conf.getProperty(ORG_NAME);
+        String orgDomain = conf.getProperty(ORG_DOMAIN);
+        realm = orgName.toUpperCase(Locale.ENGLISH) + "."
+                + orgDomain.toUpperCase(Locale.ENGLISH);
+    }
+
+    /**
+     * Returns the port of the MiniKdc.
+     *
+     * @return the port of the MiniKdc.
+     */
+    public int getPort() {
+        return port;
+    }
+
+    /**
+     * Returns the host of the MiniKdc.
+     *
+     * @return the host of the MiniKdc.
+     */
+    public String getHost() {
+        return conf.getProperty(KDC_BIND_ADDRESS);
+    }
+
+    /**
+     * Returns the realm of the MiniKdc.
+     *
+     * @return the realm of the MiniKdc.
+     */
+    public String getRealm() {
+        return realm;
+    }
+
+    public File getKrb5conf() {
+        krb5conf = new File(System.getProperty(JAVA_SECURITY_KRB5_CONF));
+        return krb5conf;
+    }
+
+    /**
+     * Starts the MiniKdc.
+     *
+     * @throws Exception thrown if the MiniKdc could not be started.
+     */
+    public synchronized void start() throws Exception {
+        if (simpleKdc != null) {
+            throw new RuntimeException("Already started");
+        }
+        simpleKdc = new SimpleKdcServer();
+        prepareKdcServer();
+        simpleKdc.init();
+        resetDefaultRealm();
+        simpleKdc.start();
+        LOG.info("MiniKdc stated.");
+    }
+
+    private void resetDefaultRealm() throws IOException {
+        InputStream templateResource = new FileInputStream(
+                getKrb5conf().getAbsolutePath());
+        String content = IOUtil.readInput(templateResource);
+        content = content.replaceAll("default_realm = .*\n",
+                "default_realm = " + getRealm() + "\n");
+        IOUtil.writeFile(content, getKrb5conf());
+    }
+
+    private void prepareKdcServer() throws Exception {
+        // transport
+        simpleKdc.setWorkDir(workDir);
+        simpleKdc.setKdcHost(getHost());
+        simpleKdc.setKdcRealm(realm);
+        if (transport == null) {
+            transport = conf.getProperty(TRANSPORT);
+        }
+        if (port == 0) {
+            port = NetworkUtil.getServerPort();
+        }
+        if (transport != null) {
+            if (transport.trim().equals("TCP")) {
+                simpleKdc.setKdcTcpPort(port);
+                simpleKdc.setAllowUdp(false);
+            } else if (transport.trim().equals("UDP")) {
+                simpleKdc.setKdcUdpPort(port);
+                simpleKdc.setAllowTcp(false);
+            } else {
+                throw new IllegalArgumentException("Invalid transport: " + transport);
+            }
+        } else {
+            throw new IllegalArgumentException("Need to set transport!");
+        }
+        simpleKdc.getKdcConfig().setString(KdcConfigKey.KDC_SERVICE_NAME,
+                conf.getProperty(INSTANCE));
+        if (conf.getProperty(DEBUG) != null) {
+            krb5Debug = getAndSet(SUN_SECURITY_KRB5_DEBUG, conf.getProperty(DEBUG));
+        }
+    }
+
+    /**
+     * Stops the MiniKdc.
+     */
+    public synchronized void stop() {
+        if (simpleKdc != null) {
+            try {
+                simpleKdc.stop();
+            } catch (KrbException e) {
+                e.printStackTrace();
+            } finally {
+                if (conf.getProperty(DEBUG) != null) {
+                    System.setProperty(SUN_SECURITY_KRB5_DEBUG,
+                            Boolean.toString(krb5Debug));
+                }
+            }
+        }
+        delete(workDir);
+        try {
+            // Will be fixed in next Kerby version.
+            Thread.sleep(1000);
+        } catch (InterruptedException e) {
+            e.printStackTrace();
+        }
+        LOG.info("MiniKdc stopped.");
+    }
+
+    private void delete(File f) {
+        if (f.isFile()) {
+            if (!f.delete()) {
+                LOG.warn("WARNING: cannot delete file " + f.getAbsolutePath());
+            }
+        } else {
+            for (File c: f.listFiles()) {
+                delete(c);
+            }
+            if (!f.delete()) {
+                LOG.warn("WARNING: cannot delete directory " + f.getAbsolutePath());
+            }
+        }
+    }
+
+    /**
+     * Creates a principal in the KDC with the specified user and password.
+     *
+     * @param principal principal name, do not include the domain.
+     * @param password password.
+     * @throws Exception thrown if the principal could not be created.
+     */
+    public synchronized void createPrincipal(String principal, String password)
+            throws Exception {
+        simpleKdc.createPrincipal(principal, password);
+    }
+
+    /**
+     * Creates multiple principals in the KDC and adds them to a keytab file.
+     *
+     * @param keytabFile keytab file to add the created principals.
+     * @param principals principals to add to the KDC, do not include the domain.
+     * @throws Exception thrown if the principals or the keytab file could not be
+     * created.
+     */
+    public synchronized void createPrincipal(File keytabFile,
+                                             String ... principals)
+            throws Exception {
+        simpleKdc.createPrincipals(principals);
+        if (keytabFile.exists() && !keytabFile.delete()) {
+            LOG.error("Failed to delete keytab file: " + keytabFile);
+        }
+        for (String principal : principals) {
+            simpleKdc.getKadmin().exportKeytab(keytabFile, principal);
+        }
+    }
+
+    /**
+     * Set the System property; return the old value for caching.
+     *
+     * @param sysprop property
+     * @param debug true or false
+     * @return the previous value
+     */
+    private boolean getAndSet(String sysprop, String debug) {
+        boolean old = Boolean.getBoolean(sysprop);
+        System.setProperty(sysprop, debug);
+        return old;
+    }
+}
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index 0b158cf..418e9a4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -128,6 +128,7 @@
     <google.errorprone.version>2.1.2</google.errorprone.version>
     <grpc.version>1.12.0</grpc.version>
     <guava.version>21.0</guava.version>
+    <kerby.version>1.1.1</kerby.version>
     <hadoop.version>2.7.3</hadoop.version>
     <hamcrest.version>1.3</hamcrest.version>
     <hdrhistogram.version>2.1.10</hdrhistogram.version>
@@ -171,7 +172,6 @@
     <jacoco-maven-plugin.version>0.8.0</jacoco-maven-plugin.version>
     <maven-antrun-plugin.version>1.8</maven-antrun-plugin.version>
     <maven-assembly-plugin.version>3.1.0</maven-assembly-plugin.version>
-    <maven-bundle-plugin.version>3.2.0</maven-bundle-plugin.version>
     <maven-checkstyle-plugin.version>3.0.0</maven-checkstyle-plugin.version>
     <maven-clean-plugin.version>2.5</maven-clean-plugin.version>
     <maven-compiler-plugin.version>3.7.0</maven-compiler-plugin.version>
@@ -591,7 +591,7 @@
       <dependency>
         <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-minikdc</artifactId>
-        <version>${hadoop.version}</version>
+        <version>${hadoop.minikdc.version}</version>
       </dependency>
       <dependency>
         <groupId>org.arquillian.cube</groupId>