You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by gu...@apache.org on 2013/10/29 18:25:59 UTC

svn commit: r1536823 [8/8] - in /hive/branches/tez: ./ cli/src/test/org/apache/hadoop/hive/cli/ common/src/java/conf/ common/src/java/org/apache/hadoop/hive/common/ common/src/java/org/apache/hadoop/hive/common/type/ common/src/java/org/apache/hadoop/h...

Modified: hive/branches/tez/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (original)
+++ hive/branches/tez/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java Tue Oct 29 17:25:55 2013
@@ -210,7 +210,7 @@ public abstract class ThriftCLIService e
           resp.setOperationHandle(operationHandle.toTOperationHandle());
           resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-       LOG.warn("Error fetching results: ", e);
+       LOG.warn("Error executing statement: ", e);
        resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -224,7 +224,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(operationHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error executing statement: ", e);
+      LOG.warn("Error getting type info: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -238,7 +238,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting type info: ", e);
+      LOG.warn("Error getting catalogs: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -253,7 +253,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting catalogs: ", e);
+      LOG.warn("Error getting schemas: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -269,7 +269,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting schemas: ", e);
+      LOG.warn("Error getting tables: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -283,7 +283,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting tables: ", e);
+      LOG.warn("Error getting table types: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -302,7 +302,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting table types: ", e);
+      LOG.warn("Error getting columns: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -318,7 +318,7 @@ public abstract class ThriftCLIService e
       resp.setOperationHandle(opHandle.toTOperationHandle());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting columns: ", e);
+      LOG.warn("Error getting functions: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -332,7 +332,7 @@ public abstract class ThriftCLIService e
       resp.setOperationState(operationState.toTOperationState());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting functions: ", e);
+      LOG.warn("Error getting operation status: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -345,7 +345,7 @@ public abstract class ThriftCLIService e
       cliService.cancelOperation(new OperationHandle(req.getOperationHandle()));
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting operation status: ", e);
+      LOG.warn("Error cancelling operation: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -358,7 +358,7 @@ public abstract class ThriftCLIService e
       cliService.closeOperation(new OperationHandle(req.getOperationHandle()));
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error canceling operation: ", e);
+      LOG.warn("Error closing operation: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -373,7 +373,7 @@ public abstract class ThriftCLIService e
       resp.setSchema(schema.toTTableSchema());
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error closing operation: ", e);
+      LOG.warn("Error getting result set metadata: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;
@@ -391,7 +391,7 @@ public abstract class ThriftCLIService e
       resp.setHasMoreRows(false);
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {
-      LOG.warn("Error getting result set metadata: ", e);
+      LOG.warn("Error fetching results: ", e);
       resp.setStatus(HiveSQLException.toTStatus(e));
     }
     return resp;

Modified: hive/branches/tez/service/src/test/org/apache/hive/service/auth/TestCustomAuthentication.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/test/org/apache/hive/service/auth/TestCustomAuthentication.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/service/src/test/org/apache/hive/service/auth/TestCustomAuthentication.java (original)
+++ hive/branches/tez/service/src/test/org/apache/hive/service/auth/TestCustomAuthentication.java Tue Oct 29 17:25:55 2013
@@ -26,102 +26,72 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import javax.security.sasl.AuthenticationException;
+import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.sql.Connection;
 import java.sql.DriverManager;
+import java.sql.SQLException;
 import java.util.HashMap;
 import java.util.Map;
 
 public class TestCustomAuthentication {
 
-  private static HiveServer2 hiveserver2 = null;
-
-  private static File configFile = null;
+  private static HiveServer2 hiveserver2;
+  private static HiveConf hiveConf;
+  private static byte[] hiveConfBackup;
 
   @BeforeClass
   public static void setUp() throws Exception {
-    createConfig();
-    startServer();
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    stopServer();
-    removeConfig();
-  }
-
-  private static void startServer() throws Exception{
-
-    HiveConf hiveConf = new HiveConf();
+    hiveConf = new HiveConf();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    hiveConf.writeXml(baos);
+    baos.close();
+    hiveConfBackup = baos.toByteArray();
+    hiveConf.set("hive.server2.authentication", "CUSTOM");
+    hiveConf.set("hive.server2.custom.authentication.class",
+        "org.apache.hive.service.auth.TestCustomAuthentication$SimpleAuthenticationProviderImpl");
+    FileOutputStream fos = new FileOutputStream(new File(hiveConf.getHiveSiteLocation().toURI()));
+    hiveConf.writeXml(fos);
+    fos.close();
     hiveserver2 = new HiveServer2();
     hiveserver2.init(hiveConf);
     hiveserver2.start();
     Thread.sleep(1000);
     System.out.println("hiveServer2 start ......");
-
   }
 
-  private static void stopServer(){
-    try {
-      if (hiveserver2 != null) {
-        hiveserver2.stop();
-        hiveserver2 = null;
-      }
-      Thread.sleep(1000);
-    } catch (Exception e) {
-      e.printStackTrace();
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if(hiveConf != null && hiveConfBackup != null) {
+      FileOutputStream fos = new FileOutputStream(new File(hiveConf.getHiveSiteLocation().toURI()));
+      fos.write(hiveConfBackup);
+      fos.close();
+    }
+    if (hiveserver2 != null) {
+      hiveserver2.stop();
+      hiveserver2 = null;
     }
+    Thread.sleep(1000);
     System.out.println("hiveServer2 stop ......");
   }
 
-  private static void createConfig() throws Exception{
-
-    Configuration conf = new Configuration(false);
-    conf.set("hive.server2.authentication", "CUSTOM");
-    conf.set("hive.server2.custom.authentication.class",
-        "org.apache.hive.service.auth.TestCustomAuthentication$SimpleAuthenticationProviderImpl");
-
-    configFile = new File("../build/service/test/resources","hive-site.xml");
-
-    FileOutputStream out = new FileOutputStream(configFile);
-    conf.writeXml(out);
-  }
-
-  private static void removeConfig(){
-    try {
-      configFile.delete();
-    } catch (Exception e){
-      System.out.println(e.getMessage());
-    }
-  }
-
   @Test
-  public void testCustomAuthentication() throws Exception{
+  public void testCustomAuthentication() throws Exception {
 
     String url = "jdbc:hive2://localhost:10000/default";
+    Class.forName("org.apache.hive.jdbc.HiveDriver");
 
-    Exception exception = null;
-    try{
-      Class.forName("org.apache.hive.jdbc.HiveDriver");
-      Connection connection =  DriverManager.getConnection(url, "wronguser", "pwd");
-      connection.close();
-    } catch (Exception e){
-      exception = e;
-    }
-
-    Assert.assertNotNull(exception);
-
-    exception = null;
-    try{
-      Class.forName("org.apache.hive.jdbc.HiveDriver");
-      Connection connection =  DriverManager.getConnection(url, "hiveuser", "hive");
-      connection.close();
-    } catch (Exception e){
-      exception = e;
+    try {
+      DriverManager.getConnection(url, "wronguser", "pwd");
+      Assert.fail("Expected Exception");
+    } catch(SQLException e) {
+      Assert.assertNotNull(e.getMessage());
+      Assert.assertTrue(e.getMessage(), e.getMessage().contains("Peer indicated failure: Error validating the login"));
     }
 
-    Assert.assertNull(exception);
+    Connection connection = DriverManager.getConnection(url, "hiveuser", "hive");
+    connection.close();
 
     System.out.println(">>> PASSED testCustomAuthentication");
   }

Modified: hive/branches/tez/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java (original)
+++ hive/branches/tez/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java Tue Oct 29 17:25:55 2013
@@ -20,10 +20,13 @@ package org.apache.hive.service.cli;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
 
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Map;
 
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -224,4 +227,68 @@ public abstract class CLIServiceTest {
     System.out.println(ophandle + " after cancelling, state= " + state);
     assertEquals("Query should be cancelled", OperationState.CANCELED, state);
   }
+
+  /**
+   * Test per statement configuration overlay.
+   * Create a table using hiveconf: var substitution, with the conf var passed
+   * via confOverlay.Verify the confOverlay works for the query and does set the
+   * value in the session configuration
+   * @throws Exception
+   */
+  @Test
+  public void testConfOverlay() throws Exception {
+    SessionHandle sessionHandle = client.openSession("tom", "password", new HashMap<String, String>());
+    assertNotNull(sessionHandle);
+    String tabName = "TEST_CONF_EXEC";
+    String tabNameVar = "tabNameVar";
+
+    String setLockMgr = "SET " + HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname
+        + " = false";
+    OperationHandle opHandle = client.executeStatement(sessionHandle, setLockMgr, null);
+    client.closeOperation(opHandle);
+
+    String dropTable = "DROP TABLE IF EXISTS " + tabName;
+    opHandle = client.executeStatement(sessionHandle, dropTable, null);
+    client.closeOperation(opHandle);
+
+    // set a pass a property to operation and check if its set the query config
+    Map <String, String> confOverlay = new HashMap<String, String>();
+    confOverlay.put(tabNameVar, tabName);
+
+    // execute statement with the conf overlay
+    String createTab = "CREATE TABLE ${hiveconf:" + tabNameVar + "} (id int)";
+    opHandle = client.executeStatement(sessionHandle, createTab, confOverlay);
+    assertNotNull(opHandle);
+    // query should pass and create the table
+    assertEquals("Query should be finished",
+        OperationState.FINISHED, client.getOperationStatus(opHandle));
+    client.closeOperation(opHandle);
+
+    // select from  the new table should pass
+    String selectTab = "SELECT * FROM " + tabName;
+    opHandle = client.executeStatement(sessionHandle, selectTab, null);
+    assertNotNull(opHandle);
+    // query should pass and create the table
+    assertEquals("Query should be finished",
+        OperationState.FINISHED, client.getOperationStatus(opHandle));
+    client.closeOperation(opHandle);
+
+    // the settings in confoverly should not be part of session config
+    // another query referring that property with the conf overlay should fail
+    selectTab = "SELECT * FROM ${hiveconf:" + tabNameVar + "}";
+    try {
+      opHandle = client.executeStatement(sessionHandle, selectTab, null);
+      fail("Query should fail");
+    } catch (HiveSQLException e) {
+      // Expected exception
+    }
+
+    // cleanup
+    dropTable = "DROP TABLE IF EXISTS " + tabName;
+    opHandle = client.executeStatement(sessionHandle, dropTable, null);
+    client.closeOperation(opHandle);
+
+
+    client.closeSession(sessionHandle);
+  }
 }

Modified: hive/branches/tez/service/src/test/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/test/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/service/src/test/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java (original)
+++ hive/branches/tez/service/src/test/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java Tue Oct 29 17:25:55 2013
@@ -30,8 +30,7 @@ import org.junit.BeforeClass;
  *
  */
 public class TestEmbeddedThriftBinaryCLIService extends CLIServiceTest {
-
-  private static ThriftCLIService service;
+  protected static ThriftCLIService service;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {

Modified: hive/branches/tez/shims/src/0.20S/java/org/apache/hadoop/mapred/WebHCatJTShim20S.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/shims/src/0.20S/java/org/apache/hadoop/mapred/WebHCatJTShim20S.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/shims/src/0.20S/java/org/apache/hadoop/mapred/WebHCatJTShim20S.java (original)
+++ hive/branches/tez/shims/src/0.20S/java/org/apache/hadoop/mapred/WebHCatJTShim20S.java Tue Oct 29 17:25:55 2013
@@ -18,82 +18,89 @@
 package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.hive.shims.HadoopShims.WebHCatJTShim;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.net.URI;
 
 /**
  * This is in org.apache.hadoop.mapred package because it relies on 
  * JobSubmissionProtocol which is package private
  */
 public class WebHCatJTShim20S implements WebHCatJTShim {
-    private JobSubmissionProtocol cnx;
+  private JobSubmissionProtocol cnx;
 
-    /**
-     * Create a connection to the Job Tracker.
-     */
-    public WebHCatJTShim20S(Configuration conf, UserGroupInformation ugi)
-            throws IOException {
-      cnx = (JobSubmissionProtocol)
-              RPC.getProxy(JobSubmissionProtocol.class,
-                      JobSubmissionProtocol.versionID,
-                      getAddress(conf),
-                      ugi,
-                      conf,
-                      NetUtils.getSocketFactory(conf,
-                              JobSubmissionProtocol.class));
-    }
-
-    /**
-     * Grab a handle to a job that is already known to the JobTracker.
-     *
-     * @return Profile of the job, or null if not found.
-     */
-    public JobProfile getJobProfile(org.apache.hadoop.mapred.JobID jobid)
-            throws IOException {
-      return cnx.getJobProfile(jobid);
-    }
-
-    /**
-     * Grab a handle to a job that is already known to the JobTracker.
-     *
-     * @return Status of the job, or null if not found.
-     */
-    public org.apache.hadoop.mapred.JobStatus getJobStatus(org.apache.hadoop.mapred.JobID jobid)
-            throws IOException {
-      return cnx.getJobStatus(jobid);
-    }
-
-
-    /**
-     * Kill a job.
-     */
-    public void killJob(org.apache.hadoop.mapred.JobID jobid)
-            throws IOException {
-      cnx.killJob(jobid);
-    }
-
-    /**
-     * Get all the jobs submitted.
-     */
-    public org.apache.hadoop.mapred.JobStatus[] getAllJobs()
-            throws IOException {
-      return cnx.getAllJobs();
-    }
-
-    /**
-     * Close the connection to the Job Tracker.
-     */
-    public void close() {
-      RPC.stopProxy(cnx);
-    }
-    private InetSocketAddress getAddress(Configuration conf) {
-      String jobTrackerStr = conf.get("mapred.job.tracker", "localhost:8012");
-      return NetUtils.createSocketAddr(jobTrackerStr);
-    }
+  /**
+   * Create a connection to the Job Tracker.
+   */
+  public WebHCatJTShim20S(Configuration conf, UserGroupInformation ugi)
+          throws IOException {
+    cnx = (JobSubmissionProtocol)
+            RPC.getProxy(JobSubmissionProtocol.class,
+                    JobSubmissionProtocol.versionID,
+                    getAddress(conf),
+                    ugi,
+                    conf,
+                    NetUtils.getSocketFactory(conf,
+                            JobSubmissionProtocol.class));
   }
 
+  /**
+   * Grab a handle to a job that is already known to the JobTracker.
+   *
+   * @return Profile of the job, or null if not found.
+   */
+  public JobProfile getJobProfile(org.apache.hadoop.mapred.JobID jobid)
+          throws IOException {
+    return cnx.getJobProfile(jobid);
+  }
+
+  /**
+   * Grab a handle to a job that is already known to the JobTracker.
+   *
+   * @return Status of the job, or null if not found.
+   */
+  public org.apache.hadoop.mapred.JobStatus getJobStatus(org.apache.hadoop.mapred.JobID jobid)
+          throws IOException {
+    return cnx.getJobStatus(jobid);
+  }
+
+
+  /**
+   * Kill a job.
+   */
+  public void killJob(org.apache.hadoop.mapred.JobID jobid)
+          throws IOException {
+    cnx.killJob(jobid);
+  }
+
+  /**
+   * Get all the jobs submitted.
+   */
+  public org.apache.hadoop.mapred.JobStatus[] getAllJobs()
+          throws IOException {
+    return cnx.getAllJobs();
+  }
+
+  /**
+   * Close the connection to the Job Tracker.
+   */
+  public void close() {
+    RPC.stopProxy(cnx);
+  }
+  private InetSocketAddress getAddress(Configuration conf) {
+    String jobTrackerStr = conf.get("mapred.job.tracker", "localhost:8012");
+    return NetUtils.createSocketAddr(jobTrackerStr);
+  }
+  @Override
+  public void addCacheFile(URI uri, Job job) {
+    DistributedCache.addCacheFile(uri, job.getConfiguration());
+  }
+}
+

Modified: hive/branches/tez/shims/src/0.23/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/shims/src/0.23/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/shims/src/0.23/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java (original)
+++ hive/branches/tez/shims/src/0.23/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java Tue Oct 29 17:25:55 2013
@@ -88,7 +88,6 @@ public class Hadoop23Shims extends Hadoo
 
   @Override
   public JobTrackerState getJobTrackerState(ClusterStatus clusterStatus) throws Exception {
-    JobTrackerState state;
     switch (clusterStatus.getJobTrackerStatus()) {
     case INITIALIZING:
       return JobTrackerState.INITIALIZING;
@@ -212,19 +211,16 @@ public class Hadoop23Shims extends Hadoo
     public void shutdown() throws IOException {
       mr.shutdown();
     }
-    
+
     @Override
     public void setupConfiguration(Configuration conf) {
       JobConf jConf = mr.createJobConf();
       for (Map.Entry<String, String> pair: jConf) {
-	//System.out.println("XXX Var: "+pair.getKey() +"="+pair.getValue());
-        //if (conf.get(pair.getKey()) == null) {
-          conf.set(pair.getKey(), pair.getValue());
-	  //}
+        conf.set(pair.getKey(), pair.getValue());
       }
     }
   }
-  
+
   // Don't move this code to the parent class. There's a binary
   // incompatibility between hadoop 1 and 2 wrt MiniDFSCluster and we
   // need to have two different shim classes even though they are

Modified: hive/branches/tez/shims/src/0.23/java/org/apache/hadoop/mapred/WebHCatJTShim23.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/shims/src/0.23/java/org/apache/hadoop/mapred/WebHCatJTShim23.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/shims/src/0.23/java/org/apache/hadoop/mapred/WebHCatJTShim23.java (original)
+++ hive/branches/tez/shims/src/0.23/java/org/apache/hadoop/mapred/WebHCatJTShim23.java Tue Oct 29 17:25:55 2013
@@ -18,10 +18,12 @@
 package org.apache.hadoop.mapred;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.hive.shims.HadoopShims.WebHCatJTShim;
 
 import java.io.IOException;
+import java.net.URI;
 
 public class WebHCatJTShim23 implements WebHCatJTShim {
   private JobClient jc;
@@ -88,4 +90,8 @@ public class WebHCatJTShim23 implements 
     } catch (IOException e) {
     }
   }
+  @Override
+  public void addCacheFile(URI uri, Job job) {
+    job.addCacheFile(uri);
+  }
 }

Modified: hive/branches/tez/shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java (original)
+++ hive/branches/tez/shims/src/common/java/org/apache/hadoop/hive/shims/HadoopShims.java Tue Oct 29 17:25:55 2013
@@ -561,6 +561,11 @@ public interface HadoopShims {
      * Close the connection to the Job Tracker.
      */
     public void close();
+    /**
+     * Does exactly what org.apache.hadoop.mapreduce.Job#addCacheFile(URI) in Hadoop 2.
+     * Assumes that both parameters are not {@code null}.
+     */
+    public void addCacheFile(URI uri, Job job);
   }
 
   /**

Modified: hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java?rev=1536823&r1=1536822&r2=1536823&view=diff
==============================================================================
--- hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java (original)
+++ hive/branches/tez/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestReportParser.java Tue Oct 29 17:25:55 2013
@@ -50,10 +50,12 @@ public class TestReportParser {
   public void test() throws Exception {
     File reportDir = new File("src/test/resources/test-outputs");
     for(File file : reportDir.listFiles()) {
-      if(file.getName().endsWith(".xml")) {
-        Files.copy(file, new File(baseDir, "TEST-" + file.getName()));
-      } else {
-        Files.copy(file, new File(baseDir, file.getName()));
+      if(file.isFile()) {
+        if(file.getName().endsWith(".xml")) {
+          Files.copy(file, new File(baseDir, "TEST-" + file.getName()));
+        } else {
+          Files.copy(file, new File(baseDir, file.getName()));
+        }
       }
     }
     JUnitReportParser parser = new JUnitReportParser(LOG, baseDir);