Return-Path: X-Original-To: apmail-hive-commits-archive@www.apache.org Delivered-To: apmail-hive-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 1E45F10725 for ; Fri, 16 Aug 2013 01:32:29 +0000 (UTC) Received: (qmail 82308 invoked by uid 500); 16 Aug 2013 01:27:50 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 81578 invoked by uid 500); 16 Aug 2013 01:27:12 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 80148 invoked by uid 99); 16 Aug 2013 01:23:43 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 16 Aug 2013 01:23:43 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 16 Aug 2013 01:23:34 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 8533E2388C3D; Fri, 16 Aug 2013 01:22:20 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1514554 [18/18] - in /hive/branches/vectorization: ./ beeline/src/java/org/apache/hive/beeline/ cli/src/java/org/apache/hadoop/hive/cli/ cli/src/test/org/apache/hadoop/hive/cli/ common/src/java/org/apache/hadoop/hive/conf/ conf/ contrib/sr... Date: Fri, 16 Aug 2013 01:22:02 -0000 To: commits@hive.apache.org From: hashutosh@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20130816012220.8533E2388C3D@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Modified: hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PrepPhase.java Fri Aug 16 01:21:54 2013 @@ -21,18 +21,18 @@ package org.apache.hive.ptest.execution; import java.io.File; import java.io.IOException; import java.io.PrintWriter; +import java.util.List; import java.util.concurrent.TimeUnit; import org.slf4j.Logger; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; public class PrepPhase extends Phase { private final File mScratchDir; private final File mPatchFile; - public PrepPhase(ImmutableList hostExecutors, + public PrepPhase(List hostExecutors, LocalCommandFactory localCommandFactory, ImmutableMap templateDefaults, File scratchDir, File patchFile, Logger logger) { @@ -41,11 +41,9 @@ public class PrepPhase extends Phase { this.mPatchFile = patchFile; } @Override -public void execute() throws Exception { - long prepStart = System.currentTimeMillis(); + public void execute() throws Exception { + execLocally("rm -rf $workingDir/scratch"); execLocally("mkdir -p $workingDir/scratch"); - execInstances("mkdir -p $localDir/$instanceName/logs $localDir/$instanceName/maven $localDir/$instanceName/scratch"); - execInstances("mkdir -p $localDir/$instanceName/ivy $localDir/$instanceName/${repositoryName}-source"); if(mPatchFile != null) { File smartApplyPatch = new File(mScratchDir, "smart-apply-patch.sh"); PrintWriter writer = new PrintWriter(smartApplyPatch); @@ -61,7 +59,7 @@ public void execute() throws Exception { } long start; long elapsedTime; - // source prep + // source prep start = System.currentTimeMillis(); File sourcePrepScript = new File(mScratchDir, "source-prep.sh"); Templates.writeTemplateResult("source-prep.vm", sourcePrepScript, getTemplateDefaults()); @@ -70,24 +68,5 @@ public void execute() throws Exception { elapsedTime = TimeUnit.MINUTES.convert((System.currentTimeMillis() - start), TimeUnit.MILLISECONDS); logger.info("PERF: source prep took " + elapsedTime + " minutes"); - // rsync source - start = System.currentTimeMillis(); - rsyncFromLocalToRemoteInstances("$workingDir/${repositoryName}-source", "$localDir/$instanceName/"); - elapsedTime = TimeUnit.MINUTES.convert((System.currentTimeMillis() - start), - TimeUnit.MILLISECONDS); - logger.info("PERF: rsync source took " + elapsedTime + " minutes"); - start = System.currentTimeMillis(); - rsyncFromLocalToRemoteInstances("$workingDir/maven", "$localDir/$instanceName/"); - elapsedTime = TimeUnit.MINUTES.convert((System.currentTimeMillis() - start), - TimeUnit.MILLISECONDS); - logger.info("PERF: rsync maven took " + elapsedTime + " minutes"); - start = System.currentTimeMillis(); - rsyncFromLocalToRemoteInstances("$workingDir/ivy", "$localDir/$instanceName/"); - elapsedTime = TimeUnit.MINUTES.convert((System.currentTimeMillis() - start), - TimeUnit.MILLISECONDS); - logger.info("PERF: rsync ivy took " + elapsedTime + " minutes"); - elapsedTime = TimeUnit.MINUTES.convert((System.currentTimeMillis() - prepStart), - TimeUnit.MILLISECONDS); - logger.info("PERF: prep phase took " + elapsedTime + " minutes"); } -} \ No newline at end of file +} Modified: hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ReportingPhase.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ReportingPhase.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ReportingPhase.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ReportingPhase.java Fri Aug 16 01:21:54 2013 @@ -18,9 +18,10 @@ */ package org.apache.hive.ptest.execution; +import java.util.List; + import org.slf4j.Logger; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; /** @@ -28,13 +29,13 @@ import com.google.common.collect.Immutab */ public class ReportingPhase extends Phase { - public ReportingPhase(ImmutableList hostExecutors, + public ReportingPhase(List hostExecutors, LocalCommandFactory localCommandFactory, ImmutableMap templateDefaults, Logger logger) { super(hostExecutors, localCommandFactory, templateDefaults, logger); } @Override -public void execute() throws Exception { + public void execute() throws Exception { execLocally("mkdir $logDir/test-results"); execLocally("find $logDir/{failed,succeeded} -maxdepth 2 -name 'TEST*.xml' -exec cp {} $logDir/test-results \\; 2>/dev/null"); execLocally("cd $logDir/ && tar -zvcf test-results.tar.gz test-results/"); Modified: hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/ExecutionContextConfiguration.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/ExecutionContextConfiguration.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/ExecutionContextConfiguration.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/ExecutionContextConfiguration.java Fri Aug 16 01:21:54 2013 @@ -42,7 +42,7 @@ public class ExecutionContextConfigurati private final String mGlobalLogDirectory; private final String mProfileDirectory; private final int mMaxLogDirectoriesPerProfile; - + @VisibleForTesting public ExecutionContextConfiguration(Context context) throws IOException { Modified: hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java Fri Aug 16 01:21:54 2013 @@ -110,5 +110,5 @@ public class QFileTestBatch implements T } else if (!tests.equals(other.tests)) return false; return true; - } + } } Modified: hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java Fri Aug 16 01:21:54 2013 @@ -100,8 +100,8 @@ public class TestConfiguration { this.jiraName = Strings.nullToEmpty(jiraName); } public boolean isClearLibraryCache() { - return clearLibraryCache; - } + return clearLibraryCache; + } public void setClearLibraryCache(boolean clearLibraryCache) { this.clearLibraryCache = clearLibraryCache; } Modified: hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java Fri Aug 16 01:21:54 2013 @@ -54,7 +54,7 @@ public class TestParser { Set included = Sets.newHashSet(splitter.split(unitContext.getString("include", ""))); if(!included.isEmpty() && !excluded.isEmpty()) { throw new IllegalArgumentException(String.format("Included and excluded mutally exclusive." + - " Included = %s, excluded = %s", included.toString(), excluded.toString())); + " Included = %s, excluded = %s", included.toString(), excluded.toString())); } List unitTestsDirs = Lists.newArrayList(); for(String unitTestDir : Splitter.on(" ").omitEmptyStrings() Modified: hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudComputeService.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudComputeService.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudComputeService.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudComputeService.java Fri Aug 16 01:21:54 2013 @@ -18,6 +18,8 @@ */ package org.apache.hive.ptest.execution.context; +import java.util.Collections; +import java.util.Properties; import java.util.Set; import org.jclouds.ContextBuilder; @@ -25,11 +27,11 @@ import org.jclouds.aws.ec2.compute.AWSEC import org.jclouds.compute.ComputeService; import org.jclouds.compute.ComputeServiceContext; import org.jclouds.compute.RunNodesException; +import org.jclouds.compute.config.ComputeServiceProperties; import org.jclouds.compute.domain.ComputeMetadata; import org.jclouds.compute.domain.NodeMetadata; import org.jclouds.compute.domain.NodeMetadata.Status; import org.jclouds.compute.domain.Template; -import org.jclouds.ec2.domain.InstanceType; import org.jclouds.logging.log4j.config.Log4JLoggingModule; import com.google.common.base.Predicate; @@ -39,18 +41,25 @@ import com.google.common.collect.Sets; public class CloudComputeService { private final ComputeServiceContext mComputeServiceContext; private final ComputeService mComputeService; + private final String mInstanceType; private final String mGroupName; + private final String mGroupTag; private final String mImageId; private final String mkeyPair; private final String mSecurityGroup; private final float mMaxBid; - public CloudComputeService(String apiKey, String accessKey, String groupName, + public CloudComputeService(String apiKey, String accessKey, String instanceType, String groupName, String imageId, String keyPair, String securityGroup, float maxBid) { + mInstanceType = instanceType; mGroupName = groupName; mImageId = imageId; mkeyPair = keyPair; mSecurityGroup = securityGroup; mMaxBid = maxBid; + mGroupTag = "group=" + mGroupName; + Properties overrides = new Properties(); + overrides.put(ComputeServiceProperties.POLL_INITIAL_PERIOD, String.valueOf(10L * 1000L)); + overrides.put(ComputeServiceProperties.POLL_MAX_PERIOD, String.valueOf(30L * 1000L)); mComputeServiceContext = ContextBuilder.newBuilder("aws-ec2") .credentials(apiKey, accessKey) .modules(ImmutableSet.of(new Log4JLoggingModule())) @@ -58,13 +67,13 @@ public class CloudComputeService { mComputeService = mComputeServiceContext.getComputeService(); } public Set createNodes(int count) - throws RunNodesException { + throws RunNodesException { Set result = Sets.newHashSet(); Template template = mComputeService.templateBuilder() - .hardwareId(InstanceType.M1_XLARGE).imageId(mImageId).build(); + .hardwareId(mInstanceType).imageId(mImageId).build(); template.getOptions().as(AWSEC2TemplateOptions.class).keyPair(mkeyPair) - .securityGroupIds(mSecurityGroup).blockOnPort(22, 60) - .spotPrice(mMaxBid); + .securityGroupIds(mSecurityGroup).blockOnPort(22, 60) + .spotPrice(mMaxBid).tags(Collections.singletonList(mGroupTag)); result.addAll(mComputeService.createNodesInGroup(mGroupName, count, template)); return result; } @@ -76,7 +85,8 @@ public class CloudComputeService { public boolean apply(ComputeMetadata computeMetadata) { NodeMetadata nodeMetadata = (NodeMetadata) computeMetadata; return nodeMetadata.getStatus() == Status.RUNNING - && mGroupName.equalsIgnoreCase(nodeMetadata.getGroup()); + && (mGroupName.equalsIgnoreCase(nodeMetadata.getGroup()) || + nodeMetadata.getTags().contains(mGroupTag)); } })); return result; Modified: hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/context/CloudExecutionContextProvider.java Fri Aug 16 01:21:54 2013 @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.RandomAccessFile; import java.util.Collections; import java.util.Date; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; @@ -64,7 +65,9 @@ public class CloudExecutionContextProvid public static final String MAX_BID = "maxBid"; public static final String SLAVE_LOCAL_DIRECTORIES = "localDirs"; public static final String USERNAME = "user"; + public static final String INSTANCE_TYPE = "instanceType"; public static final String NUM_THREADS = "numThreads"; + private final RandomAccessFile mHostLog; private final String mPrivateKey; private final String mUser; @@ -122,7 +125,7 @@ public class CloudExecutionContextProvid mHostLog.setLength(0); Thread thread = new Thread() { @Override - public void run() { + public void run() { while (true) { try { TimeUnit.MINUTES.sleep(15); @@ -168,13 +171,12 @@ public class CloudExecutionContextProvid } } Preconditions.checkState(hostsNotRemoved.isEmpty(), - "Host " + hostsNotRemoved + " was in bad hosts but could be removed"); + "Host " + hostsNotRemoved + " was in bad hosts but could not be removed"); } @Override public synchronized ExecutionContext createExecutionContext() throws CreateHostsFailedException, ServiceNotAvailableException { - boolean error = true; try { Set nodes = createNodes(mNumHosts); Set hosts = Sets.newHashSet(); @@ -182,18 +184,15 @@ public class CloudExecutionContextProvid hosts.add(new Host(node.getHostname(), mUser, mSlaveLocalDirs, mNumThreads)); } - error = false; return new ExecutionContext(this, hosts, mWorkingDir.getAbsolutePath(), mPrivateKey); } finally { - if (!error) { - syncLog(); - } + syncLog(); } } private Set createNodes(int numHosts) - throws CreateHostsFailedException { + throws CreateHostsFailedException { Set result = Sets.newHashSet(); int attempts = 0; int numRequired = numHosts; @@ -205,10 +204,9 @@ public class CloudExecutionContextProvid } catch (RunNodesException e) { exception = e; LOG.warn("Error creating nodes", e); - persistHostnamesToLog(e.getNodeErrors().keySet()); + terminateInternal(e.getNodeErrors().keySet()); result.addAll(verifyHosts(e.getSuccessfulNodes())); } - persistHostnamesToLog(result); // ok to persist more than once LOG.info("Successfully created " + result.size() + " nodes"); numRequired = numHosts - result.size(); if(numRequired > 0) { @@ -240,16 +238,41 @@ public class CloudExecutionContextProvid } } - private Set verifyHosts(Set hosts) { - Set result = Sets.newHashSet(); - for(NodeMetadata node : hosts) { - SSHCommand command = new SSHCommand(mSSHCommandExecutor, mPrivateKey, mUser, node.getHostname(), 0, "pkill -f java"); - mSSHCommandExecutor.execute(command); - if(command.getExitCode() == Constants.EXIT_CODE_UNKNOWN) { - LOG.error("Node " + node.getHostname() + " is bad on startup"); - terminateInternal(node); - } else { - result.add(node); + private Set verifyHosts(Set hosts) + throws CreateHostsFailedException { + persistHostnamesToLog(hosts); + final Set result = Collections.synchronizedSet(new HashSet()); + ExecutorService executorService = Executors.newFixedThreadPool(Math.min(hosts.size(), 25)); + try { + for(final NodeMetadata node : hosts) { + executorService.submit(new Runnable() { + @Override + public void run() { + SSHCommand command = new SSHCommand(mSSHCommandExecutor, mPrivateKey, mUser, node.getHostname(), 0, "pkill -f java"); + mSSHCommandExecutor.execute(command); + if(command.getExitCode() == Constants.EXIT_CODE_UNKNOWN || + command.getException() != null) { + if(command.getException() == null) { + LOG.error("Node " + node.getHostname() + " is bad on startup"); + } else { + LOG.error("Node " + node.getHostname() + " is bad on startup", command.getException()); + } + terminateInternal(node); + } else { + result.add(node); + } + } + }); + } + executorService.shutdown(); + if(!executorService.awaitTermination(10, TimeUnit.MINUTES)) { + LOG.error("Verify command still executing on a host after 10 minutes"); + } + } catch (InterruptedException e) { + throw new CreateHostsFailedException("Interrupted while trying to create hosts", e); + } finally { + if(!executorService.isShutdown()) { + executorService.shutdownNow(); } } return result; @@ -376,11 +399,12 @@ public class CloudExecutionContextProvid .split(context.getString(SLAVE_LOCAL_DIRECTORIES, "/home/hiveptest/")), String.class); Integer numThreads = context.getInteger(NUM_THREADS, 3); + String instanceType = context.getString(INSTANCE_TYPE, "c1.xlarge"); CloudComputeService cloudComputeService = new CloudComputeService(apiKey, accessKey, - groupName, imageId, keyPair, securityGroup, maxBid); + instanceType, groupName, imageId, keyPair, securityGroup, maxBid); CloudExecutionContextProvider service = new CloudExecutionContextProvider( dataDir, numHosts, cloudComputeService, new SSHCommandExecutor(LOG), workingDirectory, privateKey, user, localDirs, numThreads, 10, 10); return service; } -} \ No newline at end of file +} Modified: hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/AbstractSSHCommand.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/AbstractSSHCommand.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/AbstractSSHCommand.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/AbstractSSHCommand.java Fri Aug 16 01:21:54 2013 @@ -21,50 +21,50 @@ package org.apache.hive.ptest.execution. import java.util.concurrent.Callable; public abstract class AbstractSSHCommand implements Callable { - private final String privateKey; - private final String user; - private final String host; - private final int instance; - private int exitCode = -1; - private Exception exception; - private String output; + private final String privateKey; + private final String user; + private final String host; + private final int instance; + private int exitCode = -1; + private Exception exception; + private String output; - public AbstractSSHCommand(String privateKey, - String user, String host, int instance) { - this.privateKey = privateKey; - this.user = user; - this.host = host; - this.instance = instance; - } + public AbstractSSHCommand(String privateKey, + String user, String host, int instance) { + this.privateKey = privateKey; + this.user = user; + this.host = host; + this.instance = instance; + } - public void setException(Exception exception) { - this.exception = exception; - } - public void setExitCode(int exitCode) { - this.exitCode = exitCode; - } - public void setOutput(String output) { - this.output = output; - } - public int getExitCode() { - return exitCode; - } - public Exception getException() { - return exception; - } - public String getOutput() { - return output; - } - public String getPrivateKey() { - return privateKey; - } - public String getUser() { - return user; - } - public String getHost() { - return host; - } - public int getInstance() { - return instance; - } + public void setException(Exception exception) { + this.exception = exception; + } + public void setExitCode(int exitCode) { + this.exitCode = exitCode; + } + public void setOutput(String output) { + this.output = output; + } + public int getExitCode() { + return exitCode; + } + public Exception getException() { + return exception; + } + public String getOutput() { + return output; + } + public String getPrivateKey() { + return privateKey; + } + public String getUser() { + return user; + } + public String getHost() { + return host; + } + public int getInstance() { + return instance; + } } Modified: hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/RSyncCommandExecutor.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/RSyncCommandExecutor.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/RSyncCommandExecutor.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/RSyncCommandExecutor.java Fri Aug 16 01:21:54 2013 @@ -53,12 +53,12 @@ public class RSyncCommandExecutor { retry = false; if(command.getType() == RSyncCommand.Type.TO_LOCAL) { cmd = new LocalCommand(mLogger, collector, - String.format("rsync -qaPe \"ssh -i %s\" %s@%s:%s %s", + String.format("timeout 1h rsync -qaPe \"ssh -i %s\" --timeout 600 %s@%s:%s %s", command.getPrivateKey(), command.getUser(), command.getHost(), command.getRemoteFile(), command.getLocalFile())); } else if(command.getType() == RSyncCommand.Type.FROM_LOCAL) { cmd = new LocalCommand(mLogger, collector, - String.format("rsync -qaPe \"ssh -i %s\" --delete --delete-during --force %s %s@%s:%s", + String.format("timeout 1h rsync -qaPe \"ssh -i %s\" --timeout 600 --delete --delete-during --force %s %s@%s:%s", command.getPrivateKey(), command.getLocalFile(), command.getUser(), command.getHost(), command.getRemoteFile())); } else { Modified: hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommandExecutor.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommandExecutor.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommandExecutor.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ssh/SSHCommandExecutor.java Fri Aug 16 01:21:54 2013 @@ -50,11 +50,14 @@ public class SSHCommandExecutor { if(attempts++ <= 3 && cmd.getExitCode() == Constants.EXIT_CODE_UNKNOWN) { mLogger.warn("Command exited with " + cmd.getExitCode() + ", will retry: " + command); retry = true; - TimeUnit.SECONDS.sleep(20); + TimeUnit.SECONDS.sleep(5); } } while (retry); // an error occurred, re-try command.setExitCode(cmd.getExitCode()); } catch (Exception e) { + if(command.getExitCode() == Constants.EXIT_CODE_SUCCESS) { + command.setExitCode(Constants.EXIT_CODE_EXCEPTION); + } command.setException(e); } finally { command.setOutput(collector.getOutput()); Modified: hive/branches/vectorization/testutils/ptest2/src/main/resources/batch-exec.vm URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/resources/batch-exec.vm?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/resources/batch-exec.vm (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/resources/batch-exec.vm Fri Aug 16 01:21:54 2013 @@ -40,7 +40,7 @@ cd $localDir/$instanceName/${repositoryN ret=$? if [[ $ret -ne 0 ]] then - if [[ $numOfFailedTests -lt 5 ]] + if [[ $numOfFailedTests -lt $maxSourceDirs ]] then cp -R $localDir/$instanceName/${repositoryName}-source $logDir/source else Modified: hive/branches/vectorization/testutils/ptest2/src/main/resources/log4j.properties URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/resources/log4j.properties?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/resources/log4j.properties (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/resources/log4j.properties Fri Aug 16 01:21:54 2013 @@ -20,7 +20,7 @@ log4j.threshhold=ALL log4j.appender.FILE=org.apache.log4j.RollingFileAppender log4j.appender.FILE.File=${hive.ptest.logdir}/ptest.log -log4j.appender.FILE.MaxFileSize=10MB +log4j.appender.FILE.MaxFileSize=50MB log4j.appender.FILE.MaxBackupIndex=1 log4j.appender.FILE.layout=org.apache.log4j.PatternLayout log4j.appender.FILE.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n @@ -29,6 +29,5 @@ log4j.logger.org.apache.http=INFO log4j.logger.org.springframework=INFO log4j.logger.org.jclouds=INFO log4j.logger.jclouds=INFO -log4j.logger.jclouds.compute=TRACE log4j.logger.org.apache.hive=DEBUG -log4j.logger.org.apache.http=TRACE \ No newline at end of file +log4j.logger.org.apache.http=TRACE Modified: hive/branches/vectorization/testutils/ptest2/src/main/resources/source-prep.vm URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/main/resources/source-prep.vm?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/main/resources/source-prep.vm (original) +++ hive/branches/vectorization/testutils/ptest2/src/main/resources/source-prep.vm Fri Aug 16 01:21:54 2013 @@ -79,10 +79,9 @@ cd $workingDir/ fi if [[ "$clearLibraryCache" == "true" ]] then - rm -rf $workingDir/ivy $workingDir/maven - mkdir $workingDir/ivy $workingDir/maven + rm -rf $workingDir/ivy $workingDir/maven + mkdir $workingDir/ivy $workingDir/maven fi - ant $antArgs -Divy.default.ivy.user.dir=$workingDir/ivy -Dmvn.local.repo=$workingDir/maven clean package - ant $antArgs -Divy.default.ivy.user.dir=$workingDir/ivy -Dmvn.local.repo=$workingDir/maven -Dtestcase=nothing test + ant $antArgs -Divy.default.ivy.user.dir=$workingDir/ivy -Dmvn.local.repo=$workingDir/maven clean package test -Dtestcase=nothing ) 2>&1 | tee $logDir/source-prep.txt -exit ${PIPESTATUS[0]} +exit ${PIPESTATUS[0]} \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/api/server/TestTestExecutor.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/api/server/TestTestExecutor.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/api/server/TestTestExecutor.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/api/server/TestTestExecutor.java Fri Aug 16 01:21:54 2013 @@ -92,7 +92,7 @@ public class TestTestExecutor { profilePropertiesOutputStream.close(); ptestBuilder = new PTest.Builder() { @Override - public PTest build(TestConfiguration configuration, ExecutionContext executionContext, + public PTest build(TestConfiguration configuration, ExecutionContext executionContext, String buildTag, File logDir, LocalCommandFactory localCommandFactory, SSHCommandExecutor sshCommandExecutor, RSyncCommandExecutor rsyncCommandExecutor, Logger logger) throws Exception { return ptest; Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/AbstractTestPhase.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/AbstractTestPhase.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/AbstractTestPhase.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/AbstractTestPhase.java Fri Aug 16 01:21:54 2013 @@ -29,7 +29,9 @@ import java.util.concurrent.Executors; import junit.framework.Assert; +import org.apache.hive.ptest.api.server.TestLogger; import org.apache.hive.ptest.execution.conf.Host; +import org.apache.hive.ptest.execution.context.ExecutionContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,6 +73,8 @@ public abstract class AbstractTestPhase protected ImmutableMap templateDefaults; protected ImmutableList hostExecutors; protected HostExecutor hostExecutor; + protected ExecutionContext executionContext; + protected HostExecutorBuilder hostExecutorBuilder; protected Logger logger; public void initialize(String name) throws Exception { @@ -80,12 +84,14 @@ public abstract class AbstractTestPhase 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); localCommandFactory = new MockLocalCommandFactory(LOG); localCommand = mock(LocalCommand.class); localCommandFactory.setInstance(localCommand); sshCommandExecutor = spy(new MockSSHCommandExecutor(LOG)); rsyncCommandExecutor = spy(new MockRSyncCommandExecutor(LOG)); - logger = LOG; + logger = new TestLogger(System.err, TestLogger.LEVEL.TRACE); templateDefaults = ImmutableMap.builder() .put("localDir", LOCAL_DIR) .put("workingDir", WORKING_DIR) Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockLocalCommandFactory.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockLocalCommandFactory.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockLocalCommandFactory.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockLocalCommandFactory.java Fri Aug 16 01:21:54 2013 @@ -39,7 +39,7 @@ public class MockLocalCommandFactory ext return commands; } @Override -public LocalCommand create(LocalCommand.CollectPolicy policy, String command) + public LocalCommand create(LocalCommand.CollectPolicy policy, String command) throws IOException { commands.add(command); return this.instance; Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockRSyncCommandExecutor.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockRSyncCommandExecutor.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockRSyncCommandExecutor.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockRSyncCommandExecutor.java Fri Aug 16 01:21:54 2013 @@ -62,7 +62,7 @@ public class MockRSyncCommandExecutor ex if(queue == null || queue.isEmpty()) { command.setExitCode(0); } else { - command.setExitCode(queue.remove()); + command.setExitCode(queue.remove()); } } Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockSSHCommandExecutor.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockSSHCommandExecutor.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockSSHCommandExecutor.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockSSHCommandExecutor.java Fri Aug 16 01:21:54 2013 @@ -38,7 +38,7 @@ public class MockSSHCommandExecutor exte mCommands = Lists.newArrayList(); mFailures = Maps.newHashMap(); } - public List getCommands() { + public synchronized List getCommands() { return mCommands; } public synchronized void putFailure(String command, Integer... exitCodes) { @@ -54,14 +54,14 @@ public class MockSSHCommandExecutor exte } } @Override -public synchronized void execute(SSHCommand command) { + public synchronized void execute(SSHCommand command) { mCommands.add(command.getCommand()); command.setOutput(""); Queue queue = mFailures.get(command.getCommand()); if(queue == null || queue.isEmpty()) { command.setExitCode(0); } else { - command.setExitCode(queue.remove()); + command.setExitCode(queue.remove()); } } } Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestCleanupPhase.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestCleanupPhase.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestCleanupPhase.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestCleanupPhase.java Fri Aug 16 01:21:54 2013 @@ -1,43 +0,0 @@ -/* - * 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.approvaltests.Approvals; -import org.approvaltests.reporters.JunitReporter; -import org.approvaltests.reporters.UseReporter; -import org.junit.Before; -import org.junit.Test; - -@UseReporter(JunitReporter.class) -public class TestCleanupPhase extends AbstractTestPhase { - private CleanupPhase phase; - - @Before - public void setup() throws Exception { - initialize(getClass().getSimpleName()); - createHostExecutor(); - } - @Test - public void testExecute() throws Exception { - phase = new CleanupPhase(hostExecutors, localCommandFactory, - templateDefaults, logger); - phase.execute(); - Approvals.verify(getExecutedCommands()); - } -} Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestCleanupPhase.testExecute.approved.txt URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestCleanupPhase.testExecute.approved.txt?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestCleanupPhase.testExecute.approved.txt (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestCleanupPhase.testExecute.approved.txt Fri Aug 16 01:21:54 2013 @@ -1,4 +0,0 @@ -killall -q -9 -f java || true -rm -rf /some/local/dir/somehost-someuser-0/scratch /some/local/dir/somehost-someuser-0/logs -rm -rf /some/local/dir/somehost-someuser-1/scratch /some/local/dir/somehost-someuser-1/logs -rm -rf /some/working/dir/scratch \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java Fri Aug 16 01:21:54 2013 @@ -61,9 +61,9 @@ public class TestExecutionPhase extends } private ExecutionPhase getPhase() throws IOException { createHostExecutor(); - phase = new ExecutionPhase(hostExecutors, localCommandFactory, templateDefaults, - succeededLogDir, failedLogDir, Suppliers.ofInstance(testBatches), - executedTests, failedTests, logger); + phase = new ExecutionPhase(hostExecutors, executionContext, hostExecutorBuilder, + localCommandFactory, templateDefaults, succeededLogDir, failedLogDir, + Suppliers.ofInstance(testBatches), executedTests, failedTests, logger); return phase; } private void setupQFile(boolean isParallel) throws Exception { @@ -126,4 +126,4 @@ public class TestExecutionPhase extends Assert.assertEquals(Sets.newHashSet("SomeTest." + QFILENAME), executedTests); Assert.assertEquals(Sets.newHashSet("SomeTest." + QFILENAME), failedTests); } -} +} \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingQFile.approved.txt URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingQFile.approved.txt?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingQFile.approved.txt (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingQFile.approved.txt Fri Aug 16 01:21:54 2013 @@ -1,3 +1,14 @@ +/some/working/dir/ivy /some/local/dir/somehost-someuser-0 +/some/working/dir/maven /some/local/dir/somehost-someuser-0 +/some/working/dir/repositoryName-source /some/local/dir/somehost-someuser-0 /tmp/hive-ptest-units/TestExecutionPhase/logs/failed/driver-sometest /some/local/dir/somehost-someuser-0/logs/ /tmp/hive-ptest-units/TestExecutionPhase/scratch/hiveptest-driver-sometest.sh /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-sometest.sh -bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-sometest.sh \ No newline at end of file +bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-sometest.sh +killall -q -9 -f java || true +mkdir -p /some/local/dir/somehost-someuser-0/logs /some/local/dir/somehost-someuser-0/maven /some/local/dir/somehost-someuser-0/scratch /some/local/dir/somehost-someuser-0/ivy /some/local/dir/somehost-someuser-0/repositoryName-source +mkdir -p /some/local/dir/somehost-someuser-1/logs /some/local/dir/somehost-someuser-1/maven /some/local/dir/somehost-someuser-1/scratch /some/local/dir/somehost-someuser-1/ivy /some/local/dir/somehost-someuser-1/repositoryName-source +rm -rf /some/local/dir/somehost-someuser-0/scratch /some/local/dir/somehost-someuser-0/logs +rm -rf /some/local/dir/somehost-someuser-1/scratch /some/local/dir/somehost-someuser-1/logs +rsync -qaPe --delete --delete-during --force /some/local/dir/somehost-someuser-0/ivy /some/local/dir/somehost-someuser-1/ +rsync -qaPe --delete --delete-during --force /some/local/dir/somehost-someuser-0/maven /some/local/dir/somehost-someuser-1/ +rsync -qaPe --delete --delete-during --force /some/local/dir/somehost-someuser-0/repositoryName-source /some/local/dir/somehost-someuser-1/ \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingUnitTest.approved.txt URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingUnitTest.approved.txt?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingUnitTest.approved.txt (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingUnitTest.approved.txt Fri Aug 16 01:21:54 2013 @@ -1,3 +1,14 @@ +/some/working/dir/ivy /some/local/dir/somehost-someuser-0 +/some/working/dir/maven /some/local/dir/somehost-someuser-0 +/some/working/dir/repositoryName-source /some/local/dir/somehost-someuser-0 /tmp/hive-ptest-units/TestExecutionPhase/logs/failed/driver /some/local/dir/somehost-someuser-0/logs/ /tmp/hive-ptest-units/TestExecutionPhase/scratch/hiveptest-driver.sh /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver.sh -bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver.sh \ No newline at end of file +bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver.sh +killall -q -9 -f java || true +mkdir -p /some/local/dir/somehost-someuser-0/logs /some/local/dir/somehost-someuser-0/maven /some/local/dir/somehost-someuser-0/scratch /some/local/dir/somehost-someuser-0/ivy /some/local/dir/somehost-someuser-0/repositoryName-source +mkdir -p /some/local/dir/somehost-someuser-1/logs /some/local/dir/somehost-someuser-1/maven /some/local/dir/somehost-someuser-1/scratch /some/local/dir/somehost-someuser-1/ivy /some/local/dir/somehost-someuser-1/repositoryName-source +rm -rf /some/local/dir/somehost-someuser-0/scratch /some/local/dir/somehost-someuser-0/logs +rm -rf /some/local/dir/somehost-someuser-1/scratch /some/local/dir/somehost-someuser-1/logs +rsync -qaPe --delete --delete-during --force /some/local/dir/somehost-someuser-0/ivy /some/local/dir/somehost-someuser-1/ +rsync -qaPe --delete --delete-during --force /some/local/dir/somehost-someuser-0/maven /some/local/dir/somehost-someuser-1/ +rsync -qaPe --delete --delete-during --force /some/local/dir/somehost-someuser-0/repositoryName-source /some/local/dir/somehost-someuser-1/ \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingQFileTest.approved.txt URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingQFileTest.approved.txt?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingQFileTest.approved.txt (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingQFileTest.approved.txt Fri Aug 16 01:21:54 2013 @@ -1,3 +1,14 @@ +/some/working/dir/ivy /some/local/dir/somehost-someuser-0 +/some/working/dir/maven /some/local/dir/somehost-someuser-0 +/some/working/dir/repositoryName-source /some/local/dir/somehost-someuser-0 /tmp/hive-ptest-units/TestExecutionPhase/logs/succeeded/driver-sometest /some/local/dir/somehost-someuser-0/logs/ /tmp/hive-ptest-units/TestExecutionPhase/scratch/hiveptest-driver-sometest.sh /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-sometest.sh -bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-sometest.sh \ No newline at end of file +bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-sometest.sh +killall -q -9 -f java || true +mkdir -p /some/local/dir/somehost-someuser-0/logs /some/local/dir/somehost-someuser-0/maven /some/local/dir/somehost-someuser-0/scratch /some/local/dir/somehost-someuser-0/ivy /some/local/dir/somehost-someuser-0/repositoryName-source +mkdir -p /some/local/dir/somehost-someuser-1/logs /some/local/dir/somehost-someuser-1/maven /some/local/dir/somehost-someuser-1/scratch /some/local/dir/somehost-someuser-1/ivy /some/local/dir/somehost-someuser-1/repositoryName-source +rm -rf /some/local/dir/somehost-someuser-0/scratch /some/local/dir/somehost-someuser-0/logs +rm -rf /some/local/dir/somehost-someuser-1/scratch /some/local/dir/somehost-someuser-1/logs +rsync -qaPe --delete --delete-during --force /some/local/dir/somehost-someuser-0/ivy /some/local/dir/somehost-someuser-1/ +rsync -qaPe --delete --delete-during --force /some/local/dir/somehost-someuser-0/maven /some/local/dir/somehost-someuser-1/ +rsync -qaPe --delete --delete-during --force /some/local/dir/somehost-someuser-0/repositoryName-source /some/local/dir/somehost-someuser-1/ \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingUnitTest.approved.txt URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingUnitTest.approved.txt?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingUnitTest.approved.txt (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingUnitTest.approved.txt Fri Aug 16 01:21:54 2013 @@ -1,3 +1,14 @@ +/some/working/dir/ivy /some/local/dir/somehost-someuser-0 +/some/working/dir/maven /some/local/dir/somehost-someuser-0 +/some/working/dir/repositoryName-source /some/local/dir/somehost-someuser-0 /tmp/hive-ptest-units/TestExecutionPhase/logs/succeeded/driver /some/local/dir/somehost-someuser-0/logs/ /tmp/hive-ptest-units/TestExecutionPhase/scratch/hiveptest-driver.sh /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver.sh -bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver.sh \ No newline at end of file +bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver.sh +killall -q -9 -f java || true +mkdir -p /some/local/dir/somehost-someuser-0/logs /some/local/dir/somehost-someuser-0/maven /some/local/dir/somehost-someuser-0/scratch /some/local/dir/somehost-someuser-0/ivy /some/local/dir/somehost-someuser-0/repositoryName-source +mkdir -p /some/local/dir/somehost-someuser-1/logs /some/local/dir/somehost-someuser-1/maven /some/local/dir/somehost-someuser-1/scratch /some/local/dir/somehost-someuser-1/ivy /some/local/dir/somehost-someuser-1/repositoryName-source +rm -rf /some/local/dir/somehost-someuser-0/scratch /some/local/dir/somehost-someuser-0/logs +rm -rf /some/local/dir/somehost-someuser-1/scratch /some/local/dir/somehost-someuser-1/logs +rsync -qaPe --delete --delete-during --force /some/local/dir/somehost-someuser-0/ivy /some/local/dir/somehost-someuser-1/ +rsync -qaPe --delete --delete-during --force /some/local/dir/somehost-someuser-0/maven /some/local/dir/somehost-someuser-1/ +rsync -qaPe --delete --delete-during --force /some/local/dir/somehost-someuser-0/repositoryName-source /some/local/dir/somehost-someuser-1/ \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java Fri Aug 16 01:21:54 2013 @@ -148,7 +148,7 @@ public class TestHostExecutor { } @Test public void testBasic() - throws Exception { + throws Exception { HostExecutor executor = createHostExecutor(); parallelWorkQueue.addAll(Lists.newArrayList(testBatchParallel1, testBatchParallel2)); parallelWorkQueue.addAll(Lists.newArrayList(testBatchIsolated1, testBatchIsolated2)); @@ -158,7 +158,7 @@ public class TestHostExecutor { } @Test public void testParallelFailsOnExec() - throws Exception { + throws Exception { sshCommandExecutor.putFailure("bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-parallel-1.sh", Constants.EXIT_CODE_UNKNOWN); HostExecutor executor = createHostExecutor(); @@ -170,7 +170,7 @@ public class TestHostExecutor { } @Test public void testIsolatedFailsOnExec() - throws Exception { + throws Exception { sshCommandExecutor.putFailure("bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-1.sh", Constants.EXIT_CODE_UNKNOWN); HostExecutor executor = createHostExecutor(); @@ -182,7 +182,7 @@ public class TestHostExecutor { } @Test public void testParallelFailsOnRsync() - throws Exception { + throws Exception { rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-parallel-1.sh " + "/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-parallel-1.sh", Constants.EXIT_CODE_UNKNOWN); HostExecutor executor = createHostExecutor(); @@ -194,7 +194,7 @@ public class TestHostExecutor { } @Test public void testIsolatedFailsOnRsyncUnknown() - throws Exception { + throws Exception { rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-1.sh "+ "/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-1.sh", Constants.EXIT_CODE_UNKNOWN); HostExecutor executor = createHostExecutor(); @@ -206,7 +206,7 @@ public class TestHostExecutor { } @Test public void testIsolatedFailsOnRsyncOne() - throws Exception { + throws Exception { rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-1.sh "+ "/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-1.sh", 1); HostExecutor executor = createHostExecutor(); Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPhase.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPhase.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPhase.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPhase.java Fri Aug 16 01:21:54 2013 @@ -46,10 +46,10 @@ public class TestPhase extends AbstractT public void testExecLocallyFails() throws Throwable { phase = new Phase(hostExecutors, localCommandFactory, templateDefaults, logger) { - @Override - public void execute() throws Exception { - execLocally("local"); - } + @Override + public void execute() throws Exception { + execLocally("local"); + } }; when(localCommand.getExitCode()).thenReturn(1); phase.execute(); @@ -58,10 +58,10 @@ public class TestPhase extends AbstractT public void testExecLocallySucceeds() throws Throwable { phase = new Phase(hostExecutors, localCommandFactory, templateDefaults, logger) { - @Override - public void execute() throws Exception { - execLocally("local"); - } + @Override + public void execute() throws Exception { + execLocally("local"); + } }; phase.execute(); List commands = localCommandFactory.getCommands(); @@ -73,10 +73,10 @@ public class TestPhase extends AbstractT sshCommandExecutor.putFailure("echo", Constants.EXIT_CODE_UNKNOWN); phase = new Phase(hostExecutors, localCommandFactory, templateDefaults, logger) { - @Override - public void execute() throws Exception { - execInstances("echo"); - } + @Override + public void execute() throws Exception { + execInstances("echo"); + } }; phase.execute(); Approvals.verify(getExecutedCommands()); @@ -87,10 +87,10 @@ public class TestPhase extends AbstractT sshCommandExecutor.putFailure("echo", Constants.EXIT_CODE_UNKNOWN); phase = new Phase(hostExecutors, localCommandFactory, templateDefaults, logger) { - @Override - public void execute() throws Exception { - execHosts("echo"); - } + @Override + public void execute() throws Exception { + execHosts("echo"); + } }; phase.execute(); Approvals.verify(getExecutedCommands()); @@ -101,10 +101,10 @@ public class TestPhase extends AbstractT rsyncCommandExecutor.putFailure("local remote", Constants.EXIT_CODE_UNKNOWN); phase = new Phase(hostExecutors, localCommandFactory, templateDefaults, logger) { - @Override - public void execute() throws Exception { - rsyncFromLocalToRemoteInstances("local", "remote"); - } + @Override + public void execute() throws Exception { + rsyncFromLocalToRemoteInstances("local", "remote"); + } }; phase.execute(); Approvals.verify(getExecutedCommands()); @@ -115,10 +115,10 @@ public class TestPhase extends AbstractT rsyncCommandExecutor.putFailure("local remote", 1); phase = new Phase(hostExecutors, localCommandFactory, templateDefaults, logger) { - @Override - public void execute() throws Exception { - rsyncFromLocalToRemoteInstances("local", "remote"); - } + @Override + public void execute() throws Exception { + rsyncFromLocalToRemoteInstances("local", "remote"); + } }; phase.execute(); Approvals.verify(getExecutedCommands()); Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPhase.testRsyncFromLocalToRemoteInstancesWithFailureOne.approved.txt URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPhase.testRsyncFromLocalToRemoteInstancesWithFailureOne.approved.txt?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPhase.testRsyncFromLocalToRemoteInstancesWithFailureOne.approved.txt (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPhase.testRsyncFromLocalToRemoteInstancesWithFailureOne.approved.txt Fri Aug 16 01:21:54 2013 @@ -1,2 +1,3 @@ local remote -local remote \ No newline at end of file +local remote +rsync -qaPe --delete --delete-during --force remote/local remote \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPhase.testRsyncFromLocalToRemoteInstancesWithFailureUnknown.approved.txt URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPhase.testRsyncFromLocalToRemoteInstancesWithFailureUnknown.approved.txt?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPhase.testRsyncFromLocalToRemoteInstancesWithFailureUnknown.approved.txt (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPhase.testRsyncFromLocalToRemoteInstancesWithFailureUnknown.approved.txt Fri Aug 16 01:21:54 2013 @@ -1,2 +1,3 @@ local remote -local remote \ No newline at end of file +local remote +rsync -qaPe --delete --delete-during --force remote/local remote \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPrepPhase.testExecute.approved.txt URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPrepPhase.testExecute.approved.txt?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPrepPhase.testExecute.approved.txt (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestPrepPhase.testExecute.approved.txt Fri Aug 16 01:21:54 2013 @@ -1,12 +1,3 @@ -/some/working/dir/ivy /some/local/dir/somehost-someuser-0/ -/some/working/dir/ivy /some/local/dir/somehost-someuser-1/ -/some/working/dir/maven /some/local/dir/somehost-someuser-0/ -/some/working/dir/maven /some/local/dir/somehost-someuser-1/ -/some/working/dir/repositoryName-source /some/local/dir/somehost-someuser-0/ -/some/working/dir/repositoryName-source /some/local/dir/somehost-someuser-1/ bash /tmp/hive-ptest-units/TestPrepPhase/source-prep.sh -mkdir -p /some/local/dir/somehost-someuser-0/ivy /some/local/dir/somehost-someuser-0/repositoryName-source -mkdir -p /some/local/dir/somehost-someuser-0/logs /some/local/dir/somehost-someuser-0/maven /some/local/dir/somehost-someuser-0/scratch -mkdir -p /some/local/dir/somehost-someuser-1/ivy /some/local/dir/somehost-someuser-1/repositoryName-source -mkdir -p /some/local/dir/somehost-someuser-1/logs /some/local/dir/somehost-someuser-1/maven /some/local/dir/somehost-someuser-1/scratch -mkdir -p /some/working/dir/scratch \ No newline at end of file +mkdir -p /some/working/dir/scratch +rm -rf /some/working/dir/scratch \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java Fri Aug 16 01:21:54 2013 @@ -60,8 +60,27 @@ public class TestReportParser { Assert.assertEquals(3, parser.getFailedTests().size()); Assert.assertEquals(Sets. newHashSet("org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_skewjoin_union_remove_1", - "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_union_remove_9", - "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_skewjoin"), - parser.getFailedTests()); + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_union_remove_9", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_skewjoin"), + parser.getFailedTests()); + Assert.assertEquals(Sets. + newHashSet("org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_shutdown", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_binary_constant", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_skewjoin_union_remove_1", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_udf_regexp_extract", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_index_auth", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_auto_join17", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_authorization_2", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_load_dyn_part3", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_index_bitmap2", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_groupby_rollup1", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_bucketcontext_3", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_ppd_join", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_rcfile_lazydecompress", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_notable_alias1", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_union_remove_9", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_skewjoin", + "org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_multi_insert_gby"), + parser.getExecutedTests()); } } Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.java Fri Aug 16 01:21:54 2013 @@ -68,6 +68,7 @@ public class TestScripts { templateVariables.put("instanceName", "instance-1"); templateVariables.put("batchName","batch-1"); templateVariables.put("numOfFailedTests", "20"); + templateVariables.put("maxSourceDirs", String.valueOf(5)); templateVariables.put("testArguments", "-Dtest=arg1"); templateVariables.put("clearLibraryCache", "true"); templateVariables.put("javaHome", "/usr/java/jdk1.7"); Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepGit.approved.txt Fri Aug 16 01:21:54 2013 @@ -78,10 +78,9 @@ cd /some/working/dir/ fi if [[ "true" == "true" ]] then - rm -rf /some/working/dir/ivy /some/working/dir/maven - mkdir /some/working/dir/ivy /some/working/dir/maven + rm -rf /some/working/dir/ivy /some/working/dir/maven + mkdir /some/working/dir/ivy /some/working/dir/maven fi - ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven clean package - ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven -Dtestcase=nothing test + ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven clean package test -Dtestcase=nothing ) 2>&1 | tee /some/log/dir/source-prep.txt -exit ${PIPESTATUS[0]} +exit ${PIPESTATUS[0]} \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepNone.approved.txt Fri Aug 16 01:21:54 2013 @@ -78,10 +78,9 @@ cd /some/working/dir/ fi if [[ "false" == "true" ]] then - rm -rf /some/working/dir/ivy /some/working/dir/maven - mkdir /some/working/dir/ivy /some/working/dir/maven + rm -rf /some/working/dir/ivy /some/working/dir/maven + mkdir /some/working/dir/ivy /some/working/dir/maven fi - ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven clean package - ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven -Dtestcase=nothing test + ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven clean package test -Dtestcase=nothing ) 2>&1 | tee /some/log/dir/source-prep.txt -exit ${PIPESTATUS[0]} +exit ${PIPESTATUS[0]} \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== --- hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt (original) +++ hive/branches/vectorization/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestScripts.testPrepSvn.approved.txt Fri Aug 16 01:21:54 2013 @@ -78,10 +78,9 @@ cd /some/working/dir/ fi if [[ "true" == "true" ]] then - rm -rf /some/working/dir/ivy /some/working/dir/maven - mkdir /some/working/dir/ivy /some/working/dir/maven + rm -rf /some/working/dir/ivy /some/working/dir/maven + mkdir /some/working/dir/ivy /some/working/dir/maven fi - ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven clean package - ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven -Dtestcase=nothing test + ant -Dant=arg1 -Divy.default.ivy.user.dir=/some/working/dir/ivy -Dmvn.local.repo=/some/working/dir/maven clean package test -Dtestcase=nothing ) 2>&1 | tee /some/log/dir/source-prep.txt -exit ${PIPESTATUS[0]} +exit ${PIPESTATUS[0]} \ No newline at end of file Modified: hive/branches/vectorization/testutils/ptest2/src/test/resources/SomeTest-success.xml URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/resources/SomeTest-success.xml?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== Binary files - no diff available. Modified: hive/branches/vectorization/testutils/ptest2/src/test/resources/test-outputs/SomeTest-truncated.xml URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/resources/test-outputs/SomeTest-truncated.xml?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== Binary files - no diff available. Modified: hive/branches/vectorization/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml URL: http://svn.apache.org/viewvc/hive/branches/vectorization/testutils/ptest2/src/test/resources/test-outputs/skewjoin_union_remove_1.q-TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml?rev=1514554&r1=1514553&r2=1514554&view=diff ============================================================================== Binary files - no diff available.