You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vi...@apache.org on 2018/06/26 05:17:38 UTC

[2/8] hive git commit: Implements the logic to generate and build a docker image in DockerTestPrepPhase

Implements the logic to generate and build a docker image in DockerTestPrepPhase


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/41928126
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/41928126
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/41928126

Branch: refs/heads/HIVE-19429
Commit: 41928126427f3b9f0a5732816712041211675ac0
Parents: 9765a43
Author: Vihang Karajgaonkar <vi...@cloudera.com>
Authored: Sat Jun 2 17:36:51 2018 -0700
Committer: Vihang Karajgaonkar <vi...@cloudera.com>
Committed: Mon Jun 4 11:33:21 2018 -0700

----------------------------------------------------------------------
 .../hive/ptest/execution/ContainerClient.java   |  31 ++++
 .../ptest/execution/ContainerClientFactory.java |  58 +++++++
 .../execution/DockerBasedExecutionPhase.java    |   3 +
 .../hive/ptest/execution/HostExecutor.java      |   2 +-
 .../org/apache/hive/ptest/execution/Phase.java  |   1 +
 .../apache/hive/ptest/execution/PrepPhase.java  |  37 +++--
 .../execution/containers/DockerClient.java      |  94 +++++++++++
 .../execution/containers/DockerPrepPhase.java   |  73 +++++++++
 .../src/main/resources/dockerfile-template.vm   |  33 ++++
 .../hive/ptest/execution/AbstractTestPhase.java |   4 +-
 .../ptest/execution/TestPtestOnDockers.java     | 161 +++++++++++++++++++
 .../containers/TestDockerPrepPhase.java         |  73 +++++++++
 12 files changed, 553 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/41928126/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ContainerClient.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ContainerClient.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ContainerClient.java
new file mode 100644
index 0000000..a9a8ead
--- /dev/null
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ContainerClient.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.hive.ptest.execution;
+
+import org.apache.hive.ptest.execution.ContainerClientFactory.ContainerClientContext;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+public interface ContainerClient {
+  void defineImage(String dir, String label) throws IOException;
+  String getBuildCommand(String dir, long toWait, TimeUnit unit)
+      throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/41928126/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ContainerClientFactory.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ContainerClientFactory.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ContainerClientFactory.java
new file mode 100644
index 0000000..d14176b
--- /dev/null
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ContainerClientFactory.java
@@ -0,0 +1,58 @@
+/*
+ *
+ *  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.hive.ptest.execution;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.hive.ptest.api.server.TestLogger;
+import org.apache.hive.ptest.execution.containers.DockerClient;
+import org.slf4j.Logger;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public abstract class ContainerClientFactory {
+  public enum ContainerType {
+    DOCKER
+  }
+  public static ContainerClient get(ContainerType containerType, ContainerClientContext context) throws Exception {
+    switch(containerType) {
+    case DOCKER:
+      return new DockerClient(context);
+    default:
+      throw new Exception("Unknown container type");
+    }
+  }
+
+  public static class ContainerClientContext {
+    final Logger logger;
+    final Map<String, String> templateDefaults;
+    public ContainerClientContext(Logger logger, Map<String, String> templateDefaults) {
+      this.logger = logger;
+      this.templateDefaults = new HashMap<>(templateDefaults);
+    }
+    public Logger getLogger() {
+      return logger;
+    }
+
+    public Map<String, String> getTemplateDefaults() {
+      return templateDefaults;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/41928126/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/DockerBasedExecutionPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/DockerBasedExecutionPhase.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/DockerBasedExecutionPhase.java
index 4b19ccc..d7f14f0 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/DockerBasedExecutionPhase.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/DockerBasedExecutionPhase.java
@@ -23,12 +23,15 @@ import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableMap;
 import org.apache.hive.ptest.execution.conf.TestBatch;
 import org.apache.hive.ptest.execution.context.ExecutionContext;
+import org.apache.hive.ptest.execution.ssh.RemoteCommandResult;
 import org.slf4j.Logger;
 
 import java.io.File;
+import java.io.FileWriter;
 import java.io.IOException;
 import java.util.List;
 import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
 public class DockerBasedExecutionPhase extends ExecutionPhase {

http://git-wip-us.apache.org/repos/asf/hive/blob/41928126/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
index 47347eb..24f0bf8 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
@@ -55,7 +55,7 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 
-class HostExecutor {
+public class HostExecutor {
   private final Host mHost;
   private final List<Drone> mDrones;
   private final ListeningExecutorService mExecutor;

http://git-wip-us.apache.org/repos/asf/hive/blob/41928126/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/Phase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/Phase.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/Phase.java
index 34c66ce..45007f0 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/Phase.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/Phase.java
@@ -160,6 +160,7 @@ public abstract class Phase {
     results.addAll(flatten(futures));
     return results;
   }
+
   private <T extends RemoteCommandResult> List<T> flatten(List<ListenableFuture<List<ListenableFuture<T>>>> futures)
       throws Exception {
     List<T> results = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/hive/blob/41928126/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
index 8fef413..1f57e3a 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java
@@ -24,13 +24,14 @@ import java.io.PrintWriter;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.hive.ptest.execution.ssh.NonZeroExitCodeException;
 import org.slf4j.Logger;
 
 import com.google.common.collect.ImmutableMap;
 
 public class PrepPhase extends Phase {
-  private final File mScratchDir;
-  private final File mPatchFile;
+  protected final File mScratchDir;
+  protected final File mPatchFile;
 
   public PrepPhase(List<HostExecutor> hostExecutors,
       LocalCommandFactory localCommandFactory,
@@ -40,23 +41,12 @@ public class PrepPhase extends Phase {
     this.mScratchDir = scratchDir;
     this.mPatchFile = patchFile;
   }
+
   @Override
   public void execute() throws Exception {
     execLocally("rm -rf $workingDir/scratch");
     execLocally("mkdir -p $workingDir/scratch");
-    if(mPatchFile != null) {
-      File smartApplyPatch = new File(mScratchDir, "smart-apply-patch.sh");
-      PrintWriter writer = new PrintWriter(smartApplyPatch);
-      try {
-        writer.write(Templates.readResource("smart-apply-patch.sh"));
-        if(writer.checkError()) {
-          throw new IOException("Error writing to " + smartApplyPatch);
-        }
-      } finally {
-        writer.close();
-      }
-      execLocally("cp -f " + mPatchFile.getPath() + " $workingDir/scratch/build.patch");
-    }
+    createPatchFiles();
     long start;
     long elapsedTime;
     // source prep
@@ -72,4 +62,21 @@ public class PrepPhase extends Phase {
         TimeUnit.MILLISECONDS);
     logger.info("PERF: source prep took " + elapsedTime + " minutes");
   }
+
+  protected void createPatchFiles()
+      throws IOException, InterruptedException, NonZeroExitCodeException {
+    if(mPatchFile != null) {
+      File smartApplyPatch = new File(mScratchDir, "smart-apply-patch.sh");
+      PrintWriter writer = new PrintWriter(smartApplyPatch);
+      try {
+        writer.write(Templates.readResource("smart-apply-patch.sh"));
+        if(writer.checkError()) {
+          throw new IOException("Error writing to " + smartApplyPatch);
+        }
+      } finally {
+        writer.close();
+      }
+      execLocally("cp -f " + mPatchFile.getPath() + " $workingDir/scratch/build.patch");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/41928126/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/containers/DockerClient.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/containers/DockerClient.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/containers/DockerClient.java
new file mode 100644
index 0000000..d2428a9
--- /dev/null
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/containers/DockerClient.java
@@ -0,0 +1,94 @@
+/*
+ *
+ *  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.hive.ptest.execution.containers;
+
+import org.apache.hive.ptest.api.server.TestLogger;
+import org.apache.hive.ptest.execution.ContainerClient;
+import org.apache.hive.ptest.execution.ContainerClientFactory.ContainerClientContext;
+import org.apache.hive.ptest.execution.Templates;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+public class DockerClient implements ContainerClient {
+
+  private static final String USER = "ptestuser";
+  private static final String HOME_DIR = File.separatorChar + "home" + File.separatorChar + USER;
+  private static final String DOCKER_SCRATCH_DIR = HOME_DIR + File.separatorChar + "scratch" + File.separatorChar;
+  private static Logger logger;
+  private final ContainerClientContext context;
+  public DockerClient(ContainerClientContext context) {
+    logger = context.getLogger();
+    this.context = context;
+  }
+
+  @Override
+  public void defineImage(String dir, String label) throws IOException {
+    if (label == null)
+      label = UUID.randomUUID().toString();
+    File dockerfile = new File(dir, "Dockerfile");
+    logger.info("Writing {} from template", dockerfile);
+    Map<String, String> templateDefaults = context.getTemplateDefaults();
+    if (!templateDefaults.containsKey("label")) {
+      templateDefaults.put("label", label);
+    } else {
+      //TODO throw?
+    }
+    Templates.writeTemplateResult("dockerfile-template.vm", dockerfile, templateDefaults);
+  }
+
+  @Override
+  public String getBuildCommand(String dir, long toWait, TimeUnit unit) throws Exception {
+    //TODO timeout not used currently
+    long seconds = TimeUnit.SECONDS.convert(toWait, unit);
+    logger.info("Building image");
+    String dockerBuildCommand =
+        new StringBuilder("docker build")
+            //TODO do we need --tag?
+            //.append(" --tag " + imageName())
+            .append(" --build-arg ")
+            .append(" workingDir=$workingDir")
+            .append(" --build-arg ")
+            .append(" patchFilePath=$workingDir/scratch/build.patch")
+            //TODO confirm if we need antEnvOpts
+            //.append(" antEnvOpts=$antEnvOpts")
+            .append(" --build-arg ")
+            .append(" mavenEnvOpts=$mavenEnvOpts")
+            .append(" --build-arg ")
+            .append(" repository=$repository")
+            .append(" --build-arg ")
+            .append(" branch=$branch")
+            .append(" " + dir)
+            .toString();
+    return dockerBuildCommand;
+  }
+
+  private String imageName() {
+    //TODO add a proper image name using patch number
+    return "Ptest-dummy-build";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/41928126/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/containers/DockerPrepPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/containers/DockerPrepPhase.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/containers/DockerPrepPhase.java
new file mode 100644
index 0000000..b1d9ad6
--- /dev/null
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/containers/DockerPrepPhase.java
@@ -0,0 +1,73 @@
+/*
+ *
+ *  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.hive.ptest.execution.containers;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import org.apache.hive.ptest.execution.ContainerClient;
+import org.apache.hive.ptest.execution.ContainerClientFactory;
+import org.apache.hive.ptest.execution.ContainerClientFactory.ContainerClientContext;
+import org.apache.hive.ptest.execution.ContainerClientFactory.ContainerType;
+import org.apache.hive.ptest.execution.HostExecutor;
+import org.apache.hive.ptest.execution.LocalCommandFactory;
+import org.apache.hive.ptest.execution.PrepPhase;
+import org.slf4j.Logger;
+
+import java.io.File;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class DockerPrepPhase extends PrepPhase {
+  private final ContainerClient containerClient;
+  public DockerPrepPhase(List<HostExecutor> hostExecutors, LocalCommandFactory localCommandFactory,
+      ImmutableMap<String, String> templateDefaults, File scratchDir, File patchFile,
+      Logger logger) throws Exception {
+    super(hostExecutors, localCommandFactory, templateDefaults, scratchDir, patchFile, logger);
+    //TODO create context builder and pass the environment variables to the context
+    ContainerClientContext context = new ContainerClientContext(logger, templateDefaults);
+    containerClient = ContainerClientFactory.get(ContainerType.DOCKER, context);
+  }
+
+  @Override
+  public void execute() throws Exception {
+    execLocally("rm -rf $workingDir/scratch");
+    execLocally("mkdir -p $workingDir/scratch");
+    createPatchFiles();
+    long start;
+    long elapsedTime;
+    start = System.currentTimeMillis();
+    //TODO give a proper label to the build
+    containerClient.defineImage(getLocalScratchDir(), null);
+    execLocally(getDockerBuildCommand());
+    elapsedTime = TimeUnit.MINUTES.convert((System.currentTimeMillis() - start),
+        TimeUnit.MILLISECONDS);
+    logger.info("PERF: Docker source prep took " + elapsedTime + " minutes");
+  }
+
+  @VisibleForTesting
+  String getLocalScratchDir() {
+    return mScratchDir.getAbsolutePath();
+  }
+
+  @VisibleForTesting
+  public String getDockerBuildCommand() throws Exception {
+    return containerClient.getBuildCommand(getLocalScratchDir(), 30, TimeUnit.MINUTES);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/41928126/testutils/ptest2/src/main/resources/dockerfile-template.vm
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/resources/dockerfile-template.vm b/testutils/ptest2/src/main/resources/dockerfile-template.vm
new file mode 100644
index 0000000..3652720
--- /dev/null
+++ b/testutils/ptest2/src/main/resources/dockerfile-template.vm
@@ -0,0 +1,33 @@
+FROM centos
+
+RUN yum upgrade -y && \
+    yum update -y && \
+    yum install -y java-1.8.0-openjdk-devel unzip git maven
+
+RUN useradd -m ptestuser
+RUN mkdir -p /home/ptestuser
+COPY smart-apply-patch.sh scratch/build.patch /home/ptestuser/scratch/
+RUN chown -R ptestuser /home/ptestuser
+RUN chmod -R 755 /home/ptestuser
+RUN ls -l /home
+USER ptestuser
+RUN cd /home/ptestuser
+WORKDIR /home/ptestuser
+ARG workingDir
+ARG mavenEnvOpts
+ARG repository
+ARG branch
+ARG label
+
+RUN export MAVEN_OPTS="$mavenEnvOpts"
+RUN /usr/bin/git clone $repository
+RUN cd hive
+WORKDIR /home/ptestuser/hive
+RUN /usr/bin/git checkout $branch || checkout -b $branch origin/$branch
+RUN /usr/bin/git reset --hard origin/$branch
+RUN /usr/bin/git merge --ff-only origin/$branch
+RUN /home/ptestuser/scratch/smart-apply-patch.sh /home/ptestuser/scratch/build.patch
+RUN /usr/bin/mvn -B -T 4 -q install -Dtest=TestMetastoreConf
+RUN cd itests
+RUN /usr/bin/mvn -B -T 4 -q  install -DskipSparkTests -DskipTests
+RUN echo This build is labeled $label

http://git-wip-us.apache.org/repos/asf/hive/blob/41928126/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/AbstractTestPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/AbstractTestPhase.java b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/AbstractTestPhase.java
index 71e1a64..01c65a6 100644
--- a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/AbstractTestPhase.java
+++ b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/AbstractTestPhase.java
@@ -58,6 +58,7 @@ public abstract class AbstractTestPhase {
   protected static final String BRANCH = "branch";
   protected static final String REPOSITORY = "repository";
   protected static final String REPOSITORY_NAME = "repositoryName";
+  protected static final String MAVEN_OPTS = "-Xmx128m";
 
   protected Host host;
   protected File baseDir;
@@ -100,6 +101,7 @@ public abstract class AbstractTestPhase {
         .put("logDir", logDir.getAbsolutePath())
         .put("repository", REPOSITORY)
         .put("repositoryName", REPOSITORY_NAME)
+        .put("mavenEnvOpts", MAVEN_OPTS)
         .build();
     host = new Host(HOST, USER, new String[] { LOCAL_DIR }, 2);
   }
@@ -115,7 +117,7 @@ public abstract class AbstractTestPhase {
     String osName = System.getProperty("os.name");
     return osName.contains("OS X");
   }
-  static File createBaseDir(String name) throws IOException {
+  public static File createBaseDir(String name) throws IOException {
     File baseDir;
     if(isOSX()) {
       // else osx gives ugly temp path which screws up approvals

http://git-wip-us.apache.org/repos/asf/hive/blob/41928126/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPtestOnDockers.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPtestOnDockers.java b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPtestOnDockers.java
new file mode 100644
index 0000000..516a12e
--- /dev/null
+++ b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPtestOnDockers.java
@@ -0,0 +1,161 @@
+/*
+ *
+ *  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.hive.ptest.execution;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.commons.io.FileUtils;
+import org.apache.hive.ptest.execution.LocalCommand.CollectLogPolicy;
+import org.apache.hive.ptest.execution.conf.Host;
+import org.apache.hive.ptest.execution.containers.DockerPrepPhase;
+import org.apache.hive.ptest.execution.containers.TestDockerPrepPhase;
+import org.apache.hive.ptest.execution.context.ExecutionContext;
+import org.apache.hive.ptest.execution.ssh.NonZeroExitCodeException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.concurrent.Executors;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+
+/**
+ * This test assumes that the host where this test is being run has docker installed
+ */
+public class TestPtestOnDockers {
+  //TODO add logic to ignore this test if docker is not found on the machine
+
+  private DockerPrepPhase phase;
+  private static File dummyPatchFile;
+  private static final Logger logger = LoggerFactory.getLogger(TestDockerPrepPhase.class);
+
+  private File baseDir;
+  private File scratchDir;
+  private File logDir;
+  private File succeededLogDir;
+  private File failedLogDir;
+  private ListeningExecutorService executor;
+  private LocalCommandFactory localCommandFactory;
+  private LocalCommand localCommand;
+  private ImmutableMap<String, String> templateDefaults;
+  private ImmutableList<HostExecutor> hostExecutors;
+  private HostExecutor hostExecutor;
+  private ExecutionContext executionContext;
+  private HostExecutorBuilder hostExecutorBuilder;
+  private Host host;
+
+  private static final String LOCAL_DIR = "/some/local/dir";
+  private static final String PRIVATE_KEY = "some.private.key";
+  private static final String USER = "someuser";
+  private static final String HOST = "somehost";
+  private static final int INSTANCE = 13;
+  private static final String INSTANCE_NAME = HOST + "-" + USER + "-" + INSTANCE;
+  private static final String REAL_BRANCH = "master";
+  private static final String REAL_REPOSITORY = "https://git-wip-us.apache.org/repos/asf/hive.git";
+  private static final String REAL_REPOSITORY_NAME = "apache-hive";
+  private static final String REAL_MAVEN_OPTS = "-Xmx1g";
+  private MockSSHCommandExecutor sshCommandExecutor;
+  private MockRSyncCommandExecutor rsyncCommandExecutor;
+
+  public void initialize(String name) throws Exception {
+    baseDir = AbstractTestPhase.createBaseDir(name);
+    logDir = Dirs.create(new File(baseDir, "logs"));
+    scratchDir = Dirs.create(new File(baseDir, "scratch"));
+    succeededLogDir = Dirs.create(new File(logDir, "succeeded"));
+    failedLogDir = Dirs.create(new File(logDir, "failed"));
+    executor = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(2));
+    executionContext = mock(ExecutionContext.class);
+    hostExecutorBuilder = mock(HostExecutorBuilder.class);
+    //use real localCommandFactory
+    localCommandFactory = new LocalCommandFactory(logger);
+    sshCommandExecutor = spy(new MockSSHCommandExecutor(logger));
+    rsyncCommandExecutor = spy(new MockRSyncCommandExecutor(logger));
+    templateDefaults = ImmutableMap.<String, String>builder()
+        .put("localDir", LOCAL_DIR)
+        //use baseDir as working directory
+        .put("workingDir", baseDir.getAbsolutePath())
+        .put("instanceName", INSTANCE_NAME)
+        .put("branch", REAL_BRANCH)
+        .put("logDir", logDir.getAbsolutePath())
+        .put("repository", REAL_REPOSITORY)
+        .put("repositoryName", REAL_REPOSITORY_NAME)
+        .put("mavenEnvOpts", REAL_MAVEN_OPTS)
+        .build();
+    host = new Host(HOST, USER, new String[] { LOCAL_DIR }, 2);
+  }
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    dummyPatchFile = File.createTempFile("dummy", "patch");
+    dummyPatchFile.deleteOnExit();
+    Assert.assertTrue("Could not create dummy patch file " + dummyPatchFile.getAbsolutePath(),
+        dummyPatchFile.exists());
+  }
+
+  @Before
+  public void setup() throws Exception {
+    initialize(getClass().getSimpleName());
+    createHostExecutor();
+    phase = new DockerPrepPhase(hostExecutors, localCommandFactory,
+        templateDefaults, baseDir, dummyPatchFile, logger);
+  }
+
+  private void createHostExecutor() {
+    hostExecutor = new HostExecutor(host, PRIVATE_KEY, executor, sshCommandExecutor,
+        rsyncCommandExecutor, templateDefaults, scratchDir, succeededLogDir, failedLogDir, 1, true,
+        logger);
+    hostExecutors = ImmutableList.of(hostExecutor);
+  }
+
+  @After
+  public void teardown() {
+    phase = null;
+    FileUtils.deleteQuietly(baseDir);
+  }
+
+  /**
+   * This test requires docker to be installed to test on local machine
+   * @throws Exception
+   */
+  @Test
+  public void testDockerFile() throws Exception {
+    phase.execute();
+    File dockerFile = new File(scratchDir, "Dockerfile");
+    Assert.assertTrue("Docker file was not found" , dockerFile.exists());
+    Assert.assertTrue("patch file not found", new File(scratchDir, "build.patch").exists());
+    LocalCommandFactory localCommandFactory = new LocalCommandFactory(logger);
+    CollectLogPolicy localCollector = new CollectLogPolicy(logger);
+    localCommandFactory.create(localCollector, "cp -f " + dockerFile.getAbsolutePath() + " /tmp/myDockerFile");
+    LocalCommand localCmd = localCommandFactory.create(localCollector, phase.getDockerBuildCommand());
+    if(localCmd.getExitCode() != 0) {
+      throw new NonZeroExitCodeException(String.format(
+          "Command '%s' failed with exit status %d and output '%s'",
+          localCmd, localCmd.getExitCode(), localCollector.getOutput()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/41928126/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/containers/TestDockerPrepPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/containers/TestDockerPrepPhase.java b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/containers/TestDockerPrepPhase.java
new file mode 100644
index 0000000..8efc5e0
--- /dev/null
+++ b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/containers/TestDockerPrepPhase.java
@@ -0,0 +1,73 @@
+/*
+ *
+ *  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.hive.ptest.execution.containers;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hive.ptest.execution.AbstractTestPhase;
+
+import org.apache.hive.ptest.execution.LocalCommand;
+import org.apache.hive.ptest.execution.LocalCommand.CollectLogPolicy;
+import org.apache.hive.ptest.execution.LocalCommandFactory;
+import org.apache.hive.ptest.execution.ssh.NonZeroExitCodeException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+
+public class TestDockerPrepPhase extends AbstractTestPhase {
+  private DockerPrepPhase phase;
+  private static File dummyPatchFile;
+  private static final Logger logger = LoggerFactory.getLogger(TestDockerPrepPhase.class);
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    dummyPatchFile = File.createTempFile("dummy", "patch");
+    dummyPatchFile.deleteOnExit();
+    Assert.assertTrue("Could not create dummy patch file " + dummyPatchFile.getAbsolutePath(),
+        dummyPatchFile.exists());
+  }
+
+  @Before
+  public void setup() throws Exception {
+    initialize(getClass().getSimpleName());
+    createHostExecutor();
+    phase = new DockerPrepPhase(hostExecutors, localCommandFactory,
+        templateDefaults, baseDir, dummyPatchFile, logger);
+  }
+
+  @After
+  public void teardown() {
+    phase = null;
+    FileUtils.deleteQuietly(baseDir);
+  }
+
+  @Test
+  public void testExecute() throws Exception {
+    phase.execute();
+    Assert.assertNotNull("Scratch directory needs to be set", phase.getLocalScratchDir());
+    File dockerFile = new File(phase.getLocalScratchDir(), "Dockerfile");
+    Assert.assertTrue("Docker file not found", dockerFile.exists());
+  }
+}