commit 5d4ce178f864b10148be807344f2e9c54e15ce89 Author: Owen O'Malley Date: Thu Apr 3 11:33:55 2014 +0200 BUG-15888. Fix Streaming tests to use RawFileSystem instead of pfile. diff --git streaming/src/test/org/apache/hive/streaming/TestStreaming.java streaming/src/test/org/apache/hive/streaming/TestStreaming.java index 93f06cb..4f191f7 100644 --- streaming/src/test/org/apache/hive/streaming/TestStreaming.java +++ streaming/src/test/org/apache/hive/streaming/TestStreaming.java @@ -19,7 +19,11 @@ package org.apache.hive.streaming; import junit.framework.Assert; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RawLocalFileSystem; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hive.common.ValidTxnList; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; @@ -46,19 +50,71 @@ import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.util.Shell; import org.apache.thrift.TException; import org.junit.Before; import org.junit.Test; +import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.StringTokenizer; public class TestStreaming { + public static class RawFileSystem extends RawLocalFileSystem { + private static final URI NAME; + static { + try { + NAME = new URI("raw:///"); + } catch (URISyntaxException se) { + throw new IllegalArgumentException("bad uri", se); + } + } + + @Override + public URI getUri() { + return NAME; + } + + static String execCommand(File f, String... cmd) throws IOException { + String[] args = new String[cmd.length + 1]; + System.arraycopy(cmd, 0, args, 0, cmd.length); + args[cmd.length] = f.getCanonicalPath(); + String output = Shell.execCommand(args); + return output; + } + + @Override + public FileStatus getFileStatus(Path path) throws IOException { + File file = pathToFile(path); + if (!file.exists()) { + throw new FileNotFoundException("Can't find " + path); + } + // get close enough + short mod = 0; + if (file.canRead()) { + mod |= 0444; + } + if (file.canWrite()) { + mod |= 0200; + } + if (file.canExecute()) { + mod |= 0111; + } + return new FileStatus(file.length(), file.isDirectory(), 1, 1024, + file.lastModified(), file.lastModified(), + FsPermission.createImmutable(mod), "owen", "users", null, path); + } + } + private static final String COL1 = "id"; private static final String COL2 = "msg"; @@ -92,6 +148,7 @@ public TestStreaming() throws Exception { //port = MetaStoreUtils.findFreePort(); conf = new HiveConf(this.getClass()); + conf.set("fs.raw.impl", RawFileSystem.class.getName()); TxnDbUtil.setConfValues(conf); if(metaStoreURI!=null) { conf.setVar(HiveConf.ConfVars.METASTOREURIS, metaStoreURI); @@ -113,10 +170,10 @@ public TestStreaming() throws Exception { public void setup() throws Exception { // drop and recreate the necessary databases and tables dropDB(msClient, dbName); - createDbAndTable(msClient, dbName, tblName, partitionVals); + createDbAndTable(msClient, dbName, tblName, partitionVals, conf); dropDB(msClient, dbName2); - createDbAndTable(msClient, dbName2, tblName2, partitionVals); + createDbAndTable(msClient, dbName2, tblName2, partitionVals, conf); } private void printResults(ArrayList res) { @@ -678,7 +735,8 @@ public static void dropDB(IMetaStoreClient client, String databaseName) { } public static void createDbAndTable(IMetaStoreClient client, String databaseName, - String tableName, List partVals) + String tableName, List partVals, + HiveConf conf) throws Exception { Database db = new Database(); db.setName(databaseName); @@ -691,6 +749,10 @@ public static void createDbAndTable(IMetaStoreClient client, String databaseName StorageDescriptor sd = new StorageDescriptor(); sd.setCols(getTableColumns()); sd.setNumBuckets(1); + Path root = new Path("raw:///tmp/streaming"); + FileSystem fs = root.getFileSystem(conf); + fs.delete(root, true); + sd.setLocation(root + "/" + databaseName + "/" + tableName); tbl.setPartitionKeys(getPartitionKeys()); tbl.setSd(sd);